2020-07-06 20:15:13 +00:00
|
|
|
package stream
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"fmt"
|
|
|
|
"testing"
|
|
|
|
"time"
|
|
|
|
|
|
|
|
"github.com/stretchr/testify/require"
|
2020-11-05 22:57:25 +00:00
|
|
|
|
|
|
|
"github.com/hashicorp/consul/acl"
|
2023-02-17 21:14:46 +00:00
|
|
|
"github.com/hashicorp/consul/proto/private/pbsubscribe"
|
2022-05-10 20:25:51 +00:00
|
|
|
"github.com/hashicorp/consul/sdk/testutil"
|
2020-07-06 20:15:13 +00:00
|
|
|
)
|
|
|
|
|
2020-07-07 00:04:24 +00:00
|
|
|
type intTopic int
|
|
|
|
|
|
|
|
func (i intTopic) String() string {
|
|
|
|
return fmt.Sprintf("%d", i)
|
|
|
|
}
|
|
|
|
|
|
|
|
var testTopic Topic = intTopic(999)
|
2020-07-06 20:15:13 +00:00
|
|
|
|
2020-10-01 20:34:00 +00:00
|
|
|
func TestEventPublisher_SubscribeWithIndex0(t *testing.T) {
|
|
|
|
req := &SubscribeRequest{
|
2022-04-05 14:26:14 +00:00
|
|
|
Topic: testTopic,
|
2022-04-19 17:03:03 +00:00
|
|
|
Subject: StringSubject("sub-key"),
|
2020-07-06 20:15:13 +00:00
|
|
|
}
|
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
publisher := NewEventPublisher(0)
|
|
|
|
registerTestSnapshotHandlers(t, publisher)
|
2020-09-09 20:26:11 +00:00
|
|
|
go publisher.Run(ctx)
|
|
|
|
|
2020-10-01 20:34:00 +00:00
|
|
|
sub, err := publisher.Subscribe(req)
|
2020-07-06 20:15:13 +00:00
|
|
|
require.NoError(t, err)
|
2020-10-15 22:06:04 +00:00
|
|
|
defer sub.Unsubscribe()
|
2020-10-01 20:34:00 +00:00
|
|
|
eventCh := runSubscription(ctx, sub)
|
2020-07-06 20:15:13 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, testSnapshotEvent, next)
|
2020-07-06 20:15:13 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot())
|
2020-07-06 20:15:13 +00:00
|
|
|
|
|
|
|
assertNoResult(t, eventCh)
|
|
|
|
|
2020-07-06 22:44:51 +00:00
|
|
|
events := []Event{{
|
|
|
|
Topic: testTopic,
|
2020-10-27 18:40:06 +00:00
|
|
|
Payload: simplePayload{key: "sub-key", value: "the-published-event-payload"},
|
2020-07-06 22:44:51 +00:00
|
|
|
}}
|
2020-07-08 04:31:22 +00:00
|
|
|
publisher.Publish(events)
|
2020-07-06 20:15:13 +00:00
|
|
|
|
|
|
|
// Subscriber should see the published event
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
2020-10-27 18:40:06 +00:00
|
|
|
expected := Event{
|
|
|
|
Topic: testTopic,
|
|
|
|
Payload: simplePayload{key: "sub-key", value: "the-published-event-payload"},
|
|
|
|
}
|
2020-10-01 20:34:00 +00:00
|
|
|
require.Equal(t, expected, next)
|
2022-01-28 12:27:00 +00:00
|
|
|
|
|
|
|
// Subscriber should not see events for other keys
|
|
|
|
publisher.Publish([]Event{{
|
|
|
|
Topic: testTopic,
|
|
|
|
Payload: simplePayload{key: "other-key", value: "this-should-not-reach-the-subscriber"},
|
|
|
|
}})
|
|
|
|
assertNoResult(t, eventCh)
|
2020-10-01 20:34:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
var testSnapshotEvent = Event{
|
|
|
|
Topic: testTopic,
|
2020-10-27 18:40:06 +00:00
|
|
|
Payload: simplePayload{key: "sub-key", value: "snapshot-event-payload"},
|
2020-10-01 20:34:00 +00:00
|
|
|
Index: 1,
|
2020-07-06 20:15:13 +00:00
|
|
|
}
|
|
|
|
|
2020-10-27 18:40:06 +00:00
|
|
|
type simplePayload struct {
|
2020-11-05 22:57:25 +00:00
|
|
|
key string
|
|
|
|
value string
|
|
|
|
noReadPerm bool
|
2020-10-27 18:40:06 +00:00
|
|
|
}
|
|
|
|
|
2020-11-05 22:57:25 +00:00
|
|
|
func (p simplePayload) HasReadPermission(acl.Authorizer) bool {
|
|
|
|
return !p.noReadPerm
|
|
|
|
}
|
|
|
|
|
2022-04-19 17:03:03 +00:00
|
|
|
func (p simplePayload) Subject() Subject { return StringSubject(p.key) }
|
2022-01-28 12:27:00 +00:00
|
|
|
|
peering: initial sync (#12842)
- Add endpoints related to peering: read, list, generate token, initiate peering
- Update node/service/check table indexing to account for peers
- Foundational changes for pushing service updates to a peer
- Plumb peer name through Health.ServiceNodes path
see: ENT-1765, ENT-1280, ENT-1283, ENT-1283, ENT-1756, ENT-1739, ENT-1750, ENT-1679,
ENT-1709, ENT-1704, ENT-1690, ENT-1689, ENT-1702, ENT-1701, ENT-1683, ENT-1663,
ENT-1650, ENT-1678, ENT-1628, ENT-1658, ENT-1640, ENT-1637, ENT-1597, ENT-1634,
ENT-1613, ENT-1616, ENT-1617, ENT-1591, ENT-1588, ENT-1596, ENT-1572, ENT-1555
Co-authored-by: R.B. Boyer <rb@hashicorp.com>
Co-authored-by: freddygv <freddy@hashicorp.com>
Co-authored-by: Chris S. Kim <ckim@hashicorp.com>
Co-authored-by: Evan Culver <eculver@hashicorp.com>
Co-authored-by: Nitya Dhanushkodi <nitya@hashicorp.com>
2022-04-21 22:34:40 +00:00
|
|
|
func (p simplePayload) ToSubscriptionEvent(idx uint64) *pbsubscribe.Event {
|
|
|
|
panic("simplePayload does not implement ToSubscriptionEvent")
|
|
|
|
}
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
func registerTestSnapshotHandlers(t *testing.T, publisher *EventPublisher) {
|
|
|
|
t.Helper()
|
|
|
|
|
|
|
|
testTopicHandler := func(req SubscribeRequest, buf SnapshotAppender) (uint64, error) {
|
|
|
|
if req.Topic != testTopic {
|
|
|
|
return 0, fmt.Errorf("unexpected topic: %v", req.Topic)
|
|
|
|
}
|
|
|
|
buf.Append([]Event{testSnapshotEvent})
|
|
|
|
return 1, nil
|
2020-07-06 20:15:13 +00:00
|
|
|
}
|
2022-04-12 13:47:42 +00:00
|
|
|
|
proxycfg: server-local config entry data sources
This is the OSS portion of enterprise PR 2056.
This commit provides server-local implementations of the proxycfg.ConfigEntry
and proxycfg.ConfigEntryList interfaces, that source data from streaming events.
It makes use of the LocalMaterializer type introduced for peering replication,
adding the necessary support for authorization.
It also adds support for "wildcard" subscriptions (within a topic) to the event
publisher, as this is needed to fetch service-resolvers for all services when
configuring mesh gateways.
Currently, events will be emitted for just the ingress-gateway, service-resolver,
and mesh config entry types, as these are the only entries required by proxycfg
— the events will be emitted on topics named IngressGateway, ServiceResolver,
and MeshConfig topics respectively.
Though these events will only be consumed "locally" for now, they can also be
consumed via the gRPC endpoint (confirmed using grpcurl) so using them from
client agents should be a case of swapping the LocalMaterializer for an
RPCMaterializer.
2022-07-01 15:09:47 +00:00
|
|
|
require.NoError(t, publisher.RegisterHandler(testTopic, testTopicHandler, false))
|
2020-07-06 20:15:13 +00:00
|
|
|
}
|
|
|
|
|
2020-10-01 20:34:00 +00:00
|
|
|
func runSubscription(ctx context.Context, sub *Subscription) <-chan eventOrErr {
|
|
|
|
eventCh := make(chan eventOrErr, 1)
|
2020-07-06 20:15:13 +00:00
|
|
|
go func() {
|
|
|
|
for {
|
2020-07-08 04:31:22 +00:00
|
|
|
es, err := sub.Next(ctx)
|
2020-10-01 20:34:00 +00:00
|
|
|
eventCh <- eventOrErr{
|
2020-10-05 16:38:38 +00:00
|
|
|
Event: es,
|
|
|
|
Err: err,
|
2020-07-06 20:15:13 +00:00
|
|
|
}
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
return eventCh
|
|
|
|
}
|
|
|
|
|
2020-10-01 20:34:00 +00:00
|
|
|
type eventOrErr struct {
|
2020-10-05 16:38:38 +00:00
|
|
|
Event Event
|
|
|
|
Err error
|
2020-07-06 20:15:13 +00:00
|
|
|
}
|
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
func getNextEvent(t *testing.T, eventCh <-chan eventOrErr) Event {
|
2020-07-06 20:15:13 +00:00
|
|
|
t.Helper()
|
|
|
|
select {
|
|
|
|
case next := <-eventCh:
|
2020-10-01 20:34:00 +00:00
|
|
|
require.NoError(t, next.Err)
|
2020-10-05 16:38:38 +00:00
|
|
|
return next.Event
|
2020-07-06 20:15:13 +00:00
|
|
|
case <-time.After(100 * time.Millisecond):
|
2020-10-01 20:34:00 +00:00
|
|
|
t.Fatalf("timeout waiting for event from subscription")
|
2020-10-05 16:38:38 +00:00
|
|
|
return Event{}
|
2020-07-06 20:15:13 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-10-01 20:34:00 +00:00
|
|
|
func assertNoResult(t *testing.T, eventCh <-chan eventOrErr) {
|
2020-07-06 20:15:13 +00:00
|
|
|
t.Helper()
|
|
|
|
select {
|
|
|
|
case next := <-eventCh:
|
|
|
|
require.NoError(t, next.Err)
|
2020-10-05 16:38:38 +00:00
|
|
|
t.Fatalf("received unexpected event: %#v", next.Event.Payload)
|
2020-10-01 20:34:00 +00:00
|
|
|
case <-time.After(25 * time.Millisecond):
|
2020-07-06 20:15:13 +00:00
|
|
|
}
|
|
|
|
}
|
2020-07-16 19:26:26 +00:00
|
|
|
|
|
|
|
func TestEventPublisher_ShutdownClosesSubscriptions(t *testing.T) {
|
|
|
|
ctx, cancel := context.WithCancel(context.Background())
|
|
|
|
t.Cleanup(cancel)
|
|
|
|
|
2020-07-14 23:23:44 +00:00
|
|
|
fn := func(req SubscribeRequest, buf SnapshotAppender) (uint64, error) {
|
2020-07-16 19:26:26 +00:00
|
|
|
return 0, nil
|
|
|
|
}
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
publisher := NewEventPublisher(time.Second)
|
|
|
|
registerTestSnapshotHandlers(t, publisher)
|
proxycfg: server-local config entry data sources
This is the OSS portion of enterprise PR 2056.
This commit provides server-local implementations of the proxycfg.ConfigEntry
and proxycfg.ConfigEntryList interfaces, that source data from streaming events.
It makes use of the LocalMaterializer type introduced for peering replication,
adding the necessary support for authorization.
It also adds support for "wildcard" subscriptions (within a topic) to the event
publisher, as this is needed to fetch service-resolvers for all services when
configuring mesh gateways.
Currently, events will be emitted for just the ingress-gateway, service-resolver,
and mesh config entry types, as these are the only entries required by proxycfg
— the events will be emitted on topics named IngressGateway, ServiceResolver,
and MeshConfig topics respectively.
Though these events will only be consumed "locally" for now, they can also be
consumed via the gRPC endpoint (confirmed using grpcurl) so using them from
client agents should be a case of swapping the LocalMaterializer for an
RPCMaterializer.
2022-07-01 15:09:47 +00:00
|
|
|
publisher.RegisterHandler(intTopic(22), fn, false)
|
|
|
|
publisher.RegisterHandler(intTopic(33), fn, false)
|
2020-09-09 20:26:11 +00:00
|
|
|
go publisher.Run(ctx)
|
2020-07-16 19:26:26 +00:00
|
|
|
|
2022-04-05 14:26:14 +00:00
|
|
|
sub1, err := publisher.Subscribe(&SubscribeRequest{Topic: intTopic(22), Subject: SubjectNone})
|
2020-07-16 19:26:26 +00:00
|
|
|
require.NoError(t, err)
|
|
|
|
defer sub1.Unsubscribe()
|
|
|
|
|
2022-04-05 14:26:14 +00:00
|
|
|
sub2, err := publisher.Subscribe(&SubscribeRequest{Topic: intTopic(33), Subject: SubjectNone})
|
2020-07-16 19:26:26 +00:00
|
|
|
require.NoError(t, err)
|
|
|
|
defer sub2.Unsubscribe()
|
|
|
|
|
|
|
|
cancel() // Shutdown
|
|
|
|
|
|
|
|
err = consumeSub(context.Background(), sub1)
|
2022-05-23 14:00:06 +00:00
|
|
|
require.Equal(t, err, ErrShuttingDown)
|
2020-07-16 19:26:26 +00:00
|
|
|
|
|
|
|
_, err = sub2.Next(context.Background())
|
2022-05-23 14:00:06 +00:00
|
|
|
require.Equal(t, err, ErrShuttingDown)
|
2020-07-16 19:26:26 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func consumeSub(ctx context.Context, sub *Subscription) error {
|
|
|
|
for {
|
2020-10-05 16:38:38 +00:00
|
|
|
event, err := sub.Next(ctx)
|
2020-07-16 19:26:26 +00:00
|
|
|
switch {
|
|
|
|
case err != nil:
|
|
|
|
return err
|
2020-10-05 16:38:38 +00:00
|
|
|
case event.IsEndOfSnapshot():
|
2020-07-16 19:26:26 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2020-10-01 20:34:00 +00:00
|
|
|
|
|
|
|
func TestEventPublisher_SubscribeWithIndex0_FromCache(t *testing.T) {
|
|
|
|
req := &SubscribeRequest{
|
2022-04-05 14:26:14 +00:00
|
|
|
Topic: testTopic,
|
2022-04-19 17:03:03 +00:00
|
|
|
Subject: StringSubject("sub-key"),
|
2020-10-01 20:34:00 +00:00
|
|
|
}
|
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
publisher := NewEventPublisher(time.Second)
|
|
|
|
registerTestSnapshotHandlers(t, publisher)
|
2020-10-01 20:34:00 +00:00
|
|
|
go publisher.Run(ctx)
|
2022-01-28 12:27:00 +00:00
|
|
|
|
2020-10-15 22:06:04 +00:00
|
|
|
sub, err := publisher.Subscribe(req)
|
2020-10-01 20:34:00 +00:00
|
|
|
require.NoError(t, err)
|
2022-01-28 12:27:00 +00:00
|
|
|
defer sub.Unsubscribe()
|
2020-10-01 20:34:00 +00:00
|
|
|
|
|
|
|
publisher.snapshotHandlers[testTopic] = func(_ SubscribeRequest, _ SnapshotAppender) (uint64, error) {
|
|
|
|
return 0, fmt.Errorf("error should not be seen, cache should have been used")
|
|
|
|
}
|
|
|
|
|
2020-10-15 22:06:04 +00:00
|
|
|
sub, err = publisher.Subscribe(req)
|
2020-10-01 20:34:00 +00:00
|
|
|
require.NoError(t, err)
|
2022-01-28 12:27:00 +00:00
|
|
|
defer sub.Unsubscribe()
|
2020-10-01 20:34:00 +00:00
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
2020-10-05 16:38:38 +00:00
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, testSnapshotEvent, next)
|
2020-10-01 20:34:00 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot())
|
2020-10-01 20:34:00 +00:00
|
|
|
|
|
|
|
// Now subscriber should block waiting for updates
|
|
|
|
assertNoResult(t, eventCh)
|
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
expected := Event{
|
2020-10-01 20:34:00 +00:00
|
|
|
Topic: testTopic,
|
2020-10-27 18:40:06 +00:00
|
|
|
Payload: simplePayload{key: "sub-key", value: "the-published-event-payload"},
|
2020-10-01 20:34:00 +00:00
|
|
|
Index: 3,
|
2020-10-05 16:38:38 +00:00
|
|
|
}
|
|
|
|
publisher.Publish([]Event{expected})
|
2020-10-01 20:34:00 +00:00
|
|
|
|
|
|
|
// Subscriber should see the published event
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
2020-10-01 20:34:00 +00:00
|
|
|
require.Equal(t, expected, next)
|
|
|
|
}
|
|
|
|
|
|
|
|
func TestEventPublisher_SubscribeWithIndexNotZero_CanResume(t *testing.T) {
|
|
|
|
req := &SubscribeRequest{
|
2022-04-05 14:26:14 +00:00
|
|
|
Topic: testTopic,
|
2022-04-19 17:03:03 +00:00
|
|
|
Subject: StringSubject("sub-key"),
|
2020-10-01 20:34:00 +00:00
|
|
|
}
|
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
publisher := NewEventPublisher(time.Second)
|
|
|
|
registerTestSnapshotHandlers(t, publisher)
|
2020-10-01 20:34:00 +00:00
|
|
|
go publisher.Run(ctx)
|
2022-01-28 12:27:00 +00:00
|
|
|
|
|
|
|
simulateExistingSubscriber(t, publisher, req)
|
|
|
|
|
|
|
|
// Publish the testSnapshotEvent, to ensure that it is skipped over when
|
|
|
|
// splicing the topic buffer onto the snapshot.
|
2020-10-01 20:34:00 +00:00
|
|
|
publisher.publishEvent([]Event{testSnapshotEvent})
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "start a subscription and unsub", func(t *testing.T) {
|
2020-10-01 20:34:00 +00:00
|
|
|
sub, err := publisher.Subscribe(req)
|
|
|
|
require.NoError(t, err)
|
|
|
|
defer sub.Unsubscribe()
|
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, testSnapshotEvent, next)
|
2020-10-01 20:34:00 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot())
|
|
|
|
require.Equal(t, uint64(1), next.Index)
|
2020-10-01 20:34:00 +00:00
|
|
|
})
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "resume the subscription", func(t *testing.T) {
|
2020-10-01 20:34:00 +00:00
|
|
|
newReq := *req
|
|
|
|
newReq.Index = 1
|
|
|
|
sub, err := publisher.Subscribe(&newReq)
|
|
|
|
require.NoError(t, err)
|
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
|
|
|
assertNoResult(t, eventCh)
|
|
|
|
|
|
|
|
expected := Event{
|
|
|
|
Topic: testTopic,
|
|
|
|
Index: 3,
|
2020-10-27 18:40:06 +00:00
|
|
|
Payload: simplePayload{key: "sub-key", value: "event-3"},
|
2020-10-01 20:34:00 +00:00
|
|
|
}
|
|
|
|
publisher.publishEvent([]Event{expected})
|
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, expected, next)
|
2020-10-01 20:34:00 +00:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
func TestEventPublisher_SubscribeWithIndexNotZero_NewSnapshot(t *testing.T) {
|
|
|
|
req := &SubscribeRequest{
|
2022-04-05 14:26:14 +00:00
|
|
|
Topic: testTopic,
|
2022-04-19 17:03:03 +00:00
|
|
|
Subject: StringSubject("sub-key"),
|
2020-10-01 20:34:00 +00:00
|
|
|
}
|
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
publisher := NewEventPublisher(0)
|
|
|
|
registerTestSnapshotHandlers(t, publisher)
|
2020-10-01 20:34:00 +00:00
|
|
|
go publisher.Run(ctx)
|
|
|
|
// Include the same event in the topicBuffer
|
|
|
|
publisher.publishEvent([]Event{testSnapshotEvent})
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "start a subscription and unsub", func(t *testing.T) {
|
2020-10-01 20:34:00 +00:00
|
|
|
sub, err := publisher.Subscribe(req)
|
|
|
|
require.NoError(t, err)
|
|
|
|
defer sub.Unsubscribe()
|
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, testSnapshotEvent, next)
|
2020-10-01 20:34:00 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot())
|
|
|
|
require.Equal(t, uint64(1), next.Index)
|
2020-10-01 20:34:00 +00:00
|
|
|
})
|
|
|
|
|
|
|
|
nextEvent := Event{
|
|
|
|
Topic: testTopic,
|
|
|
|
Index: 3,
|
2020-10-27 18:40:06 +00:00
|
|
|
Payload: simplePayload{key: "sub-key", value: "event-3"},
|
2020-10-01 20:34:00 +00:00
|
|
|
}
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "publish an event while unsubed", func(t *testing.T) {
|
2020-10-01 20:34:00 +00:00
|
|
|
publisher.publishEvent([]Event{nextEvent})
|
|
|
|
})
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "resume the subscription", func(t *testing.T) {
|
2020-10-01 20:34:00 +00:00
|
|
|
newReq := *req
|
|
|
|
newReq.Index = 1
|
|
|
|
sub, err := publisher.Subscribe(&newReq)
|
|
|
|
require.NoError(t, err)
|
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
2020-10-05 16:38:38 +00:00
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsNewSnapshotToFollow(), next)
|
2020-10-01 20:34:00 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, testSnapshotEvent, next)
|
2020-10-01 20:34:00 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot())
|
2020-10-01 20:34:00 +00:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
func TestEventPublisher_SubscribeWithIndexNotZero_NewSnapshotFromCache(t *testing.T) {
|
|
|
|
req := &SubscribeRequest{
|
2022-04-05 14:26:14 +00:00
|
|
|
Topic: testTopic,
|
2022-04-19 17:03:03 +00:00
|
|
|
Subject: StringSubject("sub-key"),
|
2020-10-01 20:34:00 +00:00
|
|
|
}
|
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
publisher := NewEventPublisher(time.Second)
|
|
|
|
registerTestSnapshotHandlers(t, publisher)
|
2020-10-01 20:34:00 +00:00
|
|
|
go publisher.Run(ctx)
|
2022-01-28 12:27:00 +00:00
|
|
|
|
|
|
|
simulateExistingSubscriber(t, publisher, req)
|
|
|
|
|
|
|
|
// Publish the testSnapshotEvent, to ensure that it is skipped over when
|
|
|
|
// splicing the topic buffer onto the snapshot.
|
2020-10-01 20:34:00 +00:00
|
|
|
publisher.publishEvent([]Event{testSnapshotEvent})
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "start a subscription and unsub", func(t *testing.T) {
|
2020-10-01 20:34:00 +00:00
|
|
|
sub, err := publisher.Subscribe(req)
|
|
|
|
require.NoError(t, err)
|
|
|
|
defer sub.Unsubscribe()
|
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, testSnapshotEvent, next)
|
2020-10-01 20:34:00 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot())
|
|
|
|
require.Equal(t, uint64(1), next.Index)
|
2020-10-01 20:34:00 +00:00
|
|
|
})
|
|
|
|
|
|
|
|
nextEvent := Event{
|
|
|
|
Topic: testTopic,
|
|
|
|
Index: 3,
|
2020-10-27 18:40:06 +00:00
|
|
|
Payload: simplePayload{key: "sub-key", value: "event-3"},
|
2020-10-01 20:34:00 +00:00
|
|
|
}
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "publish an event while unsubed", func(t *testing.T) {
|
2020-10-01 20:34:00 +00:00
|
|
|
publisher.publishEvent([]Event{nextEvent})
|
|
|
|
})
|
|
|
|
|
|
|
|
publisher.snapshotHandlers[testTopic] = func(_ SubscribeRequest, _ SnapshotAppender) (uint64, error) {
|
|
|
|
return 0, fmt.Errorf("error should not be seen, cache should have been used")
|
|
|
|
}
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "resume the subscription", func(t *testing.T) {
|
2020-10-01 20:34:00 +00:00
|
|
|
newReq := *req
|
|
|
|
newReq.Index = 1
|
|
|
|
sub, err := publisher.Subscribe(&newReq)
|
|
|
|
require.NoError(t, err)
|
2020-10-15 22:06:04 +00:00
|
|
|
defer sub.Unsubscribe()
|
2020-10-01 20:34:00 +00:00
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
2020-10-05 16:38:38 +00:00
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsNewSnapshotToFollow(), next)
|
2020-10-01 20:34:00 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, testSnapshotEvent, next)
|
2020-10-01 20:34:00 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot())
|
2020-10-01 20:34:00 +00:00
|
|
|
|
2020-10-05 16:38:38 +00:00
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, nextEvent, next)
|
2020-10-01 20:34:00 +00:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2021-02-16 16:54:51 +00:00
|
|
|
func TestEventPublisher_SubscribeWithIndexNotZero_NewSnapshot_WithCache(t *testing.T) {
|
|
|
|
req := &SubscribeRequest{
|
2022-04-05 14:26:14 +00:00
|
|
|
Topic: testTopic,
|
2022-04-19 17:03:03 +00:00
|
|
|
Subject: StringSubject("sub-key"),
|
2022-04-05 14:26:14 +00:00
|
|
|
Index: 1,
|
2021-02-16 16:54:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
nextEvent := Event{
|
|
|
|
Topic: testTopic,
|
|
|
|
Index: 3,
|
|
|
|
Payload: simplePayload{key: "sub-key", value: "event-3"},
|
|
|
|
}
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
testTopicHandler := func(req SubscribeRequest, buf SnapshotAppender) (uint64, error) {
|
|
|
|
if req.Topic != testTopic {
|
|
|
|
return 0, fmt.Errorf("unexpected topic: %v", req.Topic)
|
|
|
|
}
|
|
|
|
buf.Append([]Event{testSnapshotEvent})
|
|
|
|
buf.Append([]Event{nextEvent})
|
|
|
|
return 3, nil
|
2021-02-16 16:54:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
publisher := NewEventPublisher(time.Second)
|
proxycfg: server-local config entry data sources
This is the OSS portion of enterprise PR 2056.
This commit provides server-local implementations of the proxycfg.ConfigEntry
and proxycfg.ConfigEntryList interfaces, that source data from streaming events.
It makes use of the LocalMaterializer type introduced for peering replication,
adding the necessary support for authorization.
It also adds support for "wildcard" subscriptions (within a topic) to the event
publisher, as this is needed to fetch service-resolvers for all services when
configuring mesh gateways.
Currently, events will be emitted for just the ingress-gateway, service-resolver,
and mesh config entry types, as these are the only entries required by proxycfg
— the events will be emitted on topics named IngressGateway, ServiceResolver,
and MeshConfig topics respectively.
Though these events will only be consumed "locally" for now, they can also be
consumed via the gRPC endpoint (confirmed using grpcurl) so using them from
client agents should be a case of swapping the LocalMaterializer for an
RPCMaterializer.
2022-07-01 15:09:47 +00:00
|
|
|
publisher.RegisterHandler(testTopic, testTopicHandler, false)
|
2021-02-16 16:54:51 +00:00
|
|
|
go publisher.Run(ctx)
|
2022-01-28 12:27:00 +00:00
|
|
|
|
|
|
|
simulateExistingSubscriber(t, publisher, req)
|
|
|
|
|
|
|
|
// Publish the events, to ensure they are is skipped over when splicing the
|
|
|
|
// topic buffer onto the snapshot.
|
2021-02-16 16:54:51 +00:00
|
|
|
publisher.publishEvent([]Event{testSnapshotEvent})
|
|
|
|
publisher.publishEvent([]Event{nextEvent})
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "start a subscription and unsub", func(t *testing.T) {
|
2021-02-16 16:54:51 +00:00
|
|
|
sub, err := publisher.Subscribe(req)
|
|
|
|
require.NoError(t, err)
|
|
|
|
defer sub.Unsubscribe()
|
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsNewSnapshotToFollow(), next)
|
|
|
|
|
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, testSnapshotEvent, next)
|
|
|
|
|
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, nextEvent, next)
|
|
|
|
|
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot(), next)
|
|
|
|
require.Equal(t, uint64(3), next.Index)
|
|
|
|
})
|
|
|
|
|
|
|
|
publisher.snapshotHandlers[testTopic] = func(_ SubscribeRequest, _ SnapshotAppender) (uint64, error) {
|
|
|
|
return 0, fmt.Errorf("error should not be seen, cache should have been used")
|
|
|
|
}
|
|
|
|
|
2022-05-10 20:25:51 +00:00
|
|
|
testutil.RunStep(t, "resume the subscription", func(t *testing.T) {
|
2021-02-16 16:54:51 +00:00
|
|
|
newReq := *req
|
|
|
|
newReq.Index = 0
|
|
|
|
sub, err := publisher.Subscribe(&newReq)
|
|
|
|
require.NoError(t, err)
|
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, testSnapshotEvent, next)
|
|
|
|
|
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, nextEvent, next)
|
|
|
|
|
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot())
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2020-10-15 22:06:04 +00:00
|
|
|
func TestEventPublisher_Unsubscribe_ClosesSubscription(t *testing.T) {
|
|
|
|
req := &SubscribeRequest{
|
2022-04-05 14:26:14 +00:00
|
|
|
Topic: testTopic,
|
2022-04-19 17:03:03 +00:00
|
|
|
Subject: StringSubject("sub-key"),
|
2020-10-15 22:06:04 +00:00
|
|
|
}
|
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
|
|
|
|
defer cancel()
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
publisher := NewEventPublisher(time.Second)
|
|
|
|
registerTestSnapshotHandlers(t, publisher)
|
2020-10-15 22:06:04 +00:00
|
|
|
|
|
|
|
sub, err := publisher.Subscribe(req)
|
|
|
|
require.NoError(t, err)
|
|
|
|
|
|
|
|
_, err = sub.Next(ctx)
|
|
|
|
require.NoError(t, err)
|
|
|
|
|
|
|
|
sub.Unsubscribe()
|
|
|
|
_, err = sub.Next(ctx)
|
|
|
|
require.Error(t, err)
|
|
|
|
require.Contains(t, err.Error(), "subscription was closed by unsubscribe")
|
|
|
|
}
|
2022-01-28 12:27:00 +00:00
|
|
|
|
|
|
|
func TestEventPublisher_Unsubscribe_FreesResourcesWhenThereAreNoSubscribers(t *testing.T) {
|
|
|
|
req := &SubscribeRequest{
|
2022-04-05 14:26:14 +00:00
|
|
|
Topic: testTopic,
|
2022-04-19 17:03:03 +00:00
|
|
|
Subject: StringSubject("sub-key"),
|
2022-01-28 12:27:00 +00:00
|
|
|
}
|
|
|
|
|
2022-04-12 13:47:42 +00:00
|
|
|
publisher := NewEventPublisher(time.Second)
|
|
|
|
registerTestSnapshotHandlers(t, publisher)
|
2022-01-28 12:27:00 +00:00
|
|
|
|
|
|
|
sub1, err := publisher.Subscribe(req)
|
|
|
|
require.NoError(t, err)
|
|
|
|
|
|
|
|
// Expect a topic buffer and snapshot to have been created.
|
|
|
|
publisher.lock.Lock()
|
|
|
|
require.NotNil(t, publisher.topicBuffers[req.topicSubject()])
|
|
|
|
require.NotNil(t, publisher.snapCache[req.topicSubject()])
|
|
|
|
publisher.lock.Unlock()
|
|
|
|
|
|
|
|
// Create another subscription and close the old one, to ensure the buffer and
|
|
|
|
// snapshot stick around as long as there's at least one subscriber.
|
|
|
|
sub2, err := publisher.Subscribe(req)
|
|
|
|
require.NoError(t, err)
|
|
|
|
|
|
|
|
sub1.Unsubscribe()
|
|
|
|
|
|
|
|
publisher.lock.Lock()
|
|
|
|
require.NotNil(t, publisher.topicBuffers[req.topicSubject()])
|
|
|
|
require.NotNil(t, publisher.snapCache[req.topicSubject()])
|
|
|
|
publisher.lock.Unlock()
|
|
|
|
|
|
|
|
// Close the other subscription and expect the buffer and snapshot to have
|
|
|
|
// been cleaned up.
|
|
|
|
sub2.Unsubscribe()
|
|
|
|
|
|
|
|
publisher.lock.Lock()
|
|
|
|
require.Nil(t, publisher.topicBuffers[req.topicSubject()])
|
|
|
|
require.Nil(t, publisher.snapCache[req.topicSubject()])
|
|
|
|
publisher.lock.Unlock()
|
|
|
|
}
|
|
|
|
|
|
|
|
// simulateExistingSubscriber creates a subscription that remains open throughout
|
|
|
|
// a test to prevent the topic buffer getting garbage-collected.
|
|
|
|
//
|
|
|
|
// It evicts the created snapshot from the cache immediately (simulating an
|
|
|
|
// existing subscription that has been open long enough the snapshot's TTL has
|
|
|
|
// been reached) so you can test snapshots getting created afresh.
|
|
|
|
func simulateExistingSubscriber(t *testing.T, p *EventPublisher, r *SubscribeRequest) {
|
|
|
|
t.Helper()
|
|
|
|
|
|
|
|
sub, err := p.Subscribe(r)
|
|
|
|
require.NoError(t, err)
|
|
|
|
t.Cleanup(sub.Unsubscribe)
|
|
|
|
|
|
|
|
p.lock.Lock()
|
|
|
|
delete(p.snapCache, r.topicSubject())
|
|
|
|
p.lock.Unlock()
|
|
|
|
}
|
proxycfg: server-local config entry data sources
This is the OSS portion of enterprise PR 2056.
This commit provides server-local implementations of the proxycfg.ConfigEntry
and proxycfg.ConfigEntryList interfaces, that source data from streaming events.
It makes use of the LocalMaterializer type introduced for peering replication,
adding the necessary support for authorization.
It also adds support for "wildcard" subscriptions (within a topic) to the event
publisher, as this is needed to fetch service-resolvers for all services when
configuring mesh gateways.
Currently, events will be emitted for just the ingress-gateway, service-resolver,
and mesh config entry types, as these are the only entries required by proxycfg
— the events will be emitted on topics named IngressGateway, ServiceResolver,
and MeshConfig topics respectively.
Though these events will only be consumed "locally" for now, they can also be
consumed via the gRPC endpoint (confirmed using grpcurl) so using them from
client agents should be a case of swapping the LocalMaterializer for an
RPCMaterializer.
2022-07-01 15:09:47 +00:00
|
|
|
|
|
|
|
func TestEventPublisher_Subscribe_WildcardNotSupported(t *testing.T) {
|
|
|
|
publisher := NewEventPublisher(0)
|
|
|
|
|
|
|
|
handler := func(SubscribeRequest, SnapshotAppender) (uint64, error) { return 0, nil }
|
|
|
|
require.NoError(t, publisher.RegisterHandler(testTopic, handler, false))
|
|
|
|
|
|
|
|
_, err := publisher.Subscribe(&SubscribeRequest{
|
|
|
|
Topic: testTopic,
|
|
|
|
Subject: SubjectWildcard,
|
|
|
|
})
|
|
|
|
require.Error(t, err)
|
|
|
|
require.Contains(t, err.Error(), "does not support wildcard subscriptions")
|
|
|
|
}
|
|
|
|
|
|
|
|
func TestEventPublisher_Subscribe_WildcardSupported(t *testing.T) {
|
|
|
|
ctx, cancel := context.WithCancel(context.Background())
|
|
|
|
t.Cleanup(cancel)
|
|
|
|
|
|
|
|
publisher := NewEventPublisher(0)
|
|
|
|
go publisher.Run(ctx)
|
|
|
|
|
|
|
|
var (
|
|
|
|
// These events are in the snapshot.
|
|
|
|
a1 = Event{
|
|
|
|
Topic: testTopic,
|
|
|
|
Payload: simplePayload{key: "a", value: "1"},
|
|
|
|
Index: 1,
|
|
|
|
}
|
|
|
|
b1 = Event{
|
|
|
|
Topic: testTopic,
|
|
|
|
Payload: simplePayload{key: "b", value: "1"},
|
|
|
|
Index: 1,
|
|
|
|
}
|
|
|
|
|
|
|
|
// These events are published after the subscription begins.
|
|
|
|
a2 = Event{
|
|
|
|
Topic: testTopic,
|
|
|
|
Payload: simplePayload{key: "a", value: "2"},
|
|
|
|
Index: 2,
|
|
|
|
}
|
|
|
|
b2 = Event{
|
|
|
|
Topic: testTopic,
|
|
|
|
Payload: simplePayload{key: "b", value: "2"},
|
|
|
|
Index: 2,
|
|
|
|
}
|
|
|
|
)
|
|
|
|
|
|
|
|
handler := func(_ SubscribeRequest, buf SnapshotAppender) (uint64, error) {
|
|
|
|
buf.Append([]Event{a1, b1})
|
|
|
|
return 1, nil
|
|
|
|
}
|
|
|
|
require.NoError(t, publisher.RegisterHandler(testTopic, handler, true))
|
|
|
|
|
|
|
|
sub, err := publisher.Subscribe(&SubscribeRequest{
|
|
|
|
Topic: testTopic,
|
|
|
|
Subject: SubjectWildcard,
|
|
|
|
})
|
|
|
|
require.NoError(t, err)
|
|
|
|
t.Cleanup(sub.Unsubscribe)
|
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
|
|
|
|
|
|
|
next := getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, &PayloadEvents{
|
|
|
|
Items: []Event{a1, b1},
|
|
|
|
}, next.Payload)
|
|
|
|
|
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.True(t, next.IsEndOfSnapshot(), "expected end of snapshot")
|
|
|
|
|
|
|
|
publisher.Publish([]Event{a2, b2})
|
|
|
|
next = getNextEvent(t, eventCh)
|
|
|
|
require.Equal(t, &PayloadEvents{
|
|
|
|
Items: []Event{a2, b2},
|
|
|
|
}, next.Payload)
|
|
|
|
}
|
|
|
|
|
|
|
|
func TestEventPublisher_Publish_WildcardNotAllowed(t *testing.T) {
|
|
|
|
publisher := NewEventPublisher(0)
|
|
|
|
|
|
|
|
require.Panics(t, func() {
|
|
|
|
publisher.Publish([]Event{
|
|
|
|
{
|
|
|
|
Topic: testTopic,
|
|
|
|
Payload: wildcardPayload{},
|
|
|
|
},
|
|
|
|
})
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
type wildcardPayload struct{}
|
|
|
|
|
|
|
|
func (wildcardPayload) Subject() Subject { return SubjectWildcard }
|
|
|
|
func (wildcardPayload) HasReadPermission(acl.Authorizer) bool { return true }
|
|
|
|
func (wildcardPayload) ToSubscriptionEvent(uint64) *pbsubscribe.Event { return &pbsubscribe.Event{} }
|
|
|
|
|
|
|
|
func TestEventPublisher_SnapshotIndex0(t *testing.T) {
|
|
|
|
ctx, cancel := context.WithCancel(context.Background())
|
|
|
|
t.Cleanup(cancel)
|
|
|
|
|
|
|
|
publisher := NewEventPublisher(10 * time.Second)
|
|
|
|
go publisher.Run(ctx)
|
|
|
|
|
|
|
|
publisher.RegisterHandler(testTopic, func(SubscribeRequest, SnapshotAppender) (uint64, error) {
|
|
|
|
return 0, nil
|
|
|
|
}, false)
|
|
|
|
|
|
|
|
sub, err := publisher.Subscribe(&SubscribeRequest{
|
|
|
|
Topic: testTopic,
|
|
|
|
Subject: StringSubject("sub-key"),
|
|
|
|
})
|
|
|
|
require.NoError(t, err)
|
|
|
|
t.Cleanup(sub.Unsubscribe)
|
|
|
|
|
|
|
|
eventCh := runSubscription(ctx, sub)
|
|
|
|
event := getNextEvent(t, eventCh)
|
|
|
|
require.True(t, event.IsEndOfSnapshot())
|
|
|
|
|
|
|
|
// Even though the snapshot handler returned 0, the subscriber shouldn't see it.
|
|
|
|
require.Equal(t, uint64(1), event.Index)
|
|
|
|
}
|