From 93fc591f21839a6c1977585028f2bf345c8b2cd9 Mon Sep 17 00:00:00 2001 From: Ali Ok Date: Thu, 30 Sep 2021 16:15:40 +0300 Subject: [PATCH 1/7] Fix Kafka channel event loss during subscription becoming ready --- .../consolidated/dispatcher/dispatcher.go | 23 +- .../dispatcher/dispatcher_it_test.go | 6 +- .../dispatcher/dispatcher_test.go | 20 +- .../dispatcher/subscription_endpoint.go | 60 -- .../dispatcher/subscription_endpoint_test.go | 155 ----- .../reconciler/controller/controller.go | 35 +- .../reconciler/controller/kafkachannel.go | 111 +++- .../controller/kafkachannel_test.go | 272 +++++---- .../reconciler/controller/lister.go | 83 --- .../reconciler/dispatcher/kafkachannel.go | 2 +- pkg/channel/consolidated/status/status.go | 533 ------------------ .../consolidated/status/status_test.go | 340 ----------- .../dispatcher/controller/kafkachannel.go | 6 +- .../controller/kafkachannel_test.go | 2 +- .../dispatcher/dispatcher/dispatcher.go | 8 +- .../dispatcher/dispatcher/dispatcher_test.go | 4 +- pkg/common/consumer/consumer_factory.go | 33 +- pkg/common/consumer/consumer_factory_test.go | 66 ++- pkg/common/consumer/consumer_manager.go | 27 +- pkg/common/consumer/consumer_manager_test.go | 9 +- .../consumer/consumergroup_offsets_checker.go | 74 +++ pkg/common/consumer/testing/mocks.go | 12 +- pkg/common/kafka/offset/offsets.go | 58 +- pkg/common/kafka/offset/offsets_test.go | 212 ++++--- pkg/common/testing/mocks.go | 113 ++++ pkg/source/adapter/adapter.go | 4 +- .../knative.dev/networking/pkg/prober/LICENSE | 201 ------- vendor/knative.dev/networking/LICENSE | 201 ------- .../knative.dev/networking/pkg/prober/doc.go | 18 - .../networking/pkg/prober/prober.go | 199 ------- 30 files changed, 727 insertions(+), 2160 deletions(-) delete mode 100644 pkg/channel/consolidated/dispatcher/subscription_endpoint.go delete mode 100644 pkg/channel/consolidated/dispatcher/subscription_endpoint_test.go delete mode 100644 pkg/channel/consolidated/reconciler/controller/lister.go delete mode 100644 pkg/channel/consolidated/status/status.go delete mode 100644 pkg/channel/consolidated/status/status_test.go create mode 100644 pkg/common/consumer/consumergroup_offsets_checker.go create mode 100644 pkg/common/testing/mocks.go delete mode 100644 third_party/VENDOR-LICENSE/knative.dev/networking/pkg/prober/LICENSE delete mode 100644 vendor/knative.dev/networking/LICENSE delete mode 100644 vendor/knative.dev/networking/pkg/prober/doc.go delete mode 100644 vendor/knative.dev/networking/pkg/prober/prober.go diff --git a/pkg/channel/consolidated/dispatcher/dispatcher.go b/pkg/channel/consolidated/dispatcher/dispatcher.go index 8c0053d25b..e1da740f3f 100644 --- a/pkg/channel/consolidated/dispatcher/dispatcher.go +++ b/pkg/channel/consolidated/dispatcher/dispatcher.go @@ -43,10 +43,6 @@ import ( "knative.dev/eventing-kafka/pkg/common/tracing" ) -const ( - dispatcherReadySubHeader = "K-Subscriber-Status" -) - type TopicFunc func(separator, namespace, name string) string type KafkaDispatcherArgs struct { @@ -86,7 +82,7 @@ func NewDispatcher(ctx context.Context, args *KafkaDispatcherArgs) (*KafkaDispat dispatcher := &KafkaDispatcher{ dispatcher: eventingchannels.NewMessageDispatcher(logging.FromContext(ctx).Desugar()), - kafkaConsumerFactory: consumer.NewConsumerGroupFactory(args.Brokers, args.Config.Sarama.Config), + kafkaConsumerFactory: consumer.NewConsumerGroupFactory(args.Brokers, args.Config.Sarama.Config, &consumer.KafkaConsumerGroupOffsetsChecker{}), channelSubscriptions: make(map[types.NamespacedName]*KafkaSubscription), subsConsumerGroups: make(map[types.UID]sarama.ConsumerGroup), subscriptions: make(map[types.UID]Subscription), @@ -95,15 +91,6 @@ func NewDispatcher(ctx context.Context, args *KafkaDispatcherArgs) (*KafkaDispat topicFunc: args.TopicFunc, } - // initialize and start the subscription endpoint server - subscriptionEndpoint := &subscriptionEndpoint{ - dispatcher: dispatcher, - logger: logging.FromContext(ctx), - } - go func() { - subscriptionEndpoint.start() - }() - podName, err := env.GetRequiredConfigValue(logging.FromContext(ctx).Desugar(), env.PodNameEnvVarKey) if err != nil { return nil, err @@ -171,7 +158,7 @@ func (k UpdateError) Error() string { } // ReconcileConsumers will be called by new CRD based kafka channel dispatcher controller. -func (d *KafkaDispatcher) ReconcileConsumers(config *ChannelConfig) error { +func (d *KafkaDispatcher) ReconcileConsumers(ctx context.Context, config *ChannelConfig) error { channelNamespacedName := types.NamespacedName{ Namespace: config.Namespace, Name: config.Name, @@ -215,7 +202,7 @@ func (d *KafkaDispatcher) ReconcileConsumers(config *ChannelConfig) error { failedToSubscribe := make(UpdateError) for subUid, subSpec := range toAddSubs { - if err := d.subscribe(channelNamespacedName, subSpec); err != nil { + if err := d.subscribe(ctx, channelNamespacedName, subSpec); err != nil { failedToSubscribe[subUid] = err } } @@ -286,7 +273,7 @@ func (d *KafkaDispatcher) CleanupChannel(name, namespace, hostname string) error // subscribe reads kafkaConsumers which gets updated in UpdateConfig in a separate go-routine. // subscribe must be called under updateLock. -func (d *KafkaDispatcher) subscribe(channelRef types.NamespacedName, sub Subscription) error { +func (d *KafkaDispatcher) subscribe(ctx context.Context, channelRef types.NamespacedName, sub Subscription) error { d.logger.Infow("Subscribing to Kafka Channel", zap.Any("channelRef", channelRef), zap.Any("subscription", sub.UID)) topicName := d.topicFunc(utils.KafkaChannelSeparator, channelRef.Namespace, channelRef.Name) @@ -310,7 +297,7 @@ func (d *KafkaDispatcher) subscribe(channelRef types.NamespacedName, sub Subscri } d.logger.Debugw("Starting consumer group", zap.Any("channelRef", channelRef), zap.Any("subscription", sub.UID), zap.String("topic", topicName), zap.String("consumer group", groupID)) - consumerGroup, err := d.kafkaConsumerFactory.StartConsumerGroup(groupID, []string{topicName}, d.logger, handler) + consumerGroup, err := d.kafkaConsumerFactory.StartConsumerGroup(ctx, groupID, []string{topicName}, handler) if err != nil { // we can not create a consumer - logging that, with reason diff --git a/pkg/channel/consolidated/dispatcher/dispatcher_it_test.go b/pkg/channel/consolidated/dispatcher/dispatcher_it_test.go index cbd86d41c2..b58f64e393 100644 --- a/pkg/channel/consolidated/dispatcher/dispatcher_it_test.go +++ b/pkg/channel/consolidated/dispatcher/dispatcher_it_test.go @@ -55,6 +55,8 @@ func TestDispatcher(t *testing.T) { t.Skipf("This test can't run in CI") } + ctx := context.TODO() + logger, err := zap.NewDevelopment(zap.AddStacktrace(zap.WarnLevel)) if err != nil { t.Fatal(err) @@ -179,7 +181,7 @@ func TestDispatcher(t *testing.T) { }, } require.NoError(t, dispatcher.RegisterChannelHost(channelAConfig)) - require.NoError(t, dispatcher.ReconcileConsumers(channelAConfig)) + require.NoError(t, dispatcher.ReconcileConsumers(ctx, channelAConfig)) channelBConfig := &ChannelConfig{ Namespace: "default", @@ -195,7 +197,7 @@ func TestDispatcher(t *testing.T) { }, } require.NoError(t, dispatcher.RegisterChannelHost(channelBConfig)) - require.NoError(t, dispatcher.ReconcileConsumers(channelBConfig)) + require.NoError(t, dispatcher.ReconcileConsumers(ctx, channelBConfig)) time.Sleep(5 * time.Second) diff --git a/pkg/channel/consolidated/dispatcher/dispatcher_test.go b/pkg/channel/consolidated/dispatcher/dispatcher_test.go index 2b34159aec..5245c3d888 100644 --- a/pkg/channel/consolidated/dispatcher/dispatcher_test.go +++ b/pkg/channel/consolidated/dispatcher/dispatcher_test.go @@ -50,7 +50,7 @@ type mockKafkaConsumerFactory struct { createErr bool } -func (c mockKafkaConsumerFactory) StartConsumerGroup(groupID string, topics []string, logger *zap.SugaredLogger, handler consumer.KafkaConsumerHandler, options ...consumer.SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { +func (c mockKafkaConsumerFactory) StartConsumerGroup(ctx context.Context, groupID string, topics []string, handler consumer.KafkaConsumerHandler, options ...consumer.SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { if c.createErr { return nil, errors.New("error creating consumer") } @@ -267,9 +267,11 @@ func TestDispatcher_UpdateConsumers(t *testing.T) { logger: zaptest.NewLogger(t).Sugar(), } + ctx := context.TODO() + // Initialize using oldConfig require.NoError(t, d.RegisterChannelHost(tc.oldConfig)) - require.NoError(t, d.ReconcileConsumers(tc.oldConfig)) + require.NoError(t, d.ReconcileConsumers(ctx, tc.oldConfig)) oldSubscribers := sets.NewString() for _, sub := range d.subscriptions { @@ -283,7 +285,7 @@ func TestDispatcher_UpdateConsumers(t *testing.T) { } // Update with new config - err := d.ReconcileConsumers(tc.newConfig) + err := d.ReconcileConsumers(ctx, tc.newConfig) if tc.createErr != "" { if err == nil { t.Errorf("Expected UpdateConfig error: '%v'. Actual nil", tc.createErr) @@ -363,6 +365,8 @@ func TestDispatcher_MultipleChannelsInParallel(t *testing.T) { logger: zaptest.NewLogger(t).Sugar(), } + ctx := context.TODO() + // Let's register channel configs first for _, c := range configs { require.NoError(t, d.RegisterChannelHost(c)) @@ -374,7 +378,7 @@ func TestDispatcher_MultipleChannelsInParallel(t *testing.T) { wg.Add(1) go func(c *ChannelConfig) { defer wg.Done() - assert.NoError(t, d.ReconcileConsumers(c)) + assert.NoError(t, d.ReconcileConsumers(ctx, c)) }(c) } } @@ -418,6 +422,8 @@ func TestKafkaDispatcher_CleanupChannel(t *testing.T) { logger: zaptest.NewLogger(t).Sugar(), } + ctx := context.TODO() + channelConfig := &ChannelConfig{ Namespace: "default", Name: "test-channel", @@ -438,7 +444,7 @@ func TestKafkaDispatcher_CleanupChannel(t *testing.T) { }, } require.NoError(t, d.RegisterChannelHost(channelConfig)) - require.NoError(t, d.ReconcileConsumers(channelConfig)) + require.NoError(t, d.ReconcileConsumers(ctx, channelConfig)) require.NoError(t, d.CleanupChannel(channelConfig.Name, channelConfig.Namespace, channelConfig.HostName)) require.NotContains(t, d.subscriptions, "subscription-1") @@ -461,6 +467,8 @@ func TestSubscribeError(t *testing.T) { channelSubscriptions: map[types.NamespacedName]*KafkaSubscription{}, } + ctx := context.TODO() + channelRef := types.NamespacedName{ Name: "test-channel", Namespace: "test-ns", @@ -470,7 +478,7 @@ func TestSubscribeError(t *testing.T) { UID: "test-sub", Subscription: fanout.Subscription{}, } - err := d.subscribe(channelRef, subRef) + err := d.subscribe(ctx, channelRef, subRef) if err == nil { t.Errorf("Expected error want %s, got %s", "error creating consumer", err) } diff --git a/pkg/channel/consolidated/dispatcher/subscription_endpoint.go b/pkg/channel/consolidated/dispatcher/subscription_endpoint.go deleted file mode 100644 index 942d9e26cf..0000000000 --- a/pkg/channel/consolidated/dispatcher/subscription_endpoint.go +++ /dev/null @@ -1,60 +0,0 @@ -package dispatcher - -import ( - "encoding/json" - nethttp "net/http" - "strings" - - "go.uber.org/zap" - "k8s.io/apimachinery/pkg/types" -) - -// subscriptionEndpoint is serving the subscription status of the Kafka channel. -// A prober in the controller calls the endpoint to see if the subscriper is ready. -type subscriptionEndpoint struct { - dispatcher *KafkaDispatcher - logger *zap.SugaredLogger -} - -func (d *subscriptionEndpoint) ServeHTTP(w nethttp.ResponseWriter, r *nethttp.Request) { - if r.Method != nethttp.MethodGet { - w.WriteHeader(nethttp.StatusMethodNotAllowed) - d.logger.Errorf("Received request method that wasn't GET: %s", r.Method) - return - } - uriSplit := strings.Split(r.RequestURI, "/") - if len(uriSplit) != 3 { - w.WriteHeader(nethttp.StatusNotFound) - d.logger.Errorf("Unable to process request: %s", r.RequestURI) - return - } - channelRefNamespace, channelRefName := uriSplit[1], uriSplit[2] - channelRef := types.NamespacedName{ - Name: channelRefName, - Namespace: channelRefNamespace, - } - if _, ok := d.dispatcher.channelSubscriptions[channelRef]; !ok { - w.WriteHeader(nethttp.StatusNotFound) - return - } - d.dispatcher.channelSubscriptions[channelRef].readySubscriptionsLock.RLock() - defer d.dispatcher.channelSubscriptions[channelRef].readySubscriptionsLock.RUnlock() - var subscriptions = make(map[string][]int32) - w.Header().Set(dispatcherReadySubHeader, channelRefName) - for s, ps := range d.dispatcher.channelSubscriptions[channelRef].channelReadySubscriptions { - subscriptions[s] = ps.List() - } - jsonResult, err := json.Marshal(subscriptions) - if err != nil { - d.logger.Errorf("Error marshalling json for sub-status channelref: %s/%s, %w", channelRefNamespace, channelRefName, err) - return - } - _, err = w.Write(jsonResult) - if err != nil { - d.logger.Errorf("Error writing jsonResult to serveHTTP writer: %w", err) - } -} - -func (d *subscriptionEndpoint) start() { - d.logger.Fatal(nethttp.ListenAndServe(":8081", d)) -} diff --git a/pkg/channel/consolidated/dispatcher/subscription_endpoint_test.go b/pkg/channel/consolidated/dispatcher/subscription_endpoint_test.go deleted file mode 100644 index 63d7b0a6f9..0000000000 --- a/pkg/channel/consolidated/dispatcher/subscription_endpoint_test.go +++ /dev/null @@ -1,155 +0,0 @@ -package dispatcher - -import ( - "fmt" - "io/ioutil" - "net/http" - "net/http/httptest" - "testing" - - "github.com/google/go-cmp/cmp" - "k8s.io/apimachinery/pkg/types" - "k8s.io/apimachinery/pkg/util/sets" - klogtesting "knative.dev/pkg/logging/testing" -) - -func TestServeHTTP(t *testing.T) { - - httpGet := "GET" - httpPost := "POST" - testCases := []struct { - name string - responseReturnCode int - desiredJson []byte - channelSubs map[types.NamespacedName]*KafkaSubscription - requestURI string - httpMethod string - }{ - { - name: "channelref not found", - httpMethod: httpGet, - responseReturnCode: http.StatusNotFound, - desiredJson: []byte{}, - requestURI: "/exist/thisDoesNot", - }, { - name: "nop", - httpMethod: httpGet, - responseReturnCode: http.StatusNotFound, - desiredJson: []byte{}, - requestURI: "///", - }, { - name: "no ready subscribers", - httpMethod: httpGet, - responseReturnCode: http.StatusOK, - desiredJson: []byte(`{}`), - channelSubs: map[types.NamespacedName]*KafkaSubscription{ - {Name: "foo", Namespace: "bar"}: { - subs: sets.NewString(), - channelReadySubscriptions: map[string]sets.Int32{}, - }, - }, - requestURI: "/bar/foo", - }, { - name: "different channelref called from populated channref (different ns)", - httpMethod: httpGet, - desiredJson: []byte{}, - responseReturnCode: http.StatusNotFound, - channelSubs: map[types.NamespacedName]*KafkaSubscription{ - {Name: "foo", Namespace: "baz"}: { - subs: sets.NewString("a", "b"), - channelReadySubscriptions: map[string]sets.Int32{ - "a": sets.NewInt32(0), - "b": sets.NewInt32(0), - }, - }, - }, - requestURI: "/bar/foo", - }, { - name: "return correct subscription", - httpMethod: httpGet, - desiredJson: []byte(`{"a":[0],"b":[0,2,5]}`), - responseReturnCode: http.StatusOK, - channelSubs: map[types.NamespacedName]*KafkaSubscription{ - {Name: "foo", Namespace: "bar"}: { - subs: sets.NewString("a", "b"), - channelReadySubscriptions: map[string]sets.Int32{ - "a": sets.NewInt32(0), - "b": sets.NewInt32(0, 2, 5), - }, - }, - }, - requestURI: "/bar/foo", - }, { - name: "return correct subscription from multiple chanrefs", - httpMethod: httpGet, - desiredJson: []byte(`{"a":[0],"b":[0,2,5]}`), - responseReturnCode: http.StatusOK, - channelSubs: map[types.NamespacedName]*KafkaSubscription{ - {Name: "table", Namespace: "flip"}: { - subs: sets.NewString("c", "d"), - channelReadySubscriptions: map[string]sets.Int32{ - "c": sets.NewInt32(0), - "d": sets.NewInt32(0), - }}, - {Name: "foo", Namespace: "bar"}: { - subs: sets.NewString("a", "b"), - channelReadySubscriptions: map[string]sets.Int32{ - "a": sets.NewInt32(0), - "b": sets.NewInt32(0, 2, 5), - }, - }, - }, - requestURI: "/bar/foo", - }, { - name: "bad request uri", - httpMethod: httpGet, - desiredJson: []byte{}, - responseReturnCode: http.StatusNotFound, - requestURI: "/here/be/dragons/there/are/too/many/slashes", - }, { - name: "bad request method (POST)", - httpMethod: httpPost, - desiredJson: []byte{}, - responseReturnCode: http.StatusMethodNotAllowed, - }, - } - logger := klogtesting.TestLogger(t) - d := &KafkaDispatcher{ - channelSubscriptions: make(map[types.NamespacedName]*KafkaSubscription), - logger: logger, - } - subscriptionEndpoint := &subscriptionEndpoint{ - dispatcher: d, - logger: logger, - } - - ts := httptest.NewServer(subscriptionEndpoint) - defer ts.Close() - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - t.Logf("Running %s", t.Name()) - d.channelSubscriptions = tc.channelSubs - - request, _ := http.NewRequest(tc.httpMethod, fmt.Sprintf("%s%s", ts.URL, tc.requestURI), nil) - // resp, err := http.Get(fmt.Sprintf("%s%s", ts.URL, tc.requestURI)) - resp, err := http.DefaultClient.Do(request) - if err != nil { - t.Errorf("Could not send request to subscriber endpoint: %v", err) - } - if resp.StatusCode != tc.responseReturnCode { - t.Errorf("unepxected status returned: want: %d, got: %d", tc.responseReturnCode, resp.StatusCode) - } - respBody, err := ioutil.ReadAll(resp.Body) - defer resp.Body.Close() - if err != nil { - t.Errorf("Could not read response from subscriber endpoint: %v", err) - } - if testing.Verbose() && len(respBody) > 0 { - t.Logf("http response: %s\n", string(respBody)) - } - if diff := cmp.Diff(tc.desiredJson, respBody); diff != "" { - t.Errorf("unexpected readysubscriber status response: (-want, +got) = %v", diff) - } - }) - } -} diff --git a/pkg/channel/consolidated/reconciler/controller/controller.go b/pkg/channel/consolidated/reconciler/controller/controller.go index f96afaf1fe..9d132966de 100644 --- a/pkg/channel/consolidated/reconciler/controller/controller.go +++ b/pkg/channel/consolidated/reconciler/controller/controller.go @@ -26,11 +26,9 @@ import ( corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/labels" - "k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/util/wait" "k8s.io/client-go/tools/cache" "k8s.io/utils/pointer" - eventingduckv1 "knative.dev/eventing/pkg/apis/duck/v1" eventingClient "knative.dev/eventing/pkg/client/injection/client" kubeclient "knative.dev/pkg/client/injection/kube/client" deploymentinformer "knative.dev/pkg/client/injection/kube/informers/apps/v1/deployment" @@ -45,9 +43,6 @@ import ( knativeReconciler "knative.dev/pkg/reconciler" "knative.dev/pkg/system" - "knative.dev/eventing-kafka/pkg/apis/messaging/v1beta1" - "knative.dev/eventing-kafka/pkg/channel/consolidated/status" - kafkamessagingv1beta1 "knative.dev/eventing-kafka/pkg/client/informers/externalversions/messaging/v1beta1" kafkaChannelClient "knative.dev/eventing-kafka/pkg/client/injection/client" "knative.dev/eventing-kafka/pkg/client/injection/informers/messaging/v1beta1/kafkachannel" kafkaChannelReconciler "knative.dev/eventing-kafka/pkg/client/injection/reconciler/messaging/v1beta1/kafkachannel" @@ -110,17 +105,6 @@ func NewController( impl := kafkaChannelReconciler.NewImpl(ctx, r) - statusProber := status.NewProber( - logger.Named("status-manager"), - NewProbeTargetLister(logger, endpointsInformer.Lister()), - func(c v1beta1.KafkaChannel, s eventingduckv1.SubscriberSpec) { - logger.Debugf("Ready callback triggered for channel: %s/%s subscription: %s", c.Namespace, c.Name, string(s.UID)) - impl.EnqueueKey(types.NamespacedName{Namespace: c.Namespace, Name: c.Name}) - }, - ) - r.statusManager = statusProber - statusProber.Start(ctx.Done()) - // Call GlobalResync on kafkachannels. grCh := func(obj interface{}) { logger.Debug("Changes detected, doing global resync") @@ -167,34 +151,17 @@ func NewController( FilterFunc: filterFn, Handler: controller.HandleAll(grCh), }) - podInformer.Informer().AddEventHandler(cache.FilteringResourceEventHandler{ FilterFunc: knativeReconciler.ChainFilterFuncs( knativeReconciler.LabelFilterFunc(channelLabelKey, channelLabelValue, false), knativeReconciler.LabelFilterFunc(roleLabelKey, dispatcherRoleLabelValue, false), ), - Handler: cache.ResourceEventHandlerFuncs{ - // Cancel probing when a Pod is deleted - DeleteFunc: getPodInformerEventHandler(ctx, logger, statusProber, impl, kafkaChannelInformer, "Delete"), - AddFunc: getPodInformerEventHandler(ctx, logger, statusProber, impl, kafkaChannelInformer, "Add"), - }, + Handler: controller.HandleAll(grCh), }) return impl } -func getPodInformerEventHandler(ctx context.Context, logger *zap.SugaredLogger, statusProber *status.Prober, impl *controller.Impl, kafkaChannelInformer kafkamessagingv1beta1.KafkaChannelInformer, handlerType string) func(obj interface{}) { - return func(obj interface{}) { - pod, ok := obj.(*corev1.Pod) - if ok && pod != nil { - logger.Debugw("%s pods. Refreshing pod probing.", handlerType, - zap.String("pod", pod.GetName())) - statusProber.RefreshPodProbing(ctx) - impl.GlobalResync(kafkaChannelInformer.Informer()) - } - } -} - func getControllerOwnerRef(ctx context.Context) (*metav1.OwnerReference, error) { logger := logging.FromContext(ctx) ctrlDeploymentLabels := labels.Set{ diff --git a/pkg/channel/consolidated/reconciler/controller/kafkachannel.go b/pkg/channel/consolidated/reconciler/controller/kafkachannel.go index 3aa017316d..3ad68aafcb 100644 --- a/pkg/channel/consolidated/reconciler/controller/kafkachannel.go +++ b/pkg/channel/consolidated/reconciler/controller/kafkachannel.go @@ -36,6 +36,7 @@ import ( corev1listers "k8s.io/client-go/listers/core/v1" rbacv1listers "k8s.io/client-go/listers/rbac/v1" "k8s.io/client-go/tools/cache" + "knative.dev/eventing-kafka/pkg/common/kafka/offset" v1 "knative.dev/eventing/pkg/apis/duck/v1" "knative.dev/eventing/pkg/apis/eventing" eventingclientset "knative.dev/eventing/pkg/client/clientset/versioned" @@ -48,7 +49,6 @@ import ( "knative.dev/eventing-kafka/pkg/apis/messaging/v1beta1" "knative.dev/eventing-kafka/pkg/channel/consolidated/reconciler/controller/resources" - "knative.dev/eventing-kafka/pkg/channel/consolidated/status" "knative.dev/eventing-kafka/pkg/channel/consolidated/utils" kafkaclientset "knative.dev/eventing-kafka/pkg/client/clientset/versioned" kafkaScheme "knative.dev/eventing-kafka/pkg/client/clientset/versioned/scheme" @@ -128,6 +128,7 @@ type Reconciler struct { kafkaAuthConfig *client.KafkaAuthConfig kafkaConfigError error kafkaClientSet kafkaclientset.Interface + kafkaClient sarama.Client // Using a shared kafkaClusterAdmin does not work currently because of an issue with // Shopify/sarama, see https://github.com/Shopify/sarama/issues/1162. kafkaClusterAdmin sarama.ClusterAdmin @@ -138,7 +139,6 @@ type Reconciler struct { endpointsLister corev1listers.EndpointsLister serviceAccountLister corev1listers.ServiceAccountLister roleBindingLister rbacv1listers.RoleBindingLister - statusManager status.Manager controllerRef metav1.OwnerReference } @@ -168,8 +168,17 @@ func (r *Reconciler) ReconcileKind(ctx context.Context, kc *v1beta1.KafkaChannel return r.kafkaConfigError } - kafkaClusterAdmin, err := r.createClient(ctx) + kafkaClient, err := r.createKafkaClient() if err != nil { + logger.Errorw("Can't obtain Kafka Client", zap.Any("channel", kc), zap.Error(err)) + kc.Status.MarkConfigFailed("InvalidConfiguration", "Unable to build Kafka client for channel %s: %v", kc.Name, err) + return err + } + defer kafkaClient.Close() + + kafkaClusterAdmin, err := r.createClusterAdmin() + if err != nil { + logger.Errorw("Can't obtain Kafka cluster admin", zap.Any("channel", kc), zap.Error(err)) kc.Status.MarkConfigFailed("InvalidConfiguration", "Unable to build Kafka admin client for channel %s: %v", kc.Name, err) return err } @@ -244,7 +253,7 @@ func (r *Reconciler) ReconcileKind(ctx context.Context, kc *v1beta1.KafkaChannel Scheme: "http", Host: network.GetServiceHostname(svc.Name, svc.Namespace), }) - err = r.reconcileSubscribers(ctx, kc) + err = r.reconcileSubscribers(ctx, kc, kafkaClient, kafkaClusterAdmin) if err != nil { return fmt.Errorf("error reconciling subscribers %v", err) } @@ -254,12 +263,24 @@ func (r *Reconciler) ReconcileKind(ctx context.Context, kc *v1beta1.KafkaChannel return newReconciledNormal(kc.Namespace, kc.Name) } -func (r *Reconciler) reconcileSubscribers(ctx context.Context, ch *v1beta1.KafkaChannel) error { +func (r *Reconciler) reconcileSubscribers(ctx context.Context, ch *v1beta1.KafkaChannel, kafkaClient sarama.Client, kafkaClusterAdmin sarama.ClusterAdmin) error { after := ch.DeepCopy() after.Status.Subscribers = make([]v1.SubscriberStatus, 0) + logger := logging.FromContext(ctx) for _, s := range ch.Spec.Subscribers { - if r, _ := r.statusManager.IsReady(ctx, *ch, s); r { - logging.FromContext(ctx).Debugw("marking subscription", zap.Any("subscription", s)) + logger.Debugw("Reconciling initial offset for subscription", zap.Any("subscription", s), zap.Any("channel", ch)) + err := r.reconcileInitialOffset(ctx, ch, s, kafkaClient, kafkaClusterAdmin) + + if err != nil { + logger.Errorw("reconcile failed to initial offset for subscription. Marking the subscription not ready", zap.String("channel", fmt.Sprintf("%s.%s", ch.Namespace, ch.Name)), zap.Any("subscription", s), zap.Error(err)) + after.Status.Subscribers = append(after.Status.Subscribers, v1.SubscriberStatus{ + UID: s.UID, + ObservedGeneration: s.Generation, + Ready: corev1.ConditionFalse, + Message: fmt.Sprintf("Initial offset cannot be committed: %v", err), + }) + } else { + logger.Debugw("Reconciled initial offset for subscription. Marking the subscription ready", zap.String("channel", fmt.Sprintf("%s.%s", ch.Namespace, ch.Name)), zap.Any("subscription", s)) after.Status.Subscribers = append(after.Status.Subscribers, v1.SubscriberStatus{ UID: s.UID, ObservedGeneration: s.Generation, @@ -286,7 +307,7 @@ func (r *Reconciler) reconcileSubscribers(ctx context.Context, ch *v1beta1.Kafka if err != nil { return fmt.Errorf("Failed patching: %w", err) } - logging.FromContext(ctx).Debugw("Patched resource", zap.Any("patch", patch), zap.Any("patched", patched)) + logger.Debugw("Patched resource", zap.Any("patch", patch), zap.Any("patched", patched)) return nil } @@ -505,7 +526,24 @@ func (r *Reconciler) reconcileChannelService(ctx context.Context, dispatcherName return svc, nil } -func (r *Reconciler) createClient(ctx context.Context) (sarama.ClusterAdmin, error) { +func (r *Reconciler) createKafkaClient() (sarama.Client, error) { + kafkaClient := r.kafkaClient + if kafkaClient == nil { + var err error + + if r.kafkaConfig.EventingKafka.Sarama.Config == nil { + return nil, fmt.Errorf("error creating Kafka client: Sarama config is nil") + } + kafkaClient, err = sarama.NewClient(r.kafkaConfig.Brokers, r.kafkaConfig.EventingKafka.Sarama.Config) + if err != nil { + return nil, err + } + } + + return kafkaClient, nil +} + +func (r *Reconciler) createClusterAdmin() (sarama.ClusterAdmin, error) { // We don't currently initialize r.kafkaClusterAdmin, hence we end up creating the cluster admin client every time. // This is because of an issue with Shopify/sarama. See https://github.com/Shopify/sarama/issues/1162. // Once the issue is fixed we should use a shared cluster admin client. Also, r.kafkaClusterAdmin is currently @@ -558,6 +596,23 @@ func (r *Reconciler) reconcileTopic(ctx context.Context, channel *v1beta1.KafkaC return err } +func (r *Reconciler) reconcileInitialOffset(ctx context.Context, channel *v1beta1.KafkaChannel, sub v1.SubscriberSpec, kafkaClient sarama.Client, kafkaClusterAdmin sarama.ClusterAdmin) error { + subscriptionStatus := findSubscriptionStatus(channel, sub.UID) + if subscriptionStatus != nil && subscriptionStatus.Ready == corev1.ConditionTrue { + // subscription is ready, the offsets must have been initialized already + return nil + } + + topicName := utils.TopicName(utils.KafkaChannelSeparator, channel.Namespace, channel.Name) + groupID := fmt.Sprintf("kafka.%s.%s.%s", channel.Namespace, channel.Name, string(sub.UID)) + _, err := offset.InitOffsets(ctx, kafkaClient, kafkaClusterAdmin, []string{topicName}, groupID) + if err != nil { + logger := logging.FromContext(ctx) + logger.Errorw("error reconciling initial offset", zap.String("channel", fmt.Sprintf("%s.%s", channel.Namespace, channel.Name)), zap.Any("subscription", sub), zap.Error(err)) + } + return err +} + func (r *Reconciler) deleteTopic(ctx context.Context, channel *v1beta1.KafkaChannel, kafkaClusterAdmin sarama.ClusterAdmin) error { logger := logging.FromContext(ctx) @@ -595,6 +650,11 @@ func (r *Reconciler) updateKafkaConfig(ctx context.Context, configMap *corev1.Co return } + // Manually commit the offsets in KafkaChannel controller. + // That's because we want to make sure we initialize the offsets within the controller + // before dispatcher actually starts consuming messages. + kafkaConfig.EventingKafka.Sarama.Config.Consumer.Offsets.AutoCommit.Enable = false + r.kafkaAuthConfig = kafkaConfig.EventingKafka.Auth // For now just override the previous config. // Eventually the previous config should be snapshotted to delete Kafka topics @@ -609,20 +669,29 @@ func (r *Reconciler) FinalizeKind(ctx context.Context, kc *v1beta1.KafkaChannel) logger := logging.FromContext(ctx) channel := fmt.Sprintf("%s/%s", kc.GetNamespace(), kc.GetName()) logger.Debugw("FinalizeKind", zap.String("channel", channel)) - kafkaClusterAdmin, err := r.createClient(ctx) + + kafkaClusterAdmin, err := r.createClusterAdmin() if err != nil || r.kafkaConfig == nil { - logger.Errorw("Can't obtain Kafka Client", zap.String("channel", channel), zap.Error(err)) - } else { - defer kafkaClusterAdmin.Close() - logger.Debugw("Got client, about to delete topic") - if err := r.deleteTopic(ctx, kc, kafkaClusterAdmin); err != nil { - logger.Errorw("Error deleting Kafka channel topic", zap.String("channel", channel), zap.Error(err)) - return err - } + logger.Errorw("cannot obtain Kafka cluster admin", zap.String("channel", channel), zap.Error(err)) + // even in error case, we return `normal`, since we are fine with leaving the + // topic undeleted e.g. when we lose connection + return newReconciledNormal(kc.Namespace, kc.Name) } - for _, s := range kc.Spec.Subscribers { - logger.Debugw("Canceling probing", zap.String("channel", channel), zap.Any("subscription", s)) - r.statusManager.CancelProbing(s) + defer kafkaClusterAdmin.Close() + + logger.Debugw("got client, about to delete topic") + if err := r.deleteTopic(ctx, kc, kafkaClusterAdmin); err != nil { + logger.Errorw("error deleting Kafka channel topic", zap.String("channel", channel), zap.Error(err)) + return err } return newReconciledNormal(kc.Namespace, kc.Name) //ok to remove finalizer } + +func findSubscriptionStatus(kc *v1beta1.KafkaChannel, subUID types.UID) *v1.SubscriberStatus { + for _, subStatus := range kc.Status.Subscribers { + if subStatus.UID == subUID { + return &subStatus + } + } + return nil +} diff --git a/pkg/channel/consolidated/reconciler/controller/kafkachannel_test.go b/pkg/channel/consolidated/reconciler/controller/kafkachannel_test.go index 00b9ee377d..b770b54e41 100644 --- a/pkg/channel/consolidated/reconciler/controller/kafkachannel_test.go +++ b/pkg/channel/consolidated/reconciler/controller/kafkachannel_test.go @@ -29,6 +29,7 @@ import ( "k8s.io/apimachinery/pkg/runtime" "k8s.io/client-go/kubernetes/scheme" clientgotesting "k8s.io/client-go/testing" + commontesting "knative.dev/eventing-kafka/pkg/common/testing" eventingduckv1 "knative.dev/eventing/pkg/apis/duck/v1" eventingClient "knative.dev/eventing/pkg/client/injection/client" "knative.dev/pkg/apis" @@ -413,11 +414,15 @@ func TestAllCases(t *testing.T) { } table.Test(t, reconcilertesting.MakeFactory(func(ctx context.Context, listers *reconcilertesting.Listers, cmw configmap.Watcher, options map[string]interface{}) controller.Reconciler { - + saramaConfig := &sarama.Config{} // Optional Customization Of KafkaConfig.EventingKafka - Defaults To Empty configEventingKafkaInterface, ok := options["configEventingKafka"] if !ok || configEventingKafkaInterface == nil { - configEventingKafkaInterface = config.EventingKafkaConfig{} + configEventingKafkaInterface = config.EventingKafkaConfig{ + Sarama: config.EKSaramaConfig{ + Config: saramaConfig, + }, + } } configEventingKafka := configEventingKafkaInterface.(config.EventingKafkaConfig) @@ -436,16 +441,19 @@ func TestAllCases(t *testing.T) { deploymentLister: listers.GetDeploymentLister(), serviceLister: listers.GetServiceLister(), endpointsLister: listers.GetEndpointsLister(), - kafkaClusterAdmin: &mockClusterAdmin{}, - kafkaClientSet: fakekafkaclient.Get(ctx), - KubeClientSet: kubeclient.Get(ctx), - EventingClientSet: eventingClient.Get(ctx), - statusManager: &fakeStatusManager{ - FakeIsReady: func(ctx context.Context, ch v1beta1.KafkaChannel, - sub eventingduckv1.SubscriberSpec) (bool, error) { - return true, nil + kafkaClusterAdmin: &commontesting.MockClusterAdmin{ + MockListConsumerGroupsFunc: func() (map[string]string, error) { + cgs := map[string]string{ + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub1UID): "consumer", + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub2UID): "consumer", + } + return cgs, nil }, }, + kafkaClient: &mockKafkaClient{saramaConfig}, + kafkaClientSet: fakekafkaclient.Get(ctx), + KubeClientSet: kubeclient.Get(ctx), + EventingClientSet: eventingClient.Get(ctx), } return kafkachannel.NewReconciler(ctx, logging.FromContext(ctx), r.kafkaClientSet, listers.GetKafkaChannelLister(), controller.GetEventRecorder(ctx), r) }, zap.L())) @@ -486,15 +494,19 @@ func TestTopicExists(t *testing.T) { } row.Test(t, reconcilertesting.MakeFactory(func(ctx context.Context, listers *reconcilertesting.Listers, cmw configmap.Watcher, options map[string]interface{}) controller.Reconciler { - + saramaConfig := &sarama.Config{} r := &Reconciler{ systemNamespace: testNS, dispatcherImage: testDispatcherImage, dispatcherServiceAccount: testDispatcherserviceAccount, kafkaConfigMapHash: testConfigMapHash, kafkaConfig: &KafkaConfig{ - Brokers: []string{brokerName}, - EventingKafka: &config.EventingKafkaConfig{}, + Brokers: []string{brokerName}, + EventingKafka: &config.EventingKafkaConfig{ + Sarama: config.EKSaramaConfig{ + Config: saramaConfig, + }, + }, }, kafkachannelLister: listers.GetKafkaChannelLister(), // TODO fix @@ -502,24 +514,26 @@ func TestTopicExists(t *testing.T) { deploymentLister: listers.GetDeploymentLister(), serviceLister: listers.GetServiceLister(), endpointsLister: listers.GetEndpointsLister(), - kafkaClusterAdmin: &mockClusterAdmin{ - mockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { + kafkaClusterAdmin: &commontesting.MockClusterAdmin{ + MockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { errMsg := sarama.ErrTopicAlreadyExists.Error() return &sarama.TopicError{ Err: sarama.ErrTopicAlreadyExists, ErrMsg: &errMsg, } }, + MockListConsumerGroupsFunc: func() (map[string]string, error) { + cgs := map[string]string{ + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub1UID): "consumer", + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub2UID): "consumer", + } + return cgs, nil + }, }, + kafkaClient: &mockKafkaClient{saramaConfig}, kafkaClientSet: fakekafkaclient.Get(ctx), KubeClientSet: kubeclient.Get(ctx), EventingClientSet: eventingClient.Get(ctx), - statusManager: &fakeStatusManager{ - FakeIsReady: func(ctx context.Context, channel v1beta1.KafkaChannel, - spec eventingduckv1.SubscriberSpec) (bool, error) { - return true, nil - }, - }, } return kafkachannel.NewReconciler(ctx, logging.FromContext(ctx), r.kafkaClientSet, listers.GetKafkaChannelLister(), controller.GetEventRecorder(ctx), r) }, zap.L())) @@ -564,15 +578,19 @@ func TestDeploymentUpdatedOnImageChange(t *testing.T) { } row.Test(t, reconcilertesting.MakeFactory(func(ctx context.Context, listers *reconcilertesting.Listers, cmw configmap.Watcher, options map[string]interface{}) controller.Reconciler { - + saramaConfig := &sarama.Config{} r := &Reconciler{ systemNamespace: testNS, dispatcherImage: testDispatcherImage, dispatcherServiceAccount: testDispatcherserviceAccount, kafkaConfigMapHash: testConfigMapHash, kafkaConfig: &KafkaConfig{ - Brokers: []string{brokerName}, - EventingKafka: &config.EventingKafkaConfig{}, + Brokers: []string{brokerName}, + EventingKafka: &config.EventingKafkaConfig{ + Sarama: config.EKSaramaConfig{ + Config: saramaConfig, + }, + }, }, kafkachannelLister: listers.GetKafkaChannelLister(), // TODO fix @@ -580,24 +598,26 @@ func TestDeploymentUpdatedOnImageChange(t *testing.T) { deploymentLister: listers.GetDeploymentLister(), serviceLister: listers.GetServiceLister(), endpointsLister: listers.GetEndpointsLister(), - kafkaClusterAdmin: &mockClusterAdmin{ - mockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { + kafkaClusterAdmin: &commontesting.MockClusterAdmin{ + MockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { errMsg := sarama.ErrTopicAlreadyExists.Error() return &sarama.TopicError{ Err: sarama.ErrTopicAlreadyExists, ErrMsg: &errMsg, } }, + MockListConsumerGroupsFunc: func() (map[string]string, error) { + cgs := map[string]string{ + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub1UID): "consumer", + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub2UID): "consumer", + } + return cgs, nil + }, }, + kafkaClient: &mockKafkaClient{saramaConfig}, kafkaClientSet: fakekafkaclient.Get(ctx), KubeClientSet: kubeclient.Get(ctx), EventingClientSet: eventingClient.Get(ctx), - statusManager: &fakeStatusManager{ - FakeIsReady: func(ctx context.Context, channel v1beta1.KafkaChannel, - spec eventingduckv1.SubscriberSpec) (bool, error) { - return true, nil - }, - }, } return kafkachannel.NewReconciler(ctx, logging.FromContext(ctx), r.kafkaClientSet, listers.GetKafkaChannelLister(), controller.GetEventRecorder(ctx), r) }, zap.L())) @@ -642,15 +662,19 @@ func TestDeploymentZeroReplicas(t *testing.T) { } row.Test(t, reconcilertesting.MakeFactory(func(ctx context.Context, listers *reconcilertesting.Listers, cmw configmap.Watcher, options map[string]interface{}) controller.Reconciler { - + saramaConfig := &sarama.Config{} r := &Reconciler{ systemNamespace: testNS, dispatcherImage: testDispatcherImage, dispatcherServiceAccount: testDispatcherserviceAccount, kafkaConfigMapHash: testConfigMapHash, kafkaConfig: &KafkaConfig{ - Brokers: []string{brokerName}, - EventingKafka: &config.EventingKafkaConfig{}, + Brokers: []string{brokerName}, + EventingKafka: &config.EventingKafkaConfig{ + Sarama: config.EKSaramaConfig{ + Config: saramaConfig, + }, + }, }, kafkachannelLister: listers.GetKafkaChannelLister(), // TODO fix @@ -658,24 +682,26 @@ func TestDeploymentZeroReplicas(t *testing.T) { deploymentLister: listers.GetDeploymentLister(), serviceLister: listers.GetServiceLister(), endpointsLister: listers.GetEndpointsLister(), - kafkaClusterAdmin: &mockClusterAdmin{ - mockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { + kafkaClusterAdmin: &commontesting.MockClusterAdmin{ + MockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { errMsg := sarama.ErrTopicAlreadyExists.Error() return &sarama.TopicError{ Err: sarama.ErrTopicAlreadyExists, ErrMsg: &errMsg, } }, + MockListConsumerGroupsFunc: func() (map[string]string, error) { + cgs := map[string]string{ + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub1UID): "consumer", + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub2UID): "consumer", + } + return cgs, nil + }, }, + kafkaClient: &mockKafkaClient{saramaConfig}, kafkaClientSet: fakekafkaclient.Get(ctx), KubeClientSet: kubeclient.Get(ctx), EventingClientSet: eventingClient.Get(ctx), - statusManager: &fakeStatusManager{ - FakeIsReady: func(ctx context.Context, channel v1beta1.KafkaChannel, - spec eventingduckv1.SubscriberSpec) (bool, error) { - return true, nil - }, - }, } return kafkachannel.NewReconciler(ctx, logging.FromContext(ctx), r.kafkaClientSet, listers.GetKafkaChannelLister(), controller.GetEventRecorder(ctx), r) }, zap.L())) @@ -716,15 +742,19 @@ func TestDeploymentMoreThanOneReplicas(t *testing.T) { } row.Test(t, reconcilertesting.MakeFactory(func(ctx context.Context, listers *reconcilertesting.Listers, cmw configmap.Watcher, options map[string]interface{}) controller.Reconciler { - + saramaConfig := &sarama.Config{} r := &Reconciler{ systemNamespace: testNS, dispatcherImage: testDispatcherImage, dispatcherServiceAccount: testDispatcherserviceAccount, kafkaConfigMapHash: testConfigMapHash, kafkaConfig: &KafkaConfig{ - Brokers: []string{brokerName}, - EventingKafka: &config.EventingKafkaConfig{}, + Brokers: []string{brokerName}, + EventingKafka: &config.EventingKafkaConfig{ + Sarama: config.EKSaramaConfig{ + Config: saramaConfig, + }, + }, }, kafkachannelLister: listers.GetKafkaChannelLister(), // TODO fix @@ -732,24 +762,26 @@ func TestDeploymentMoreThanOneReplicas(t *testing.T) { deploymentLister: listers.GetDeploymentLister(), serviceLister: listers.GetServiceLister(), endpointsLister: listers.GetEndpointsLister(), - kafkaClusterAdmin: &mockClusterAdmin{ - mockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { + kafkaClusterAdmin: &commontesting.MockClusterAdmin{ + MockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { errMsg := sarama.ErrTopicAlreadyExists.Error() return &sarama.TopicError{ Err: sarama.ErrTopicAlreadyExists, ErrMsg: &errMsg, } }, + MockListConsumerGroupsFunc: func() (map[string]string, error) { + cgs := map[string]string{ + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub1UID): "consumer", + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub2UID): "consumer", + } + return cgs, nil + }, }, + kafkaClient: &mockKafkaClient{saramaConfig}, kafkaClientSet: fakekafkaclient.Get(ctx), KubeClientSet: kubeclient.Get(ctx), EventingClientSet: eventingClient.Get(ctx), - statusManager: &fakeStatusManager{ - FakeIsReady: func(ctx context.Context, channel v1beta1.KafkaChannel, - spec eventingduckv1.SubscriberSpec) (bool, error) { - return true, nil - }, - }, } return kafkachannel.NewReconciler(ctx, logging.FromContext(ctx), r.kafkaClientSet, listers.GetKafkaChannelLister(), controller.GetEventRecorder(ctx), r) }, zap.L())) @@ -794,15 +826,19 @@ func TestDeploymentUpdatedOnConfigMapHashChange(t *testing.T) { } row.Test(t, reconcilertesting.MakeFactory(func(ctx context.Context, listers *reconcilertesting.Listers, cmw configmap.Watcher, options map[string]interface{}) controller.Reconciler { - + saramaConfig := &sarama.Config{} r := &Reconciler{ systemNamespace: testNS, dispatcherImage: testDispatcherImage, dispatcherServiceAccount: testDispatcherserviceAccount, kafkaConfigMapHash: testConfigMapHash, kafkaConfig: &KafkaConfig{ - Brokers: []string{brokerName}, - EventingKafka: &config.EventingKafkaConfig{}, + Brokers: []string{brokerName}, + EventingKafka: &config.EventingKafkaConfig{ + Sarama: config.EKSaramaConfig{ + Config: saramaConfig, + }, + }, }, kafkachannelLister: listers.GetKafkaChannelLister(), // TODO fix @@ -810,138 +846,114 @@ func TestDeploymentUpdatedOnConfigMapHashChange(t *testing.T) { deploymentLister: listers.GetDeploymentLister(), serviceLister: listers.GetServiceLister(), endpointsLister: listers.GetEndpointsLister(), - kafkaClusterAdmin: &mockClusterAdmin{ - mockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { + kafkaClusterAdmin: &commontesting.MockClusterAdmin{ + MockCreateTopicFunc: func(topic string, detail *sarama.TopicDetail, validateOnly bool) error { errMsg := sarama.ErrTopicAlreadyExists.Error() return &sarama.TopicError{ Err: sarama.ErrTopicAlreadyExists, ErrMsg: &errMsg, } }, + MockListConsumerGroupsFunc: func() (map[string]string, error) { + cgs := map[string]string{ + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub1UID): "consumer", + fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub2UID): "consumer", + } + return cgs, nil + }, }, + kafkaClient: &mockKafkaClient{saramaConfig}, kafkaClientSet: fakekafkaclient.Get(ctx), KubeClientSet: kubeclient.Get(ctx), EventingClientSet: eventingClient.Get(ctx), - statusManager: &fakeStatusManager{ - FakeIsReady: func(ctx context.Context, channel v1beta1.KafkaChannel, - spec eventingduckv1.SubscriberSpec) (bool, error) { - return true, nil - }, - }, } return kafkachannel.NewReconciler(ctx, logging.FromContext(ctx), r.kafkaClientSet, listers.GetKafkaChannelLister(), controller.GetEventRecorder(ctx), r) }, zap.L())) } -type mockClusterAdmin struct { - mockCreateTopicFunc func(topic string, detail *sarama.TopicDetail, validateOnly bool) error - mockDeleteTopicFunc func(topic string) error +type mockKafkaClient struct { + saramaConfig *sarama.Config } -func (ca *mockClusterAdmin) AlterPartitionReassignments(topic string, assignment [][]int32) error { - return nil +func (c mockKafkaClient) Config() *sarama.Config { + return c.saramaConfig } -func (ca *mockClusterAdmin) ListPartitionReassignments(topics string, partitions []int32) (topicStatus map[string]map[int32]*sarama.PartitionReplicaReassignmentsStatus, err error) { +func (c mockKafkaClient) Controller() (*sarama.Broker, error) { return nil, nil } -func (ca *mockClusterAdmin) DescribeLogDirs(brokers []int32) (map[int32][]sarama.DescribeLogDirsResponseDirMetadata, error) { +func (c mockKafkaClient) RefreshController() (*sarama.Broker, error) { return nil, nil } -func (ca *mockClusterAdmin) DescribeUserScramCredentials(users []string) ([]*sarama.DescribeUserScramCredentialsResult, error) { - return nil, nil +func (c mockKafkaClient) Brokers() []*sarama.Broker { + return nil } -func (ca *mockClusterAdmin) DeleteUserScramCredentials(delete []sarama.AlterUserScramCredentialsDelete) ([]*sarama.AlterUserScramCredentialsResult, error) { +func (c mockKafkaClient) Broker(brokerID int32) (*sarama.Broker, error) { return nil, nil } -func (ca *mockClusterAdmin) UpsertUserScramCredentials(upsert []sarama.AlterUserScramCredentialsUpsert) ([]*sarama.AlterUserScramCredentialsResult, error) { +func (c mockKafkaClient) Topics() ([]string, error) { return nil, nil } -func (ca *mockClusterAdmin) CreateTopic(topic string, detail *sarama.TopicDetail, validateOnly bool) error { - if ca.mockCreateTopicFunc != nil { - return ca.mockCreateTopicFunc(topic, detail, validateOnly) - } - return nil -} - -func (ca *mockClusterAdmin) Close() error { - return nil -} - -func (ca *mockClusterAdmin) DeleteTopic(topic string) error { - if ca.mockDeleteTopicFunc != nil { - return ca.mockDeleteTopicFunc(topic) - } - return nil +func (c mockKafkaClient) Partitions(topic string) ([]int32, error) { + return nil, nil } -func (ca *mockClusterAdmin) DescribeTopics(topics []string) (metadata []*sarama.TopicMetadata, err error) { +func (c mockKafkaClient) WritablePartitions(topic string) ([]int32, error) { return nil, nil } -func (ca *mockClusterAdmin) ListTopics() (map[string]sarama.TopicDetail, error) { +func (c mockKafkaClient) Leader(topic string, partitionID int32) (*sarama.Broker, error) { return nil, nil } -func (ca *mockClusterAdmin) CreatePartitions(topic string, count int32, assignment [][]int32, validateOnly bool) error { - return nil +func (c mockKafkaClient) Replicas(topic string, partitionID int32) ([]int32, error) { + return nil, nil } -func (ca *mockClusterAdmin) DeleteRecords(topic string, partitionOffsets map[int32]int64) error { - return nil +func (c mockKafkaClient) InSyncReplicas(topic string, partitionID int32) ([]int32, error) { + return nil, nil } -func (ca *mockClusterAdmin) DescribeConfig(resource sarama.ConfigResource) ([]sarama.ConfigEntry, error) { +func (c mockKafkaClient) OfflineReplicas(topic string, partitionID int32) ([]int32, error) { return nil, nil } -func (ca *mockClusterAdmin) AlterConfig(resourceType sarama.ConfigResourceType, name string, entries map[string]*string, validateOnly bool) error { +func (c mockKafkaClient) RefreshBrokers(addrs []string) error { return nil } -func (ca *mockClusterAdmin) CreateACL(resource sarama.Resource, acl sarama.Acl) error { +func (c mockKafkaClient) RefreshMetadata(topics ...string) error { return nil } -func (ca *mockClusterAdmin) ListAcls(filter sarama.AclFilter) ([]sarama.ResourceAcls, error) { - return nil, nil +func (c mockKafkaClient) GetOffset(topic string, partitionID int32, time int64) (int64, error) { + return 0, nil } -func (ca *mockClusterAdmin) DeleteACL(filter sarama.AclFilter, validateOnly bool) ([]sarama.MatchingAcl, error) { +func (c mockKafkaClient) Coordinator(consumerGroup string) (*sarama.Broker, error) { return nil, nil } -func (ca *mockClusterAdmin) ListConsumerGroups() (map[string]string, error) { - cgs := map[string]string{ - fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub1UID): "consumer", - fmt.Sprintf("kafka.%s.%s.%s", kcName, testNS, sub2UID): "consumer", - } - return cgs, nil +func (c mockKafkaClient) RefreshCoordinator(consumerGroup string) error { + return nil } -func (ca *mockClusterAdmin) DescribeConsumerGroups(groups []string) ([]*sarama.GroupDescription, error) { +func (c mockKafkaClient) InitProducerID() (*sarama.InitProducerIDResponse, error) { return nil, nil } -func (ca *mockClusterAdmin) ListConsumerGroupOffsets(group string, topicPartitions map[string][]int32) (*sarama.OffsetFetchResponse, error) { - return &sarama.OffsetFetchResponse{}, nil -} - -func (ca *mockClusterAdmin) DescribeCluster() (brokers []*sarama.Broker, controllerID int32, err error) { - return nil, 0, nil -} - -// Delete a consumer group. -func (ca *mockClusterAdmin) DeleteConsumerGroup(group string) error { +func (c mockKafkaClient) Close() error { return nil } -var _ sarama.ClusterAdmin = (*mockClusterAdmin)(nil) +func (c mockKafkaClient) Closed() bool { + return false +} func makeDeploymentWithParams(image string, replicas int32, configMapHash string, objMeta bool) *appsv1.Deployment { args := resources.DispatcherDeploymentArgs{ @@ -1087,22 +1099,6 @@ func subscribers() []eventingduckv1.SubscriberSpec { }} } -type fakeStatusManager struct { - FakeIsReady func(context.Context, v1beta1.KafkaChannel, eventingduckv1.SubscriberSpec) (bool, error) -} - -func (m *fakeStatusManager) IsReady(ctx context.Context, ch v1beta1.KafkaChannel, sub eventingduckv1.SubscriberSpec) (bool, error) { - return m.FakeIsReady(ctx, ch, sub) -} - -func (m *fakeStatusManager) CancelProbing(sub eventingduckv1.SubscriberSpec) { - //do nothing -} - -func (m *fakeStatusManager) CancelPodProbing(pod corev1.Pod) { - //do nothing -} - func makePatch(namespace, name, patch string) clientgotesting.PatchActionImpl { return clientgotesting.PatchActionImpl{ ActionImpl: clientgotesting.ActionImpl{ diff --git a/pkg/channel/consolidated/reconciler/controller/lister.go b/pkg/channel/consolidated/reconciler/controller/lister.go deleted file mode 100644 index 07fbc68169..0000000000 --- a/pkg/channel/consolidated/reconciler/controller/lister.go +++ /dev/null @@ -1,83 +0,0 @@ -/* -Copyright 2020 The Knative Authors - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package controller - -import ( - "context" - "fmt" - "net/url" - - "knative.dev/eventing-kafka/pkg/channel/consolidated/status" - - "go.uber.org/zap" - "k8s.io/apimachinery/pkg/util/sets" - v1 "k8s.io/client-go/listers/core/v1" - "knative.dev/eventing-kafka/pkg/apis/messaging/v1beta1" - "knative.dev/eventing/pkg/apis/eventing" - "knative.dev/pkg/system" -) - -type DispatcherPodsLister struct { - logger *zap.SugaredLogger - endpointLister v1.EndpointsLister -} - -func (t *DispatcherPodsLister) ListProbeTargets(ctx context.Context, kc v1beta1.KafkaChannel) (*status.ProbeTarget, error) { - scope, ok := kc.Annotations[eventing.ScopeAnnotationKey] - if !ok { - scope = scopeCluster - } - - dispatcherNamespace := system.Namespace() - if scope == scopeNamespace { - dispatcherNamespace = kc.Namespace - } - - // Get the Dispatcher Service Endpoints and propagate the status to the Channel - // endpoints has the same name as the service, so not a bug. - eps, err := t.endpointLister.Endpoints(dispatcherNamespace).Get(dispatcherName) - if err != nil { - return nil, fmt.Errorf("failed to get internal service: %w", err) - } - var readyIPs []string - - for _, sub := range eps.Subsets { - for _, address := range sub.Addresses { - readyIPs = append(readyIPs, address.IP) - } - } - - if len(readyIPs) == 0 { - return nil, fmt.Errorf("no gateway pods available") - } - - u, _ := url.Parse(fmt.Sprintf("http://%s.%s/%s/%s", dispatcherName, dispatcherNamespace, kc.Namespace, kc.Name)) - - return &status.ProbeTarget{ - PodIPs: sets.NewString(readyIPs...), - PodPort: "8081", - URL: u, - }, nil -} - -func NewProbeTargetLister(logger *zap.SugaredLogger, lister v1.EndpointsLister) status.ProbeTargetLister { - tl := DispatcherPodsLister{ - logger: logger, - endpointLister: lister, - } - return &tl -} diff --git a/pkg/channel/consolidated/reconciler/dispatcher/kafkachannel.go b/pkg/channel/consolidated/reconciler/dispatcher/kafkachannel.go index e0a26bdfc4..ccebe1dece 100644 --- a/pkg/channel/consolidated/reconciler/dispatcher/kafkachannel.go +++ b/pkg/channel/consolidated/reconciler/dispatcher/kafkachannel.go @@ -183,7 +183,7 @@ func (r *Reconciler) syncChannel(ctx context.Context, kc *v1beta1.KafkaChannel) } // Update dispatcher side - err := r.kafkaDispatcher.ReconcileConsumers(config) + err := r.kafkaDispatcher.ReconcileConsumers(ctx, config) if err != nil { logging.FromContext(ctx).Errorw("Some kafka subscriptions failed to subscribe", zap.Error(err)) return fmt.Errorf("some kafka subscriptions failed to subscribe: %v", err) diff --git a/pkg/channel/consolidated/status/status.go b/pkg/channel/consolidated/status/status.go deleted file mode 100644 index b930a46193..0000000000 --- a/pkg/channel/consolidated/status/status.go +++ /dev/null @@ -1,533 +0,0 @@ -/* -Copyright 2019 The Knative Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package status - -import ( - "context" - "encoding/json" - "fmt" - "net" - "net/http" - "net/url" - "reflect" - "sync" - "time" - - "go.uber.org/atomic" - "go.uber.org/zap" - "golang.org/x/time/rate" - corev1 "k8s.io/api/core/v1" - "k8s.io/apimachinery/pkg/types" - "k8s.io/apimachinery/pkg/util/sets" - "k8s.io/client-go/util/workqueue" - - messagingv1beta1 "knative.dev/eventing-kafka/pkg/apis/messaging/v1beta1" - eventingduckv1 "knative.dev/eventing/pkg/apis/duck/v1" - "knative.dev/networking/pkg/prober" - "knative.dev/pkg/logging" -) - -const ( - // probeConcurrency defines how many probing calls can be issued simultaneously - probeConcurrency = 100 - // probeTimeout defines the maximum amount of time a request will wait - probeTimeout = 1 * time.Second - // initialDelay defines the delay before enqueuing a probing request the first time. - // It gives times for the change to propagate and prevents unnecessary retries. - initialDelay = 200 * time.Millisecond -) - -var dialContext = (&net.Dialer{Timeout: probeTimeout}).DialContext - -// targetState represents the probing state of a subscription -type targetState struct { - sub eventingduckv1.SubscriberSpec - ch messagingv1beta1.KafkaChannel - - readyLock sync.RWMutex - // pendingCount is the number of pods that haven't been successfully probed yet - pendingCount atomic.Int32 - // readyCount is the number of pods that have the subscription ready - readyPartitions sets.Int - probedPods sets.String - lastAccessed time.Time - ready bool - cancel func() -} - -// podState represents the probing state of a Pod (for a specific subscription) -type podState struct { - // pendingCount is the number of probes for the Pod - pendingCount atomic.Int32 - - cancel func() -} - -// cancelContext is a pair of a Context and its cancel function -type cancelContext struct { - context context.Context - cancel func() -} - -type workItem struct { - targetStates *targetState - podState *podState - context context.Context - url *url.URL - podIP string - podPort string - logger *zap.SugaredLogger -} - -// ProbeTarget contains the URLs to probes for a set of Pod IPs serving out of the same port. -type ProbeTarget struct { - PodIPs sets.String - PodPort string - URL *url.URL -} - -// ProbeTargetLister lists all the targets that requires probing. -type ProbeTargetLister interface { - // ListProbeTargets returns a list of targets to be probed - ListProbeTargets(ctx context.Context, ch messagingv1beta1.KafkaChannel) (*ProbeTarget, error) -} - -// Manager provides a way to check if a Subscription is ready -type Manager interface { - IsReady(ctx context.Context, ch messagingv1beta1.KafkaChannel, sub eventingduckv1.SubscriberSpec) (bool, error) - CancelProbing(sub eventingduckv1.SubscriberSpec) - CancelPodProbing(pod corev1.Pod) -} - -// Prober provides a way to check if a VirtualService is ready by probing the Envoy pods -// handling that VirtualService. -type Prober struct { - logger *zap.SugaredLogger - - // mu guards targetStates and podContexts - mu sync.Mutex - targetStates map[types.UID]*targetState - podContexts map[string]cancelContext - - workQueue workqueue.RateLimitingInterface - - targetLister ProbeTargetLister - - readyCallback func(messagingv1beta1.KafkaChannel, eventingduckv1.SubscriberSpec) - - probeConcurrency int - - opts []interface{} -} - -// NewProber creates a new instance of Prober -func NewProber( - logger *zap.SugaredLogger, - targetLister ProbeTargetLister, - readyCallback func(messagingv1beta1.KafkaChannel, eventingduckv1.SubscriberSpec), opts ...interface{}) *Prober { - return &Prober{ - logger: logger, - targetStates: make(map[types.UID]*targetState), - podContexts: make(map[string]cancelContext), - workQueue: workqueue.NewNamedRateLimitingQueue( - workqueue.NewMaxOfRateLimiter( - // Per item exponential backoff - workqueue.NewItemExponentialFailureRateLimiter(100*time.Millisecond, 5*time.Minute), - // Global rate limiter - &workqueue.BucketRateLimiter{Limiter: rate.NewLimiter(rate.Limit(50), 1000)}, - ), - "ProbingQueue"), - targetLister: targetLister, - readyCallback: readyCallback, - probeConcurrency: probeConcurrency, - opts: opts, - } -} - -func (m *Prober) checkReadiness(state *targetState) bool { - state.readyLock.Lock() - defer state.readyLock.Unlock() - partitions := state.ch.Spec.NumPartitions - if !state.ready { - state.ready = state.readyPartitions.Len() == int(partitions) - } - m.logger.Debugw("Checking subscription readiness", - zap.Any("subscription", state.sub.UID), - zap.Any("channel", state.ch.Name), - zap.Any("pod ips", state.probedPods), - zap.Any("channel partitions", partitions), - zap.Any("ready partitions", state.readyPartitions.List()), - zap.Bool("ready", state.ready), - ) - return state.ready -} - -func (m *Prober) IsReady(ctx context.Context, ch messagingv1beta1.KafkaChannel, sub eventingduckv1.SubscriberSpec) (bool, error) { - logger := logging.FromContext(ctx) - - // Get the probe targets - target, err := m.targetLister.ListProbeTargets(ctx, ch) - if err != nil { - logger.Errorw("Error listing probe targets", zap.Error(err), - zap.Any("subscription", sub.UID)) - return false, err - } - // get the state while locking for very short scope - state, ok := func() (*targetState, bool) { - m.mu.Lock() - defer m.mu.Unlock() - s, o := m.targetStates[sub.UID] - return s, o - }() - if ok { - if !isOutdatedTargetState(state, sub, target.PodIPs) { - return m.checkReadiness(state), nil - } - m.ejectStateUnsafe(sub) - } - m.probeTarget(ctx, ch, sub, target) - return false, nil -} - -func (m *Prober) probeTarget(ctx context.Context, ch messagingv1beta1.KafkaChannel, sub eventingduckv1.SubscriberSpec, target *ProbeTarget) { - subscriptionKey := sub.UID - logger := logging.FromContext(ctx) - subCtx, cancel := context.WithCancel(context.Background()) - subscriptionState := &targetState{ - sub: sub, - ch: ch, - lastAccessed: time.Now(), - cancel: cancel, - } - - // Group the probe targets by IP - workItems := make(map[string][]*workItem) - for ip := range target.PodIPs { - workItems[ip] = append(workItems[ip], &workItem{ - targetStates: subscriptionState, - url: target.URL, - podIP: ip, - podPort: target.PodPort, - logger: logger, - }) - } - - subscriptionState.probedPods = target.PodIPs - subscriptionState.pendingCount.Store(int32(len(workItems))) - subscriptionState.readyPartitions = sets.Int{} - - for ip, ipWorkItems := range workItems { - // Get or create the context for that IP - ipCtx := func() context.Context { - m.mu.Lock() - defer m.mu.Unlock() - cancelCtx, ok := m.podContexts[ip] - if !ok { - ctx, cancel := context.WithCancel(context.Background()) - cancelCtx = cancelContext{ - context: ctx, - cancel: cancel, - } - m.podContexts[ip] = cancelCtx - } - return cancelCtx.context - }() - - podCtx, cancel := context.WithCancel(subCtx) - podState := &podState{ - pendingCount: *atomic.NewInt32(int32(len(ipWorkItems))), - cancel: cancel, - } - - // Quick and dirty way to join two contexts (i.e. podCtx is cancelled when either subCtx or ipCtx are cancelled) - go func() { - select { - case <-podCtx.Done(): - // This is the actual context, there is nothing to do except - // break to avoid leaking this goroutine. - break - case <-ipCtx.Done(): - // Cancel podCtx - cancel() - } - }() - - // Update the states when probing is cancelled - go func() { - <-podCtx.Done() - m.onProbingCancellation(subscriptionState, podState) - }() - - for _, wi := range ipWorkItems { - wi.podState = podState - wi.context = podCtx - m.workQueue.AddAfter(wi, initialDelay) - logger.Infof("Queuing probe for %s, IP: %s:%s (depth: %d)", - wi.url, wi.podIP, wi.podPort, m.workQueue.Len()) - } - } - - func() { - m.mu.Lock() - defer m.mu.Unlock() - m.targetStates[subscriptionKey] = subscriptionState - }() -} - -// Start starts the Manager background operations -func (m *Prober) Start(done <-chan struct{}) chan struct{} { - var wg sync.WaitGroup - - // Start the worker goroutines - for i := 0; i < m.probeConcurrency; i++ { - wg.Add(1) - go func() { - defer wg.Done() - for m.processWorkItem() { - } - }() - } - - // Stop processing the queue when cancelled - go func() { - <-done - m.workQueue.ShutDown() - }() - - // Return a channel closed when all work is done - ch := make(chan struct{}) - go func() { - wg.Wait() - close(ch) - }() - return ch -} - -// CancelProbing cancels probing of the provided Subscription -func (m *Prober) CancelProbing(sub eventingduckv1.SubscriberSpec) { - m.mu.Lock() - defer m.mu.Unlock() - m.ejectStateUnsafe(sub) -} - -// ejectStateUnsafe ejects a state from Cache, it's not safe for concurrent access and is meant for internal use only under proper locking. -func (m *Prober) ejectStateUnsafe(sub eventingduckv1.SubscriberSpec) { - if state, ok := m.targetStates[sub.UID]; ok { - m.logger.Debugw("Canceling state", zap.Any("subscription", sub)) - state.cancel() - delete(m.targetStates, sub.UID) - } -} - -// CancelPodProbing cancels probing of the provided Pod IP. -func (m *Prober) CancelPodProbing(pod corev1.Pod) { - m.mu.Lock() - defer m.mu.Unlock() - - if ctx, ok := m.podContexts[pod.Status.PodIP]; ok { - ctx.cancel() - delete(m.podContexts, pod.Status.PodIP) - } -} - -// RefreshPodProbing lists probe targets and invalidates any in-flight (non-ready) states whose initial probed targets changed from the -// current ones. -func (m *Prober) RefreshPodProbing(ctx context.Context) { - m.mu.Lock() - defer m.mu.Unlock() - logger := logging.FromContext(ctx) - for _, state := range m.targetStates { - if !m.checkReadiness(state) { - // This is an in-flight state - sub := state.sub - ch := state.ch - // Get the probe targets - target, err := m.targetLister.ListProbeTargets(ctx, ch) - if err != nil { - logger.Errorw("Error listing probe targets", zap.Error(err), - zap.Any("subscription", sub.UID)) - return - } - m.ejectStateUnsafe(sub) - func() { - // probeTarget requires an unlocked mutex. - m.mu.Unlock() - defer m.mu.Lock() - m.probeTarget(ctx, ch, sub, target) - }() - } - } -} - -// processWorkItem processes a single work item from workQueue. -// It returns false when there is no more items to process, true otherwise. -func (m *Prober) processWorkItem() bool { - obj, shutdown := m.workQueue.Get() - if shutdown { - return false - } - - defer m.workQueue.Done(obj) - - // Crash if the item is not of the expected type - item, ok := obj.(*workItem) - if !ok { - m.logger.Fatalf("Unexpected work item type: want: %s, got: %s\n", - reflect.TypeOf(&workItem{}).Name(), reflect.TypeOf(obj).Name()) - } - item.logger.Infof("Processing probe for %s, IP: %s:%s (depth: %d)", - item.url, item.podIP, item.podPort, m.workQueue.Len()) - - transport := http.DefaultTransport.(*http.Transport).Clone() - - transport.DialContext = func(ctx context.Context, network, addr string) (conn net.Conn, e error) { - // http.Request.URL is set to the hostname and it is substituted in here with the target IP. - return dialContext(ctx, network, net.JoinHostPort(item.podIP, item.podPort)) - } - - probeURL := deepCopy(item.url) - - ctx, cancel := context.WithTimeout(item.context, probeTimeout) - defer cancel() - var opts []interface{} - opts = append(opts, m.opts...) - opts = append(opts, m.probeVerifier(item)) - - ok, err := prober.Do( - ctx, - transport, - probeURL.String(), - opts...) - - // In case of cancellation, drop the work item - select { - case <-item.context.Done(): - m.workQueue.Forget(obj) - return true - default: - } - - if err != nil { - // In case of error, enqueue for retry - m.workQueue.AddRateLimited(obj) - item.logger.Errorw("Error occurred while probing", - zap.Any("url", item.url), zap.Any("IP", item.podIP), - zap.Any("port", item.podPort), zap.Bool("ready", ok), zap.Error(err), - zap.Int("depth", m.workQueue.Len())) - } else if !ok { - // No error, but verification failed, enqueue for retry - m.workQueue.AddRateLimited(obj) - item.logger.Debugw("Verification of pod response failed.", - zap.Any("url", item.url), zap.Any("IP", item.podIP), - zap.Any("port", item.podPort), zap.Bool("ready", ok), zap.Error(err), - zap.Int("depth", m.workQueue.Len())) - } else { - m.onProbingSuccess(item.targetStates, item.podState) - } - return true -} - -func (m *Prober) onProbingSuccess(subscriptionState *targetState, podState *podState) { - // The last probe call for the Pod succeeded, the Pod is ready - if podState.pendingCount.Dec() == 0 { - // Unlock the goroutine blocked on <-podCtx.Done() - podState.cancel() - // This is the last pod being successfully probed, the subscription might ready - if m.checkReadiness(subscriptionState) { - subscriptionState.cancel() - m.readyCallback(subscriptionState.ch, subscriptionState.sub) - } - } -} - -func (m *Prober) onProbingCancellation(subscriptionState *targetState, podState *podState) { - for { - pendingCount := podState.pendingCount.Load() - if pendingCount <= 0 { - // Probing succeeded, nothing to do - return - } - // Attempt to set pendingCount to 0. - if podState.pendingCount.CAS(pendingCount, 0) { - // This is the last pod being successfully probed, the subscription might be ready - if subscriptionState.pendingCount.Dec() == 0 { - if m.checkReadiness(subscriptionState) { - m.readyCallback(subscriptionState.ch, subscriptionState.sub) - } - } - return - } - } -} - -func (m *Prober) probeVerifier(item *workItem) prober.Verifier { - return func(r *http.Response, b []byte) (bool, error) { - m.logger.Debugw("Verifying response", zap.Int("status code", r.StatusCode), - zap.ByteString("body", b), zap.Any("subscription", item.targetStates.sub.UID), - zap.Any("channel", item.targetStates.ch)) - switch r.StatusCode { - case http.StatusOK: - var subscriptions = make(map[string][]int) - err := json.Unmarshal(b, &subscriptions) - if err != nil { - m.logger.Errorw("error unmarshaling", err) - return false, err - } - uid := string(item.targetStates.sub.UID) - key := fmt.Sprintf("%s/%s", item.targetStates.ch.Namespace, item.targetStates.ch.Name) - m.logger.Debugw("Received proper probing response from target", - zap.Any("found subscriptions", subscriptions), - zap.String("pod ip", item.podIP), - zap.String("want channel", key), - zap.String("want subscription", uid), - ) - if partitions, ok := subscriptions[uid]; ok { - func() { - item.targetStates.readyLock.Lock() - defer item.targetStates.readyLock.Unlock() - item.targetStates.readyPartitions.Insert(partitions...) - }() - return m.checkReadiness(item.targetStates), nil - } else { - return false, nil - } - case http.StatusNotFound, http.StatusServiceUnavailable: - m.logger.Errorf("unexpected status code: want %v, got %v", http.StatusOK, r.StatusCode) - return false, fmt.Errorf("unexpected status code: want %v, got %v", http.StatusOK, r.StatusCode) - default: - item.logger.Errorf("Probing of %s abandoned, IP: %s:%s: the response status is %v, expected one of: %v", - item.url, item.podIP, item.podPort, r.StatusCode, - []int{http.StatusOK, http.StatusNotFound, http.StatusServiceUnavailable}) - return true, nil - } - } -} - -// A target state is outdated if the generation is different or if the target IPs change before it becomes -// ready. -func isOutdatedTargetState(state *targetState, sub eventingduckv1.SubscriberSpec, podIPs sets.String) bool { - state.readyLock.RLock() - defer state.readyLock.RUnlock() - return state.sub.Generation != sub.Generation || (!state.ready && !state.probedPods.Equal(podIPs)) -} - -// deepCopy copies a URL into a new one -func deepCopy(in *url.URL) *url.URL { - // Safe to ignore the error since this is a deep copy - newURL, _ := url.Parse(in.String()) - return newURL -} diff --git a/pkg/channel/consolidated/status/status_test.go b/pkg/channel/consolidated/status/status_test.go deleted file mode 100644 index 4d99781f33..0000000000 --- a/pkg/channel/consolidated/status/status_test.go +++ /dev/null @@ -1,340 +0,0 @@ -/* -Copyright 2019 The Knative Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package status - -import ( - "context" - "encoding/json" - "errors" - "net/http" - "net/http/httptest" - "net/url" - "strconv" - "testing" - "time" - - "go.uber.org/atomic" - "go.uber.org/zap/zaptest" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - "k8s.io/apimachinery/pkg/types" - "k8s.io/apimachinery/pkg/util/sets" - - "knative.dev/eventing-kafka/pkg/apis/messaging/v1beta1" - messagingv1beta1 "knative.dev/eventing-kafka/pkg/apis/messaging/v1beta1" - eventingduckv1 "knative.dev/eventing/pkg/apis/duck/v1" - "knative.dev/pkg/apis" -) - -var channelObjectMeta = metav1.ObjectMeta{ - Namespace: "default", - Name: "chan4prober", -} - -const dispatcherReadySubHeader = "K-Subscriber-Status" - -type ReadyPair struct { - c v1beta1.KafkaChannel - s eventingduckv1.SubscriberSpec -} - -func TestProbeSinglePod(t *testing.T) { - ch := getChannel(1) - sub := getSubscription() - var subscriptions = map[string][]int{ - string(sub.UID): {0}, - } - - // This should be called when we want the dispatcher to return a successful result - successHandler := http.HandlerFunc(readyJSONHandler(t, subscriptions)) - - // Probes only succeed if succeed is true - var succeed atomic.Bool - - // This is a latch channel that will lock the handler goroutine until we drain it - probeRequests := make(chan *http.Request) - handler := func(w http.ResponseWriter, r *http.Request) { - probeRequests <- r - if !succeed.Load() { - w.WriteHeader(http.StatusNotFound) - return - } - - successHandler.ServeHTTP(w, r) - } - - ts := getDispatcherServer(handler) - defer ts.Close() - - lister := fakeProbeTargetLister{ - target: getTargetLister(t, ts.URL), - } - - prober, ready := getProber(t, &lister) - - done := make(chan struct{}) - cancelled := prober.Start(done) - defer func() { - close(done) - <-cancelled - }() - - assertEventuallyReady(t, prober, ch, sub, ready, &succeed, &probeRequests) -} - -func TestProbeListerFail(t *testing.T) { - ch := getChannel(1) - sub := getSubscription() - - ready := make(chan *ReadyPair) - defer close(ready) - prober := NewProber( - zaptest.NewLogger(t).Sugar(), - notFoundLister{}, - func(c v1beta1.KafkaChannel, s eventingduckv1.SubscriberSpec) { - ready <- &ReadyPair{ - c, - s, - } - }) - - // If we can't list, this must fail and return false - ok, err := prober.IsReady(context.Background(), *ch, *sub) - if err == nil { - t.Fatal("IsReady returned unexpected success") - } - if ok { - t.Fatal("IsReady() returned true") - } -} - -func TestSucceedAfterRefreshPodProbing(t *testing.T) { - // We have a channel with three partitions - ch := getChannel(3) - sub := getSubscription() - - // Dispatcher D1 will return only one ready partition - var subsD1 = map[string][]int{ - string(sub.UID): {0}, - } - - // Dispatcher D2 will return the three ready partitions - var subsD2 = map[string][]int{ - string(sub.UID): {0, 1, 2}, - } - - // The success handler for dispatcher D1, will return one partition only - successHandlerD1 := http.HandlerFunc(readyJSONHandler(t, subsD1)) - - // This is a latch channel that will lock the handler goroutine until we drain it - probeRequestsD1 := make(chan *http.Request) - - handlerD1 := func(w http.ResponseWriter, r *http.Request) { - probeRequestsD1 <- r - successHandlerD1.ServeHTTP(w, r) - } - - serverD1 := getDispatcherServer(handlerD1) - defer serverD1.Close() - - // Probes only succeed if succeed is true - var succeed atomic.Bool - - // The success handler for dispatcher D2, will return all three needed partitions - probeHandlerD2 := http.HandlerFunc(readyJSONHandler(t, subsD2)) - - // This is a latch channel that will lock the handler goroutine until we drain it - probeRequestsD2 := make(chan *http.Request) - handlerD2 := func(w http.ResponseWriter, r *http.Request) { - probeRequestsD2 <- r - if !succeed.Load() { - w.WriteHeader(http.StatusNotFound) - return - } - - probeHandlerD2.ServeHTTP(w, r) - } - - serverD2 := getDispatcherServer(handlerD2) - defer serverD2.Close() - - // Initially, lister points to d1 - lister := fakeProbeTargetLister{ - target: getTargetLister(t, serverD1.URL), - } - - prober, ready := getProber(t, &lister) - - done := make(chan struct{}) - cancelled := prober.Start(done) - defer func() { - close(done) - <-cancelled - }() - - // Assert we're not ready. - assertNeverReady(t, prober, ch, sub, ready, &probeRequestsD1) - - // Switch to new dispatcher - lister.target = getTargetLister(t, serverD2.URL) - - // Assert we're still not ready - assertNeverReady(t, prober, ch, sub, ready, &probeRequestsD1) - - // Refresh the pod probing, now the prober should probe the new disptacher - prober.RefreshPodProbing(context.Background()) - - // Assert that probing will be successful eventually - assertEventuallyReady(t, prober, ch, sub, ready, &succeed, &probeRequestsD2) -} - -func assertNeverReady(t *testing.T, prober *Prober, ch *messagingv1beta1.KafkaChannel, sub *eventingduckv1.SubscriberSpec, ready chan *ReadyPair, probeRequests *chan *http.Request) { - // The first call to IsReady must succeed and return false - ok, err := prober.IsReady(context.Background(), *ch, *sub) - if err != nil { - t.Fatal("IsReady failed:", err) - } - if ok { - t.Fatal("IsReady() returned true") - } - - // Just drain the requests in the channel to not block the handler - go func() { - for range *probeRequests { - } - }() - - select { - case <-ready: - // Prober shouldn't be ready - t.Fatal("Prober shouldn't be ready") - case <-time.After(1 * time.Second): - // Not ideal but it gives time to the prober to write to ready - break - } -} - -func assertEventuallyReady(t *testing.T, prober *Prober, ch *messagingv1beta1.KafkaChannel, sub *eventingduckv1.SubscriberSpec, ready chan *ReadyPair, succeed *atomic.Bool, probeRequests *chan *http.Request) { - - // Since succeed is still false the prober shouldn't be ready - assertNeverReady(t, prober, ch, sub, ready, probeRequests) - - // Make probes succeed - succeed.Store(true) - - // Just drain the requests in the channel to not block the handler - go func() { - for range *probeRequests { - } - }() - - select { - case <-ready: - // Wait for the probing to eventually succeed - case <-time.After(5 * time.Second): - t.Error("Timed out waiting for probing to succeed.") - } -} - -func getDispatcherServer(handler func(w http.ResponseWriter, r *http.Request)) *httptest.Server { - ts := httptest.NewServer(http.HandlerFunc(handler)) - return ts -} - -func getChannel(partitionsNum int32) *v1beta1.KafkaChannel { - return (&v1beta1.KafkaChannel{ - ObjectMeta: channelObjectMeta, - Spec: v1beta1.KafkaChannelSpec{ - NumPartitions: partitionsNum, - ReplicationFactor: 1, - }, - }).DeepCopy() -} - -func getSubscription() *eventingduckv1.SubscriberSpec { - return (&eventingduckv1.SubscriberSpec{ - UID: types.UID("90713ffd-f527-42bf-b158-57630b68ebe2"), - Generation: 1, - SubscriberURI: getURL("http://subscr.ns.local"), - }).DeepCopy() -} - -func getURL(s string) *apis.URL { - u, _ := apis.ParseURL(s) - return u -} - -// readyJSONHandler is a factory for a handler which responds with a JSON of the ready subscriptions -func readyJSONHandler(t *testing.T, subscriptions map[string][]int) func(http.ResponseWriter, *http.Request) { - return func(w http.ResponseWriter, r *http.Request) { - channelRefName := channelObjectMeta.Name - channelRefNamespace := channelObjectMeta.Namespace - w.Header().Set(dispatcherReadySubHeader, channelRefName) - jsonResult, err := json.Marshal(subscriptions) - if err != nil { - t.Fatalf("Error marshalling json for sub-status channelref: %s/%s, %v", channelRefNamespace, channelRefName, err) - } - _, err = w.Write(jsonResult) - if err != nil { - t.Fatalf("Error writing jsonResult to serveHTTP writer: %v", err) - } - } -} - -func getTargetLister(t *testing.T, dURL string) *ProbeTarget { - tsURL, err := url.Parse(dURL) - if err != nil { - t.Fatalf("Failed to parse URL %q: %v", dURL, err) - } - port, err := strconv.Atoi(tsURL.Port()) - if err != nil { - t.Fatalf("Failed to parse port %q: %v", tsURL.Port(), err) - } - hostname := tsURL.Hostname() - return &ProbeTarget{ - PodIPs: sets.NewString(hostname), - PodPort: strconv.Itoa(port), - URL: tsURL, - } -} - -func getProber(t *testing.T, lister ProbeTargetLister) (*Prober, chan *ReadyPair) { - ready := make(chan *ReadyPair) - prober := NewProber( - zaptest.NewLogger(t).Sugar(), - lister, - func(c v1beta1.KafkaChannel, s eventingduckv1.SubscriberSpec) { - ready <- &ReadyPair{ - c, - s, - } - }) - return prober, ready -} - -type fakeProbeTargetLister struct { - target *ProbeTarget -} - -func (l fakeProbeTargetLister) ListProbeTargets(ctx context.Context, kc messagingv1beta1.KafkaChannel) (*ProbeTarget, error) { - return l.target, nil -} - -type notFoundLister struct{} - -func (l notFoundLister) ListProbeTargets(ctx context.Context, kc messagingv1beta1.KafkaChannel) (*ProbeTarget, error) { - return nil, errors.New("not found") -} diff --git a/pkg/channel/distributed/dispatcher/controller/kafkachannel.go b/pkg/channel/distributed/dispatcher/controller/kafkachannel.go index 09f6ef8850..e47e0ec323 100644 --- a/pkg/channel/distributed/dispatcher/controller/kafkachannel.go +++ b/pkg/channel/distributed/dispatcher/controller/kafkachannel.go @@ -195,7 +195,7 @@ func (r Reconciler) Reconcile(ctx context.Context, key string) error { channel := original.DeepCopy() // Perform the reconciliation (will update KafkaChannel.Status) - reconcileError := r.reconcile(channel) + reconcileError := r.reconcile(ctx, channel) if reconcileError != nil { r.logger.Error("Error Reconciling KafkaChannel", zap.Error(reconcileError)) r.recorder.Eventf(channel, corev1.EventTypeWarning, channelReconcileFailed, "KafkaChannel Reconciliation Failed: %v", reconcileError) @@ -219,7 +219,7 @@ func (r Reconciler) Reconcile(ctx context.Context, key string) error { } // Reconcile The Specified KafkaChannel -func (r Reconciler) reconcile(channel *kafkav1beta1.KafkaChannel) error { +func (r Reconciler) reconcile(ctx context.Context, channel *kafkav1beta1.KafkaChannel) error { // The KafkaChannel's Subscribers var subscribers []eventingduck.SubscriberSpec @@ -232,7 +232,7 @@ func (r Reconciler) reconcile(channel *kafkav1beta1.KafkaChannel) error { } // Update The ConsumerGroups To Align With Current KafkaChannel Subscribers - subscriptions := r.dispatcher.UpdateSubscriptions(subscribers) + subscriptions := r.dispatcher.UpdateSubscriptions(ctx, subscribers) // Update The KafkaChannel Subscribable Status Based On ConsumerGroup Creation Status channel.Status.SubscribableStatus = r.createSubscribableStatus(channel.Spec.Subscribers, subscriptions) diff --git a/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go b/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go index cb2583e9a4..c5c545b415 100644 --- a/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go +++ b/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go @@ -365,7 +365,7 @@ func (m *MockDispatcher) Shutdown() { m.Called() } -func (m *MockDispatcher) UpdateSubscriptions(subscriberSpecs []eventingduck.SubscriberSpec) consumer.SubscriberStatusMap { +func (m *MockDispatcher) UpdateSubscriptions(_ context.Context, subscriberSpecs []eventingduck.SubscriberSpec) consumer.SubscriberStatusMap { args := m.Called(subscriberSpecs) return args.Get(0).(consumer.SubscriberStatusMap) } diff --git a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher.go b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher.go index 293602eb2f..f2d74c737d 100644 --- a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher.go +++ b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher.go @@ -65,7 +65,7 @@ func NewSubscriberWrapper(subscriberSpec eventingduck.SubscriberSpec, groupId st type Dispatcher interface { SecretChanged(ctx context.Context, secret *corev1.Secret) Shutdown() - UpdateSubscriptions(subscriberSpecs []eventingduck.SubscriberSpec) commonconsumer.SubscriberStatusMap + UpdateSubscriptions(ctx context.Context, subscriberSpecs []eventingduck.SubscriberSpec) commonconsumer.SubscriberStatusMap } // DispatcherImpl Is A Struct With Configuration & ConsumerGroup State @@ -85,7 +85,7 @@ var _ Dispatcher = &DispatcherImpl{} // NewDispatcher Is The Dispatcher Constructor func NewDispatcher(dispatcherConfig DispatcherConfig, controlServer controlprotocol.ServerHandler) (Dispatcher, <-chan commonconsumer.ManagerEvent) { - consumerGroupManager := commonconsumer.NewConsumerGroupManager(dispatcherConfig.Logger, controlServer, dispatcherConfig.Brokers, dispatcherConfig.SaramaConfig) + consumerGroupManager := commonconsumer.NewConsumerGroupManager(dispatcherConfig.Logger, controlServer, dispatcherConfig.Brokers, dispatcherConfig.SaramaConfig, &commonconsumer.NoopConsumerGroupOffsetsChecker{}) // Create The DispatcherImpl With Specified Configuration dispatcher := &DispatcherImpl{ @@ -123,7 +123,7 @@ func (d *DispatcherImpl) Shutdown() { } // UpdateSubscriptions manages the Dispatcher's Subscriptions to align with new state -func (d *DispatcherImpl) UpdateSubscriptions(subscriberSpecs []eventingduck.SubscriberSpec) commonconsumer.SubscriberStatusMap { +func (d *DispatcherImpl) UpdateSubscriptions(ctx context.Context, subscriberSpecs []eventingduck.SubscriberSpec) commonconsumer.SubscriberStatusMap { if d.SaramaConfig == nil { d.Logger.Error("Dispatcher has no config!") @@ -151,7 +151,7 @@ func (d *DispatcherImpl) UpdateSubscriptions(subscriberSpecs []eventingduck.Subs // Create/Start A New ConsumerGroup With Custom Handler handler := NewHandler(logger, groupId, &subscriberSpec) - err := d.consumerMgr.StartConsumerGroup(groupId, []string{d.Topic}, d.Logger.Sugar(), handler) + err := d.consumerMgr.StartConsumerGroup(ctx, groupId, []string{d.Topic}, handler) if err != nil { // Log & Return Failure diff --git a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go index d0cf859676..f529b63a56 100644 --- a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go +++ b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go @@ -365,7 +365,7 @@ func TestUpdateSubscriptions(t *testing.T) { if !testCase.wantFailure { mockManager.On("ClearNotifications").Return() for _, id := range testCase.expectStarted { - mockManager.On("StartConsumerGroup", "kafka."+id, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(testCase.createErr) + mockManager.On("StartConsumerGroup", mock.Anything, "kafka."+id, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(testCase.createErr) } for _, id := range testCase.expectErrors { mockManager.On("Errors", "kafka."+id).Return((<-chan error)(errorSource)) @@ -380,7 +380,7 @@ func TestUpdateSubscriptions(t *testing.T) { } // Perform The Test - result := dispatcher.UpdateSubscriptions(testCase.args.subscriberSpecs) + result := dispatcher.UpdateSubscriptions(ctx, testCase.args.subscriberSpecs) close(errorSource) diff --git a/pkg/common/consumer/consumer_factory.go b/pkg/common/consumer/consumer_factory.go index 0efe1eb366..dabaebf43c 100644 --- a/pkg/common/consumer/consumer_factory.go +++ b/pkg/common/consumer/consumer_factory.go @@ -22,9 +22,10 @@ import ( "github.com/Shopify/sarama" "go.uber.org/zap" + "knative.dev/pkg/logging" ) -// newConsumerGroup is a wrapper for the Sarama NewConsumerGroup function, to facilitate unit testing +// wrapper functions for the Sarama functions, to facilitate unit testing var newConsumerGroup = sarama.NewConsumerGroup // consumeFunc is a function type that matches the Sarama ConsumerGroup's Consume function @@ -32,12 +33,13 @@ type consumeFunc func(ctx context.Context, topics []string, handler sarama.Consu // KafkaConsumerGroupFactory creates the ConsumerGroup and start consuming the specified topic type KafkaConsumerGroupFactory interface { - StartConsumerGroup(groupID string, topics []string, logger *zap.SugaredLogger, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) + StartConsumerGroup(ctx context.Context, groupID string, topics []string, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) } type kafkaConsumerGroupFactoryImpl struct { - config *sarama.Config - addrs []string + config *sarama.Config + addrs []string + offsetsChecker ConsumerGroupOffsetsChecker } type customConsumerGroup struct { @@ -64,13 +66,17 @@ func (c *customConsumerGroup) Close() error { var _ sarama.ConsumerGroup = (*customConsumerGroup)(nil) // StartConsumerGroup creates a new customConsumerGroup and starts a Consume goroutine on it -func (c kafkaConsumerGroupFactoryImpl) StartConsumerGroup(groupID string, topics []string, logger *zap.SugaredLogger, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { +func (c kafkaConsumerGroupFactoryImpl) StartConsumerGroup(ctx context.Context, groupID string, topics []string, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { + logger := logging.FromContext(ctx) + consumerGroup, err := c.createConsumerGroup(groupID) if err != nil { + logger.Errorw("unable to create consumer group", zap.String("groupId", groupID), zap.Error(err)) return nil, err } + // Start the consumerGroup.Consume function in a separate goroutine - return c.startExistingConsumerGroup(consumerGroup, consumerGroup.Consume, topics, logger, handler, options...), nil + return c.startExistingConsumerGroup(groupID, consumerGroup, consumerGroup.Consume, topics, logger, handler, options...), nil } // createConsumerGroup creates a Sarama ConsumerGroup using the newConsumerGroup wrapper, with the @@ -82,6 +88,7 @@ func (c kafkaConsumerGroupFactoryImpl) createConsumerGroup(groupID string) (sara // startExistingConsumerGroup creates a goroutine that begins a custom Consume loop on the provided ConsumerGroup // This loop is cancelable via the function provided in the returned customConsumerGroup. func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup( + groupID string, saramaGroup sarama.ConsumerGroup, consume consumeFunc, topics []string, @@ -94,6 +101,16 @@ func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup( ctx, cancel := context.WithCancel(context.Background()) go func() { + // this is a blocking func + // do not proceed until the check is done + err := c.offsetsChecker.WaitForOffsetsInitialization(ctx, groupID, topics, logger, c.addrs, c.config) + if err != nil { + logger.Errorw("error while checking if offsets are initialized", zap.Any("topics", topics), zap.String("groupId", groupID), zap.Error(err)) + errorCh <- err + } + + logger.Debugw("all offsets are initialized", zap.Any("topics", topics), zap.Any("groupID", groupID)) + defer func() { close(errorCh) releasedCh <- true @@ -119,8 +136,8 @@ func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup( return &customConsumerGroup{cancel, errorCh, saramaGroup, releasedCh} } -func NewConsumerGroupFactory(addrs []string, config *sarama.Config) KafkaConsumerGroupFactory { - return kafkaConsumerGroupFactoryImpl{addrs: addrs, config: config} +func NewConsumerGroupFactory(addrs []string, config *sarama.Config, offsetsChecker ConsumerGroupOffsetsChecker) KafkaConsumerGroupFactory { + return kafkaConsumerGroupFactoryImpl{addrs: addrs, config: config, offsetsChecker: offsetsChecker} } var _ KafkaConsumerGroupFactory = (*kafkaConsumerGroupFactoryImpl)(nil) diff --git a/pkg/common/consumer/consumer_factory_test.go b/pkg/common/consumer/consumer_factory_test.go index acfb035494..2db27b3a9d 100644 --- a/pkg/common/consumer/consumer_factory_test.go +++ b/pkg/common/consumer/consumer_factory_test.go @@ -24,6 +24,8 @@ import ( "github.com/Shopify/sarama" "go.uber.org/zap" + controllertesting "knative.dev/eventing-kafka/pkg/common/commands/resetoffset/controller/testing" + commontesting "knative.dev/eventing-kafka/pkg/common/testing" ) //------ Mocks @@ -85,18 +87,58 @@ func mockedNewConsumerGroupFromClient(mockInputMessageCh chan *sarama.ConsumerMe } } +func mockedNewSaramaClient(client *controllertesting.MockClient, mustFail bool) func(addrs []string, config *sarama.Config) (sarama.Client, error) { + if !mustFail { + return func(addrs []string, config *sarama.Config) (sarama.Client, error) { + return client, nil + } + } else { + return func(addrs []string, config *sarama.Config) (sarama.Client, error) { + return nil, errors.New("failed") + } + } +} + +func mockedNewSaramaClusterAdmin(clusterAdmin sarama.ClusterAdmin, mustFail bool) func(addrs []string, config *sarama.Config) (sarama.ClusterAdmin, error) { + if !mustFail { + return func(addrs []string, config *sarama.Config) (sarama.ClusterAdmin, error) { + return clusterAdmin, nil + } + } else { + return func(addrs []string, config *sarama.Config) (sarama.ClusterAdmin, error) { + return nil, errors.New("failed") + } + } +} + //------ Tests +type mockConsumerGroupOffsetsChecker struct { +} + +func (m mockConsumerGroupOffsetsChecker) WaitForOffsetsInitialization(ctx context.Context, groupID string, topics []string, logger *zap.SugaredLogger, addrs []string, config *sarama.Config) error { + return nil +} + func TestErrorPropagationCustomConsumerGroup(t *testing.T) { + ctx := context.TODO() + client := controllertesting.NewMockClient( + controllertesting.WithClientMockClosed(false), + controllertesting.WithClientMockClose(nil)) + clusterAdmin := &commontesting.MockClusterAdmin{} + // override some functions newConsumerGroup = mockedNewConsumerGroupFromClient(nil, true, true, false, false) + newSaramaClient = mockedNewSaramaClient(client, false) + newSaramaClusterAdmin = mockedNewSaramaClusterAdmin(clusterAdmin, false) factory := kafkaConsumerGroupFactoryImpl{ - config: sarama.NewConfig(), - addrs: []string{"b1", "b2"}, + config: sarama.NewConfig(), + addrs: []string{"b1", "b2"}, + offsetsChecker: &mockConsumerGroupOffsetsChecker{}, } - consumerGroup, err := factory.StartConsumerGroup("bla", []string{}, zap.NewNop().Sugar(), nil) + consumerGroup, err := factory.StartConsumerGroup(ctx, "bla", []string{}, nil) if err != nil { t.Errorf("Should not throw error %v", err) } @@ -137,14 +179,15 @@ func assertContainsError(t *testing.T, collection []error, errorStr string) { } func TestErrorWhileCreatingNewConsumerGroup(t *testing.T) { - + ctx := context.TODO() newConsumerGroup = mockedNewConsumerGroupFromClient(nil, true, true, false, true) factory := kafkaConsumerGroupFactoryImpl{ - config: sarama.NewConfig(), - addrs: []string{"b1", "b2"}, + config: sarama.NewConfig(), + addrs: []string{"b1", "b2"}, + offsetsChecker: &mockConsumerGroupOffsetsChecker{}, } - _, err := factory.StartConsumerGroup("bla", []string{}, zap.L().Sugar(), nil) + _, err := factory.StartConsumerGroup(ctx, "bla", []string{}, nil) if err == nil || err.Error() != "failed" { t.Errorf("Should contain an error with message failed. Got %v", err) @@ -152,14 +195,15 @@ func TestErrorWhileCreatingNewConsumerGroup(t *testing.T) { } func TestErrorWhileNewConsumerGroup(t *testing.T) { - + ctx := context.TODO() newConsumerGroup = mockedNewConsumerGroupFromClient(nil, false, false, true, false) factory := kafkaConsumerGroupFactoryImpl{ - config: sarama.NewConfig(), - addrs: []string{"b1", "b2"}, + config: sarama.NewConfig(), + addrs: []string{"b1", "b2"}, + offsetsChecker: &mockConsumerGroupOffsetsChecker{}, } - consumerGroup, _ := factory.StartConsumerGroup("bla", []string{}, zap.L().Sugar(), nil) + consumerGroup, _ := factory.StartConsumerGroup(ctx, "bla", []string{}, nil) consumerGroup.(*customConsumerGroup).cancel() // Stop the consume loop from spinning after the error is generated err := <-consumerGroup.Errors() diff --git a/pkg/common/consumer/consumer_manager.go b/pkg/common/consumer/consumer_manager.go index ff6b24bfd4..02a2bcc2b7 100644 --- a/pkg/common/consumer/consumer_manager.go +++ b/pkg/common/consumer/consumer_manager.go @@ -43,6 +43,7 @@ import ( "go.uber.org/zap" "k8s.io/apimachinery/pkg/types" ctrlservice "knative.dev/control-protocol/pkg/service" + "knative.dev/pkg/logging" "knative.dev/eventing-kafka/pkg/common/controlprotocol" "knative.dev/eventing-kafka/pkg/common/controlprotocol/commands" @@ -93,7 +94,7 @@ type ManagerEvent struct { // KafkaConsumerGroupManager keeps track of Sarama consumer groups and handles messages from control-protocol clients type KafkaConsumerGroupManager interface { Reconfigure(brokers []string, config *sarama.Config) *ReconfigureError - StartConsumerGroup(groupId string, topics []string, logger *zap.SugaredLogger, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) error + StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) error CloseConsumerGroup(groupId string) error Errors(groupId string) <-chan error IsManaged(groupId string) bool @@ -115,6 +116,7 @@ type kafkaConsumerGroupManagerImpl struct { groupLock sync.RWMutex // Synchronizes write access to the groupMap notifyChannels []chan ManagerEvent eventLock sync.Mutex + offsetsChecker ConsumerGroupOffsetsChecker } // Verify that the kafkaConsumerGroupManagerImpl satisfies the KafkaConsumerGroupManager interface @@ -139,15 +141,16 @@ func (r ReconfigureError) Error() string { var _ error = (*ReconfigureError)(nil) // NewConsumerGroupManager returns a new kafkaConsumerGroupManagerImpl as a KafkaConsumerGroupManager interface -func NewConsumerGroupManager(logger *zap.Logger, serverHandler controlprotocol.ServerHandler, brokers []string, config *sarama.Config) KafkaConsumerGroupManager { +func NewConsumerGroupManager(logger *zap.Logger, serverHandler controlprotocol.ServerHandler, brokers []string, config *sarama.Config, offsetsChecker ConsumerGroupOffsetsChecker) KafkaConsumerGroupManager { manager := &kafkaConsumerGroupManagerImpl{ - logger: logger, - server: serverHandler, - groups: make(groupMap), - factory: &kafkaConsumerGroupFactoryImpl{addrs: brokers, config: config}, - groupLock: sync.RWMutex{}, - eventLock: sync.Mutex{}, + logger: logger, + server: serverHandler, + groups: make(groupMap), + factory: &kafkaConsumerGroupFactoryImpl{addrs: brokers, config: config, offsetsChecker: offsetsChecker}, + groupLock: sync.RWMutex{}, + eventLock: sync.Mutex{}, + offsetsChecker: offsetsChecker, } logger.Info("Registering Consumer Group Manager Control-Protocol Handlers") @@ -236,7 +239,7 @@ func (m *kafkaConsumerGroupManagerImpl) Reconfigure(brokers []string, config *sa } } - m.factory = &kafkaConsumerGroupFactoryImpl{addrs: brokers, config: config} + m.factory = &kafkaConsumerGroupFactoryImpl{addrs: brokers, config: config, offsetsChecker: m.offsetsChecker} // Restart any groups this function stopped m.logger.Info("Reconfigure Consumer Group Manager - Starting All Managed Consumer Groups") @@ -261,7 +264,9 @@ func (m *kafkaConsumerGroupManagerImpl) Reconfigure(brokers []string, config *sa // StartConsumerGroup uses the consumer factory to create a new ConsumerGroup, add it to the list // of managed groups (for start/stop functionality) and start the Consume loop. -func (m *kafkaConsumerGroupManagerImpl) StartConsumerGroup(groupId string, topics []string, logger *zap.SugaredLogger, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) error { +func (m *kafkaConsumerGroupManagerImpl) StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) error { + logger := logging.FromContext(ctx) + groupLogger := m.logger.With(zap.String("GroupId", groupId)) groupLogger.Info("Creating New Managed ConsumerGroup") group, err := m.factory.createConsumerGroup(groupId) @@ -281,7 +286,7 @@ func (m *kafkaConsumerGroupManagerImpl) StartConsumerGroup(groupId string, topic } // The only thing we really want from the factory is the cancel function for the customConsumerGroup - customGroup := m.factory.startExistingConsumerGroup(group, consume, topics, logger, handler, options...) + customGroup := m.factory.startExistingConsumerGroup(groupId, group, consume, topics, logger, handler, options...) managedGrp := createManagedGroup(ctx, m.logger, group, cancel, customGroup.cancel) // Add the Sarama ConsumerGroup we obtained from the factory to the managed group map, diff --git a/pkg/common/consumer/consumer_manager_test.go b/pkg/common/consumer/consumer_manager_test.go index 35ebc85d86..7b69332bb1 100644 --- a/pkg/common/consumer/consumer_manager_test.go +++ b/pkg/common/consumer/consumer_manager_test.go @@ -38,7 +38,7 @@ import ( func TestNewConsumerGroupManager(t *testing.T) { server := getMockServerHandler() - manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), server, []string{}, &sarama.Config{}) + manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), server, []string{}, &sarama.Config{}, &NoopConsumerGroupOffsetsChecker{}) assert.NotNil(t, manager) assert.NotNil(t, server.Router[commands.StopConsumerGroupOpCode]) assert.NotNil(t, server.Router[commands.StartConsumerGroupOpCode]) @@ -155,7 +155,8 @@ func TestStartConsumerGroup(t *testing.T) { }, } { t.Run(testCase.name, func(t *testing.T) { - manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), getMockServerHandler(), []string{}, &sarama.Config{}) + ctx := context.TODO() + manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), getMockServerHandler(), []string{}, &sarama.Config{}, &NoopConsumerGroupOffsetsChecker{}) mockGroup := kafkatesting.NewMockConsumerGroup() newConsumerGroup = func(addrs []string, groupID string, config *sarama.Config) (sarama.ConsumerGroup, error) { if testCase.factoryErr { @@ -164,7 +165,7 @@ func TestStartConsumerGroup(t *testing.T) { mockGroup.On("Errors").Return(mockGroup.ErrorChan) return mockGroup, nil } - err := manager.StartConsumerGroup("testid", []string{}, nil, nil) + err := manager.StartConsumerGroup(ctx, "testid", []string{}, nil, nil) assert.Equal(t, testCase.factoryErr, err != nil) time.Sleep(shortTimeout) // Give the transferErrors routine a chance to call Errors() mockGroup.AssertExpectations(t) @@ -620,7 +621,7 @@ func getManagerWithMockGroup(t *testing.T, groupId string, factoryErr bool) (Kaf mockGroup.On("Errors").Return(make(chan error)) return mockGroup, nil } - manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), serverHandler, []string{}, &sarama.Config{}) + manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), serverHandler, []string{}, &sarama.Config{}, &NoopConsumerGroupOffsetsChecker{}) if groupId != "" { mockGroup, managedGrp := createMockAndManagedGroups(t) manager.(*kafkaConsumerGroupManagerImpl).groups[groupId] = managedGrp diff --git a/pkg/common/consumer/consumergroup_offsets_checker.go b/pkg/common/consumer/consumergroup_offsets_checker.go new file mode 100644 index 0000000000..e94281d350 --- /dev/null +++ b/pkg/common/consumer/consumergroup_offsets_checker.go @@ -0,0 +1,74 @@ +package consumer + +import ( + "context" + "fmt" + "time" + + "github.com/Shopify/sarama" + "go.uber.org/zap" + "k8s.io/apimachinery/pkg/util/wait" + "knative.dev/eventing-kafka/pkg/common/kafka/offset" +) + +const ( + OffsetCheckRetryTimeout = 60 * time.Second + OffsetCheckRetryInterval = 5 * time.Second +) + +// wrapper functions for the Sarama functions, to facilitate unit testing +var newSaramaClient = sarama.NewClient +var newSaramaClusterAdmin = sarama.NewClusterAdmin + +type ConsumerGroupOffsetsChecker interface { + WaitForOffsetsInitialization(ctx context.Context, groupID string, topics []string, logger *zap.SugaredLogger, addrs []string, config *sarama.Config) error +} + +type NoopConsumerGroupOffsetsChecker struct { +} + +func (c *NoopConsumerGroupOffsetsChecker) WaitForOffsetsInitialization(ctx context.Context, groupID string, topics []string, logger *zap.SugaredLogger, addrs []string, config *sarama.Config) error { + return nil +} + +type KafkaConsumerGroupOffsetsChecker struct { +} + +func (k *KafkaConsumerGroupOffsetsChecker) WaitForOffsetsInitialization(ctx context.Context, groupID string, topics []string, logger *zap.SugaredLogger, addrs []string, config *sarama.Config) error { + logger.Debugw("checking if all offsets are initialized", zap.Any("topics", topics), zap.Any("groupID", groupID)) + + client, err := newSaramaClient(addrs, config) + if err != nil { + logger.Errorw("unable to create Kafka client", zap.Any("topics", topics), zap.String("groupId", groupID), zap.Error(err)) + return err + } + defer client.Close() + + clusterAdmin, err := newSaramaClusterAdmin(addrs, config) + if err != nil { + logger.Errorw("unable to create Kafka cluster admin client", zap.Any("topics", topics), zap.String("groupId", groupID), zap.Error(err)) + return err + } + defer clusterAdmin.Close() + + check := func() (bool, error) { + if initialized, err := offset.CheckIfAllOffsetsInitialized(client, clusterAdmin, topics, groupID); err == nil { + if initialized { + return true, nil + } else { + logger.Debugw("offsets not yet initialized, going to try again") + return false, nil + } + } else { + return false, fmt.Errorf("error checking if offsets are initialized. stopping trying. %w", err) + } + } + pollCtx, pollCtxCancel := context.WithTimeout(ctx, OffsetCheckRetryTimeout) + err = wait.PollUntil(OffsetCheckRetryInterval, check, pollCtx.Done()) + defer pollCtxCancel() + + if err != nil { + return fmt.Errorf("failed to check if offsets are initialized %w", err) + } + return nil +} diff --git a/pkg/common/consumer/testing/mocks.go b/pkg/common/consumer/testing/mocks.go index 9a7f40bacd..428d66f134 100644 --- a/pkg/common/consumer/testing/mocks.go +++ b/pkg/common/consumer/testing/mocks.go @@ -17,10 +17,10 @@ limitations under the License. package testing import ( + "context" + "github.com/Shopify/sarama" "github.com/stretchr/testify/mock" - "go.uber.org/zap" - "knative.dev/eventing-kafka/pkg/common/consumer" ) @@ -33,8 +33,8 @@ type MockKafkaConsumerGroupFactory struct { mock.Mock } -func (c *MockKafkaConsumerGroupFactory) StartConsumerGroup(groupId string, topics []string, logger *zap.SugaredLogger, handler consumer.KafkaConsumerHandler, options ...consumer.SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { - args := c.Called(groupId, topics, logger, handler, options) +func (c *MockKafkaConsumerGroupFactory) StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler consumer.KafkaConsumerHandler, options ...consumer.SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { + args := c.Called(ctx, groupId, topics, handler, options) return args.Get(0).(sarama.ConsumerGroup), args.Error(1) } @@ -60,9 +60,9 @@ func (m *MockConsumerGroupManager) Reconfigure(brokers []string, config *sarama. return m.Called(brokers, config).Get(0).(*consumer.ReconfigureError) } -func (m *MockConsumerGroupManager) StartConsumerGroup(groupId string, topics []string, logger *zap.SugaredLogger, +func (m *MockConsumerGroupManager) StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler consumer.KafkaConsumerHandler, options ...consumer.SaramaConsumerHandlerOption) error { - return m.Called(groupId, topics, logger, handler, options).Error(0) + return m.Called(ctx, groupId, topics, handler, options).Error(0) } func (m *MockConsumerGroupManager) CloseConsumerGroup(groupId string) error { diff --git a/pkg/common/kafka/offset/offsets.go b/pkg/common/kafka/offset/offsets.go index 345ce2d9bd..5caab1698c 100644 --- a/pkg/common/kafka/offset/offsets.go +++ b/pkg/common/kafka/offset/offsets.go @@ -33,22 +33,14 @@ import ( // Without InitOffsets, an event sent to a partition with an uninitialized offset // will not be forwarded when the session is closed (or a rebalancing is in progress). func InitOffsets(ctx context.Context, kafkaClient sarama.Client, kafkaAdminClient sarama.ClusterAdmin, topics []string, consumerGroup string) (int32, error) { - totalPartitions := 0 offsetManager, err := sarama.NewOffsetManagerFromClient(consumerGroup, kafkaClient) if err != nil { return -1, err } - // Retrieve all partitions - topicPartitions := make(map[string][]int32) - for _, topic := range topics { - partitions, err := kafkaClient.Partitions(topic) - totalPartitions += len(partitions) - if err != nil { - return -1, fmt.Errorf("failed to get partitions for topic %s: %w", topic, err) - } - - topicPartitions[topic] = partitions + totalPartitions, topicPartitions, err := retrieveAllPartitions(topics, kafkaClient) + if err != nil { + return -1, err } // Fetch topic offsets @@ -101,3 +93,47 @@ func InitOffsets(ctx context.Context, kafkaClient sarama.Client, kafkaAdminClien return int32(totalPartitions), nil } + +func CheckIfAllOffsetsInitialized(kafkaClient sarama.Client, kafkaAdminClient sarama.ClusterAdmin, topics []string, consumerGroup string) (bool, error) { + _, topicPartitions, err := retrieveAllPartitions(topics, kafkaClient) + if err != nil { + return false, err + } + + // Look for uninitialized offset (-1) + offsets, err := kafkaAdminClient.ListConsumerGroupOffsets(consumerGroup, topicPartitions) + if err != nil { + return false, err + } + + for _, partitions := range offsets.Blocks { + for _, block := range partitions { + if block.Offset == -1 { // not initialized? + return false, nil + } + } + } + + return true, nil +} + +func retrieveAllPartitions(topics []string, kafkaClient sarama.Client) (int, map[string][]int32, error) { + totalPartitions := 0 + + // Retrieve all partitions + topicPartitions := make(map[string][]int32) + for _, topic := range topics { + partitions, err := kafkaClient.Partitions(topic) + totalPartitions += len(partitions) + if err != nil { + return -1, nil, fmt.Errorf("failed to get partitions for topic %s: %w", topic, err) + } + + // return a copy of the partitions array in the map + // Sarama is caching this array and we don't want nobody to mess with it + clone := make([]int32, len(partitions)) + copy(clone, partitions) + topicPartitions[topic] = clone + } + return totalPartitions, topicPartitions, nil +} diff --git a/pkg/common/kafka/offset/offsets_test.go b/pkg/common/kafka/offset/offsets_test.go index e5effa5042..41285318f0 100644 --- a/pkg/common/kafka/offset/offsets_test.go +++ b/pkg/common/kafka/offset/offsets_test.go @@ -25,57 +25,57 @@ import ( logtesting "knative.dev/pkg/logging/testing" ) -func TestInitOffsets(t *testing.T) { - testCases := map[string]struct { - topics []string - topicOffsets map[string]map[int32]int64 - cgOffsets map[string]map[int32]int64 - wantCommit bool - }{ - "one topic, one partition, initialized": { - topics: []string{"my-topic"}, - topicOffsets: map[string]map[int32]int64{ - "my-topic": { - 0: 5, - }, - }, - cgOffsets: map[string]map[int32]int64{ - "my-topic": { - 0: 2, - }, +var testCases = map[string]struct { + topics []string + topicOffsets map[string]map[int32]int64 + cgOffsets map[string]map[int32]int64 + initialized bool +}{ + "one topic, one partition, initialized": { + topics: []string{"my-topic"}, + topicOffsets: map[string]map[int32]int64{ + "my-topic": { + 0: 5, }, - wantCommit: false, }, - "one topic, one partition, uninitialized": { - topics: []string{"my-topic"}, - topicOffsets: map[string]map[int32]int64{ - "my-topic": { - 0: 5, - }, + cgOffsets: map[string]map[int32]int64{ + "my-topic": { + 0: 2, }, - cgOffsets: map[string]map[int32]int64{ - "my-topic": { - 0: -1, - }, - }, - wantCommit: true, }, - "several topics, several partitions, not all initialized": { - topics: []string{"my-topic", "my-topic-2", "my-topic-3"}, - topicOffsets: map[string]map[int32]int64{ - "my-topic": {0: 5, 1: 7}, - "my-topic-2": {0: 5, 1: 7, 2: 9}, - "my-topic-3": {0: 5, 1: 7, 2: 2, 3: 10}, + initialized: true, + }, + "one topic, one partition, uninitialized": { + topics: []string{"my-topic"}, + topicOffsets: map[string]map[int32]int64{ + "my-topic": { + 0: 5, }, - cgOffsets: map[string]map[int32]int64{ - "my-topic": {0: -1, 1: 7}, - "my-topic-2": {0: 5, 1: -1, 2: -1}, - "my-topic-3": {0: 5, 1: 7, 2: -1, 3: 10}, + }, + cgOffsets: map[string]map[int32]int64{ + "my-topic": { + 0: -1, }, - wantCommit: true, }, - } + initialized: false, + }, + "several topics, several partitions, not all initialized": { + topics: []string{"my-topic", "my-topic-2", "my-topic-3"}, + topicOffsets: map[string]map[int32]int64{ + "my-topic": {0: 5, 1: 7}, + "my-topic-2": {0: 5, 1: 7, 2: 9}, + "my-topic-3": {0: 5, 1: 7, 2: 2, 3: 10}, + }, + cgOffsets: map[string]map[int32]int64{ + "my-topic": {0: -1, 1: 7}, + "my-topic-2": {0: 5, 1: -1, 2: -1}, + "my-topic-3": {0: 5, 1: 7, 2: -1, 3: 10}, + }, + initialized: false, + }, +} +func TestInitOffsets(t *testing.T) { for n, tc := range testCases { t.Run(n, func(t *testing.T) { broker := sarama.NewMockBroker(t, 1) @@ -83,52 +83,47 @@ func TestInitOffsets(t *testing.T) { group := "my-group" - offsetResponse := sarama.NewMockOffsetResponse(t).SetVersion(1) - for topic, partitions := range tc.topicOffsets { - for partition, offset := range partitions { - offsetResponse = offsetResponse.SetOffset(topic, partition, -1, offset) - } - } - - offsetFetchResponse := sarama.NewMockOffsetFetchResponse(t).SetError(sarama.ErrNoError) - for topic, partitions := range tc.cgOffsets { - for partition, offset := range partitions { - offsetFetchResponse = offsetFetchResponse.SetOffset(group, topic, partition, offset, "", sarama.ErrNoError) - } - } + configureMockBroker(t, group, tc.topicOffsets, tc.cgOffsets, tc.initialized, broker) - offsetCommitResponse := sarama.NewMockOffsetCommitResponse(t) - serr := sarama.ErrNoError - if !tc.wantCommit { - serr = sarama.ErrUnknown // could be anything + config := sarama.NewConfig() + config.Version = sarama.MaxVersion + sc, err := sarama.NewClient([]string{broker.Addr()}, config) + if err != nil { + t.Errorf("unexpected error: %v", err) } + defer sc.Close() - for topic, partitions := range tc.cgOffsets { - for partition := range partitions { - offsetCommitResponse = offsetCommitResponse.SetError(group, topic, partition, serr) - } + kac, err := sarama.NewClusterAdminFromClient(sc) + if err != nil { + t.Errorf("unexpected error: %v", err) } + defer kac.Close() - metadataResponse := sarama.NewMockMetadataResponse(t). - SetController(broker.BrokerID()). - SetBroker(broker.Addr(), broker.BrokerID()) - for topic, partitions := range tc.topicOffsets { - for partition := range partitions { - metadataResponse = metadataResponse.SetLeader(topic, partition, broker.BrokerID()) - } + // test InitOffsets + ctx := logtesting.TestContextWithLogger(t) + partitionCt, err := InitOffsets(ctx, sc, kac, tc.topics, group) + total := 0 + for _, partitions := range tc.topicOffsets { + total += len(partitions) + } + assert.Equal(t, int(partitionCt), total) + if err != nil { + t.Errorf("unexpected error: %v", err) } + }) + } +} - broker.SetHandlerByMap(map[string]sarama.MockResponse{ - "OffsetRequest": offsetResponse, - "OffsetFetchRequest": offsetFetchResponse, - "OffsetCommitRequest": offsetCommitResponse, +func TestCheckIfAllOffsetsInitialized(t *testing.T) { + for n, tc := range testCases { + t.Run(n, func(t *testing.T) { + broker := sarama.NewMockBroker(t, 1) + defer broker.Close() - "FindCoordinatorRequest": sarama.NewMockFindCoordinatorResponse(t). - SetCoordinator(sarama.CoordinatorGroup, group, broker), + group := "my-group" - "MetadataRequest": metadataResponse, - }) + configureMockBroker(t, group, tc.topicOffsets, tc.cgOffsets, tc.initialized, broker) config := sarama.NewConfig() config.Version = sarama.MaxVersion @@ -145,18 +140,61 @@ func TestInitOffsets(t *testing.T) { } defer kac.Close() - ctx := logtesting.TestContextWithLogger(t) - partitionCt, err := InitOffsets(ctx, sc, kac, tc.topics, group) - total := 0 - for _, partitions := range tc.topicOffsets { - total += len(partitions) - } - assert.Equal(t, int(partitionCt), total) + // test CheckIfAllOffsetsInitialized + retrieved, err := CheckIfAllOffsetsInitialized(sc, kac, tc.topics, group) + assert.Equal(t, retrieved, tc.initialized) if err != nil { t.Errorf("unexpected error: %v", err) } - }) } +} + +func configureMockBroker(t *testing.T, group string, topicOffsets map[string]map[int32]int64, cgOffsets map[string]map[int32]int64, initialized bool, broker *sarama.MockBroker) { + offsetResponse := sarama.NewMockOffsetResponse(t).SetVersion(1) + for topic, partitions := range topicOffsets { + for partition, offset := range partitions { + offsetResponse = offsetResponse.SetOffset(topic, partition, -1, offset) + } + } + + offsetFetchResponse := sarama.NewMockOffsetFetchResponse(t).SetError(sarama.ErrNoError) + for topic, partitions := range cgOffsets { + for partition, offset := range partitions { + offsetFetchResponse = offsetFetchResponse.SetOffset(group, topic, partition, offset, "", sarama.ErrNoError) + } + } + + offsetCommitResponse := sarama.NewMockOffsetCommitResponse(t) + serr := sarama.ErrNoError + if initialized { // means, we want a commit + serr = sarama.ErrUnknown // could be anything + + } + + for topic, partitions := range cgOffsets { + for partition := range partitions { + offsetCommitResponse = offsetCommitResponse.SetError(group, topic, partition, serr) + } + } + + metadataResponse := sarama.NewMockMetadataResponse(t). + SetController(broker.BrokerID()). + SetBroker(broker.Addr(), broker.BrokerID()) + for topic, partitions := range topicOffsets { + for partition := range partitions { + metadataResponse = metadataResponse.SetLeader(topic, partition, broker.BrokerID()) + } + } + + broker.SetHandlerByMap(map[string]sarama.MockResponse{ + "OffsetRequest": offsetResponse, + "OffsetFetchRequest": offsetFetchResponse, + "OffsetCommitRequest": offsetCommitResponse, + + "FindCoordinatorRequest": sarama.NewMockFindCoordinatorResponse(t). + SetCoordinator(sarama.CoordinatorGroup, group, broker), + "MetadataRequest": metadataResponse, + }) } diff --git a/pkg/common/testing/mocks.go b/pkg/common/testing/mocks.go new file mode 100644 index 0000000000..933a22a9a7 --- /dev/null +++ b/pkg/common/testing/mocks.go @@ -0,0 +1,113 @@ +package testing + +import "github.com/Shopify/sarama" + +var _ sarama.ClusterAdmin = (*MockClusterAdmin)(nil) + +type MockClusterAdmin struct { + MockCreateTopicFunc func(topic string, detail *sarama.TopicDetail, validateOnly bool) error + MockDeleteTopicFunc func(topic string) error + MockListConsumerGroupsFunc func() (map[string]string, error) +} + +func (ca *MockClusterAdmin) AlterPartitionReassignments(topic string, assignment [][]int32) error { + return nil +} + +func (ca *MockClusterAdmin) ListPartitionReassignments(topics string, partitions []int32) (topicStatus map[string]map[int32]*sarama.PartitionReplicaReassignmentsStatus, err error) { + return nil, nil +} + +func (ca *MockClusterAdmin) DescribeLogDirs(brokers []int32) (map[int32][]sarama.DescribeLogDirsResponseDirMetadata, error) { + return nil, nil +} + +func (ca *MockClusterAdmin) DescribeUserScramCredentials(users []string) ([]*sarama.DescribeUserScramCredentialsResult, error) { + return nil, nil +} + +func (ca *MockClusterAdmin) DeleteUserScramCredentials(delete []sarama.AlterUserScramCredentialsDelete) ([]*sarama.AlterUserScramCredentialsResult, error) { + return nil, nil +} + +func (ca *MockClusterAdmin) UpsertUserScramCredentials(upsert []sarama.AlterUserScramCredentialsUpsert) ([]*sarama.AlterUserScramCredentialsResult, error) { + return nil, nil +} + +func (ca *MockClusterAdmin) CreateTopic(topic string, detail *sarama.TopicDetail, validateOnly bool) error { + if ca.MockCreateTopicFunc != nil { + return ca.MockCreateTopicFunc(topic, detail, validateOnly) + } + return nil +} + +func (ca *MockClusterAdmin) Close() error { + return nil +} + +func (ca *MockClusterAdmin) DeleteTopic(topic string) error { + if ca.MockDeleteTopicFunc != nil { + return ca.MockDeleteTopicFunc(topic) + } + return nil +} + +func (ca *MockClusterAdmin) DescribeTopics(topics []string) (metadata []*sarama.TopicMetadata, err error) { + return nil, nil +} + +func (ca *MockClusterAdmin) ListTopics() (map[string]sarama.TopicDetail, error) { + return nil, nil +} + +func (ca *MockClusterAdmin) CreatePartitions(topic string, count int32, assignment [][]int32, validateOnly bool) error { + return nil +} + +func (ca *MockClusterAdmin) DeleteRecords(topic string, partitionOffsets map[int32]int64) error { + return nil +} + +func (ca *MockClusterAdmin) DescribeConfig(resource sarama.ConfigResource) ([]sarama.ConfigEntry, error) { + return nil, nil +} + +func (ca *MockClusterAdmin) AlterConfig(resourceType sarama.ConfigResourceType, name string, entries map[string]*string, validateOnly bool) error { + return nil +} + +func (ca *MockClusterAdmin) CreateACL(resource sarama.Resource, acl sarama.Acl) error { + return nil +} + +func (ca *MockClusterAdmin) ListAcls(filter sarama.AclFilter) ([]sarama.ResourceAcls, error) { + return nil, nil +} + +func (ca *MockClusterAdmin) DeleteACL(filter sarama.AclFilter, validateOnly bool) ([]sarama.MatchingAcl, error) { + return nil, nil +} + +func (ca *MockClusterAdmin) ListConsumerGroups() (map[string]string, error) { + if ca.MockListConsumerGroupsFunc != nil { + return ca.MockListConsumerGroupsFunc() + } + return nil, nil +} + +func (ca *MockClusterAdmin) DescribeConsumerGroups(groups []string) ([]*sarama.GroupDescription, error) { + return nil, nil +} + +func (ca *MockClusterAdmin) ListConsumerGroupOffsets(group string, topicPartitions map[string][]int32) (*sarama.OffsetFetchResponse, error) { + return &sarama.OffsetFetchResponse{}, nil +} + +func (ca *MockClusterAdmin) DescribeCluster() (brokers []*sarama.Broker, controllerID int32, err error) { + return nil, 0, nil +} + +// Delete a consumer group. +func (ca *MockClusterAdmin) DeleteConsumerGroup(group string) error { + return nil +} diff --git a/pkg/source/adapter/adapter.go b/pkg/source/adapter/adapter.go index 071329b896..d2f5a1c929 100644 --- a/pkg/source/adapter/adapter.go +++ b/pkg/source/adapter/adapter.go @@ -140,11 +140,11 @@ func (a *Adapter) Start(ctx context.Context) (err error) { a.saramaConfig = config options := []consumer.SaramaConsumerHandlerOption{consumer.WithSaramaConsumerLifecycleListener(a)} - consumerGroupFactory := consumer.NewConsumerGroupFactory(addrs, config) + consumerGroupFactory := consumer.NewConsumerGroupFactory(addrs, config, &consumer.NoopConsumerGroupOffsetsChecker{}) group, err := consumerGroupFactory.StartConsumerGroup( + ctx, a.config.ConsumerGroup, a.config.Topics, - a.logger, a, options..., ) diff --git a/third_party/VENDOR-LICENSE/knative.dev/networking/pkg/prober/LICENSE b/third_party/VENDOR-LICENSE/knative.dev/networking/pkg/prober/LICENSE deleted file mode 100644 index 261eeb9e9f..0000000000 --- a/third_party/VENDOR-LICENSE/knative.dev/networking/pkg/prober/LICENSE +++ /dev/null @@ -1,201 +0,0 @@ - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. diff --git a/vendor/knative.dev/networking/LICENSE b/vendor/knative.dev/networking/LICENSE deleted file mode 100644 index 261eeb9e9f..0000000000 --- a/vendor/knative.dev/networking/LICENSE +++ /dev/null @@ -1,201 +0,0 @@ - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. diff --git a/vendor/knative.dev/networking/pkg/prober/doc.go b/vendor/knative.dev/networking/pkg/prober/doc.go deleted file mode 100644 index 1c971e14c6..0000000000 --- a/vendor/knative.dev/networking/pkg/prober/doc.go +++ /dev/null @@ -1,18 +0,0 @@ -/* -Copyright 2019 The Knative Authors - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -// Package prober contains functionality for implementing probing in knative serving. -package prober diff --git a/vendor/knative.dev/networking/pkg/prober/prober.go b/vendor/knative.dev/networking/pkg/prober/prober.go deleted file mode 100644 index 6b216609aa..0000000000 --- a/vendor/knative.dev/networking/pkg/prober/prober.go +++ /dev/null @@ -1,199 +0,0 @@ -/* -Copyright 2020 The Knative Authors - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package prober - -import ( - "context" - "fmt" - "io/ioutil" - "net/http" - "sync" - "time" - - "go.uber.org/zap" - "k8s.io/apimachinery/pkg/util/sets" - "k8s.io/apimachinery/pkg/util/wait" - "knative.dev/pkg/logging" -) - -// Preparer is a way for the caller to modify the HTTP request before it goes out. -type Preparer func(r *http.Request) *http.Request - -// Verifier is a way for the caller to validate the HTTP response after it comes back. -type Verifier func(r *http.Response, b []byte) (bool, error) - -// WithHeader sets a header in the probe request. -func WithHeader(name, value string) Preparer { - return func(r *http.Request) *http.Request { - r.Header.Set(name, value) - return r - } -} - -// WithHost sets the host in the probe request. -func WithHost(host string) Preparer { - return func(r *http.Request) *http.Request { - r.Host = host - return r - } -} - -// WithPath sets the path in the probe request. -func WithPath(path string) Preparer { - return func(r *http.Request) *http.Request { - r.URL.Path = path - return r - } -} - -// ExpectsBody validates that the body of the probe response matches the provided string. -func ExpectsBody(body string) Verifier { - return func(r *http.Response, b []byte) (bool, error) { - if string(b) == body { - return true, nil - } - return false, fmt.Errorf("unexpected body: want %q, got %q", body, string(b)) - } -} - -// ExpectsHeader validates that the given header of the probe response matches the provided string. -func ExpectsHeader(name, value string) Verifier { - return func(r *http.Response, _ []byte) (bool, error) { - if r.Header.Get(name) == value { - return true, nil - } - return false, fmt.Errorf("unexpected header %q: want %q, got %q", name, value, r.Header.Get(name)) - } -} - -// ExpectsStatusCodes validates that the given status code of the probe response matches the provided int. -func ExpectsStatusCodes(statusCodes []int) Verifier { - return func(r *http.Response, _ []byte) (bool, error) { - for _, v := range statusCodes { - if r.StatusCode == v { - return true, nil - } - } - return false, fmt.Errorf("unexpected status code: want %v, got %v", statusCodes, r.StatusCode) - } -} - -// Do sends a single probe to given target, e.g. `http://revision.default.svc.cluster.local:81`. -// Do returns whether the probe was successful or not, or there was an error probing. -func Do(ctx context.Context, transport http.RoundTripper, target string, ops ...interface{}) (bool, error) { - req, err := http.NewRequestWithContext(ctx, http.MethodGet, target, nil) - if err != nil { - return false, fmt.Errorf("%s is not a valid URL: %w", target, err) - } - for _, op := range ops { - if po, ok := op.(Preparer); ok { - req = po(req) - } - } - - resp, err := transport.RoundTrip(req) - if err != nil { - return false, fmt.Errorf("error roundtripping %s: %w", target, err) - } - defer resp.Body.Close() - body, err := ioutil.ReadAll(resp.Body) - if err != nil { - return false, fmt.Errorf("error reading body: %w", err) - } - - for _, op := range ops { - if vo, ok := op.(Verifier); ok { - if ok, err := vo(resp, body); err != nil || !ok { - return false, err - } - } - } - return true, nil -} - -// Done is a callback that is executed when the async probe has finished. -// `arg` is given by the caller at the offering time, while `success` and `err` -// are the return values of the `Do` call. -// It is assumed that the opaque arg is consistent for a given target and -// we will coalesce concurrent Offer invocations on target. -type Done func(arg interface{}, success bool, err error) - -// Manager manages async probes and makes sure we run concurrently only a single -// probe for the same key. -type Manager struct { - cb Done - // NB: it is paramount to use a transport that will close the connection - // after every request here. Otherwise the cached connections will prohibit - // scaling to zero, due to unsuccessful probes to the Activator. - transport http.RoundTripper - - // mu guards keys. - mu sync.Mutex - keys sets.String -} - -// New creates a new Manager, that will invoke the given callback when -// async probing is finished. -func New(cb Done, transport http.RoundTripper) *Manager { - return &Manager{ - keys: sets.NewString(), - cb: cb, - transport: transport, - } -} - -// Offer executes asynchronous probe using `target` as the key. -// If a probe with the same key already exists, Offer will return false and the -// call is discarded. If the request is accepted, Offer returns true. -// Otherwise Offer starts a goroutine that periodically executes -// `Do`, until timeout is reached, the probe succeeds, or fails with an error. -// In the end the callback is invoked with the provided `arg` and probing results. -func (m *Manager) Offer(ctx context.Context, target string, arg interface{}, period, timeout time.Duration, ops ...interface{}) bool { - m.mu.Lock() - defer m.mu.Unlock() - if m.keys.Has(target) { - return false - } - m.keys.Insert(target) - m.doAsync(ctx, target, arg, period, timeout, ops...) - return true -} - -// doAsync starts a go routine that probes the target with given period. -func (m *Manager) doAsync(ctx context.Context, target string, arg interface{}, period, timeout time.Duration, ops ...interface{}) { - logger := logging.FromContext(ctx) - go func() { - defer func() { - m.mu.Lock() - defer m.mu.Unlock() - m.keys.Delete(target) - }() - var ( - result bool - inErr error - ) - err := wait.PollImmediate(period, timeout, func() (bool, error) { - result, inErr = Do(ctx, m.transport, target, ops...) - // Do not return error, which is from verifierError, as retry is expected until timeout. - return result, nil - }) - if inErr != nil { - logger.Errorw("Unable to read sockstat", zap.Error(inErr)) - } - m.cb(arg, result, err) - }() -} From 882aed3aa9085b389973734fc8d107c5c4758bab Mon Sep 17 00:00:00 2001 From: Ali Ok Date: Fri, 1 Oct 2021 14:56:13 +0300 Subject: [PATCH 2/7] ./hack/update-codegen.sh --- go.mod | 2 -- go.sum | 3 --- vendor/modules.txt | 4 ---- 3 files changed, 9 deletions(-) diff --git a/go.mod b/go.mod index 00dc7780fc..3091144e19 100644 --- a/go.mod +++ b/go.mod @@ -24,7 +24,6 @@ require ( github.com/stretchr/testify v1.7.0 github.com/xdg-go/scram v1.0.2 go.opencensus.io v0.23.0 - go.uber.org/atomic v1.9.0 go.uber.org/multierr v1.6.0 go.uber.org/zap v1.19.1 golang.org/x/sync v0.0.0-20210220032951-036812b2e83c @@ -37,7 +36,6 @@ require ( knative.dev/control-protocol v0.0.0-20210929151023-4b73baddb22a knative.dev/eventing v0.26.1-0.20210930102845-5b9ac948cefc knative.dev/hack v0.0.0-20210806075220-815cd312d65c - knative.dev/networking v0.0.0-20210929162523-749575ef53f8 knative.dev/pkg v0.0.0-20210929111822-2267a4cbebb8 knative.dev/reconciler-test v0.0.0-20210930064245-45904ca4383d ) diff --git a/go.sum b/go.sum index 74cf61544a..a5db585833 100644 --- a/go.sum +++ b/go.sum @@ -658,7 +658,6 @@ github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6So github.com/rogpeppe/fastuuid v1.2.0 h1:Ppwyp6VYCF1nvBTXL3trRso7mXMlRrw9ooo375wvi2s= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/rs/dnscache v0.0.0-20210201191234-295bba877686/go.mod h1:qe5TWALJ8/a1Lqznoc5BDHpYX/8HU60Hm2AwRmqzxqA= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= @@ -1306,8 +1305,6 @@ knative.dev/eventing v0.26.1-0.20210930102845-5b9ac948cefc/go.mod h1:HwxfiCneWF5 knative.dev/hack v0.0.0-20210806075220-815cd312d65c h1:nOXoDWAAItwr4o0dp3nHr6skgpVD4IvME/UX84YNl5k= knative.dev/hack v0.0.0-20210806075220-815cd312d65c/go.mod h1:PHt8x8yX5Z9pPquBEfIj0X66f8iWkWfR0S/sarACJrI= knative.dev/hack/schema v0.0.0-20210806075220-815cd312d65c/go.mod h1:ffjwmdcrH5vN3mPhO8RrF2KfNnbHeCE2C60A+2cv3U0= -knative.dev/networking v0.0.0-20210929162523-749575ef53f8 h1:1VblI2O16leI1bFP4HitfuIRE8SIuPwqDAput/SBh/g= -knative.dev/networking v0.0.0-20210929162523-749575ef53f8/go.mod h1:IQMjYdQwtOsqJ+fQDs8MZGB/cPxXdMc4bpP7fdtXp2Q= knative.dev/pkg v0.0.0-20210914164111-4857ab6939e3/go.mod h1:jMSqkNMsrzuy+XR4Yr/BMy7SDVbUOl3KKB6+5MR+ZU8= knative.dev/pkg v0.0.0-20210929111822-2267a4cbebb8 h1:Ut2Z22rH5WfybI5hjEfi9fDU7rK6kIduWpSAgoEXh7M= knative.dev/pkg v0.0.0-20210929111822-2267a4cbebb8/go.mod h1:r27D20afKNeK+9aNOg+0qMv8JgQcyeP+CAYQIR1jEQY= diff --git a/vendor/modules.txt b/vendor/modules.txt index e49ba15eaa..72686977a0 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -423,7 +423,6 @@ go.opentelemetry.io/otel/label go.opentelemetry.io/otel/propagation go.opentelemetry.io/otel/trace # go.uber.org/atomic v1.9.0 -## explicit go.uber.org/atomic # go.uber.org/automaxprocs v1.4.0 go.uber.org/automaxprocs/internal/cgroups @@ -1245,9 +1244,6 @@ knative.dev/eventing/test/upgrade/prober/wathola/sender ## explicit knative.dev/hack knative.dev/hack/shell -# knative.dev/networking v0.0.0-20210929162523-749575ef53f8 -## explicit -knative.dev/networking/pkg/prober # knative.dev/pkg v0.0.0-20210929111822-2267a4cbebb8 ## explicit knative.dev/pkg/apiextensions/storageversion From fa7e639fc770d697b43e948b44c9aea56984d951 Mon Sep 17 00:00:00 2001 From: Pierangelo Di Pilato Date: Wed, 6 Oct 2021 14:40:27 +0200 Subject: [PATCH 3/7] Change poll loop Signed-off-by: Pierangelo Di Pilato --- pkg/common/consumer/consumer_factory_test.go | 15 ++++++++++++++- pkg/common/consumer/consumer_handler.go | 3 ++- .../consumer/consumergroup_offsets_checker.go | 10 ++++------ 3 files changed, 20 insertions(+), 8 deletions(-) diff --git a/pkg/common/consumer/consumer_factory_test.go b/pkg/common/consumer/consumer_factory_test.go index 2db27b3a9d..1cc309de5a 100644 --- a/pkg/common/consumer/consumer_factory_test.go +++ b/pkg/common/consumer/consumer_factory_test.go @@ -24,6 +24,7 @@ import ( "github.com/Shopify/sarama" "go.uber.org/zap" + controllertesting "knative.dev/eventing-kafka/pkg/common/commands/resetoffset/controller/testing" commontesting "knative.dev/eventing-kafka/pkg/common/testing" ) @@ -111,6 +112,18 @@ func mockedNewSaramaClusterAdmin(clusterAdmin sarama.ClusterAdmin, mustFail bool } } +func mockedNewSaramaClusterAdminFromClient(clusterAdmin sarama.ClusterAdmin, mustFail bool) func(client sarama.Client) (sarama.ClusterAdmin, error) { + if !mustFail { + return func(client sarama.Client) (sarama.ClusterAdmin, error) { + return clusterAdmin, nil + } + } else { + return func(client sarama.Client) (sarama.ClusterAdmin, error) { + return nil, errors.New("failed") + } + } +} + //------ Tests type mockConsumerGroupOffsetsChecker struct { @@ -130,7 +143,7 @@ func TestErrorPropagationCustomConsumerGroup(t *testing.T) { // override some functions newConsumerGroup = mockedNewConsumerGroupFromClient(nil, true, true, false, false) newSaramaClient = mockedNewSaramaClient(client, false) - newSaramaClusterAdmin = mockedNewSaramaClusterAdmin(clusterAdmin, false) + newClusterAdminFromClient = mockedNewSaramaClusterAdminFromClient(clusterAdmin, false) factory := kafkaConsumerGroupFactoryImpl{ config: sarama.NewConfig(), diff --git a/pkg/common/consumer/consumer_handler.go b/pkg/common/consumer/consumer_handler.go index 8face040a8..9fd22f1897 100644 --- a/pkg/common/consumer/consumer_handler.go +++ b/pkg/common/consumer/consumer_handler.go @@ -24,6 +24,7 @@ import ( "github.com/Shopify/sarama" "go.uber.org/zap" + kafkasarama "knative.dev/eventing-kafka/pkg/common/kafka/sarama" ) @@ -99,7 +100,7 @@ func NewConsumerHandler(logger *zap.SugaredLogger, handler KafkaConsumerHandler, // Setup is run at the beginning of a new session, before ConsumeClaim func (consumer *SaramaConsumerHandler) Setup(session sarama.ConsumerGroupSession) error { - consumer.logger.Info("setting up handler") + consumer.logger.Info("setting up handler", zap.Any("claims", session.Claims())) consumer.lifecycleListener.Setup(session) return nil } diff --git a/pkg/common/consumer/consumergroup_offsets_checker.go b/pkg/common/consumer/consumergroup_offsets_checker.go index e94281d350..bb94a6334e 100644 --- a/pkg/common/consumer/consumergroup_offsets_checker.go +++ b/pkg/common/consumer/consumergroup_offsets_checker.go @@ -8,6 +8,7 @@ import ( "github.com/Shopify/sarama" "go.uber.org/zap" "k8s.io/apimachinery/pkg/util/wait" + "knative.dev/eventing-kafka/pkg/common/kafka/offset" ) @@ -18,7 +19,7 @@ const ( // wrapper functions for the Sarama functions, to facilitate unit testing var newSaramaClient = sarama.NewClient -var newSaramaClusterAdmin = sarama.NewClusterAdmin +var newClusterAdminFromClient = sarama.NewClusterAdminFromClient type ConsumerGroupOffsetsChecker interface { WaitForOffsetsInitialization(ctx context.Context, groupID string, topics []string, logger *zap.SugaredLogger, addrs []string, config *sarama.Config) error @@ -44,7 +45,7 @@ func (k *KafkaConsumerGroupOffsetsChecker) WaitForOffsetsInitialization(ctx cont } defer client.Close() - clusterAdmin, err := newSaramaClusterAdmin(addrs, config) + clusterAdmin, err := newClusterAdminFromClient(client) if err != nil { logger.Errorw("unable to create Kafka cluster admin client", zap.Any("topics", topics), zap.String("groupId", groupID), zap.Error(err)) return err @@ -63,10 +64,7 @@ func (k *KafkaConsumerGroupOffsetsChecker) WaitForOffsetsInitialization(ctx cont return false, fmt.Errorf("error checking if offsets are initialized. stopping trying. %w", err) } } - pollCtx, pollCtxCancel := context.WithTimeout(ctx, OffsetCheckRetryTimeout) - err = wait.PollUntil(OffsetCheckRetryInterval, check, pollCtx.Done()) - defer pollCtxCancel() - + err = wait.PollImmediate(OffsetCheckRetryInterval, OffsetCheckRetryTimeout, check) if err != nil { return fmt.Errorf("failed to check if offsets are initialized %w", err) } From e8c739f74a3d91c45b2164a6ed7963a72fe9d838 Mon Sep 17 00:00:00 2001 From: Pierangelo Di Pilato Date: Thu, 7 Oct 2021 10:15:19 +0200 Subject: [PATCH 4/7] Requeue channel on failure Signed-off-by: Pierangelo Di Pilato --- cmd/channel/distributed/dispatcher/main.go | 2 +- .../consolidated/dispatcher/dispatcher.go | 12 ++++---- .../dispatcher/dispatcher_it_test.go | 3 +- .../dispatcher/dispatcher_test.go | 9 +++--- .../reconciler/dispatcher/kafkachannel.go | 16 +++++----- .../dispatcher/controller/kafkachannel.go | 6 +++- .../controller/kafkachannel_test.go | 2 +- .../dispatcher/dispatcher/dispatcher.go | 10 +++---- .../dispatcher/dispatcher/dispatcher_test.go | 4 +-- pkg/common/consumer/consumer_factory.go | 30 ++++++++++--------- pkg/common/consumer/consumer_factory_test.go | 7 +++-- pkg/common/consumer/consumer_manager.go | 10 +++---- pkg/common/consumer/consumer_manager_test.go | 9 +++--- pkg/common/consumer/testing/mocks.go | 6 ++-- pkg/source/adapter/adapter.go | 4 ++- 15 files changed, 73 insertions(+), 57 deletions(-) diff --git a/cmd/channel/distributed/dispatcher/main.go b/cmd/channel/distributed/dispatcher/main.go index a6f9fa9263..eb47981c67 100644 --- a/cmd/channel/distributed/dispatcher/main.go +++ b/cmd/channel/distributed/dispatcher/main.go @@ -153,7 +153,7 @@ func main() { MetricsRegistry: ekConfig.Sarama.Config.MetricRegistry, SaramaConfig: ekConfig.Sarama.Config, } - dispatcher, managerEvents := dispatch.NewDispatcher(dispatcherConfig, controlProtocolServer) + dispatcher, managerEvents := dispatch.NewDispatcher(dispatcherConfig, controlProtocolServer, func(ref types.NamespacedName) {}) // Create KafkaChannel Informer kafkaClient := kafkaclientset.NewForConfigOrDie(k8sConfig) diff --git a/pkg/channel/consolidated/dispatcher/dispatcher.go b/pkg/channel/consolidated/dispatcher/dispatcher.go index e1da740f3f..48335ae21d 100644 --- a/pkg/channel/consolidated/dispatcher/dispatcher.go +++ b/pkg/channel/consolidated/dispatcher/dispatcher.go @@ -18,6 +18,7 @@ package dispatcher import ( "context" "fmt" + nethttp "net/http" "strings" "sync" @@ -29,14 +30,13 @@ import ( "go.uber.org/zap" "k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/util/sets" - "knative.dev/eventing-kafka/pkg/common/config" "knative.dev/pkg/logging" + "knative.dev/eventing-kafka/pkg/common/config" + eventingchannels "knative.dev/eventing/pkg/channel" "knative.dev/pkg/kmeta" - nethttp "net/http" - "knative.dev/eventing-kafka/pkg/channel/consolidated/utils" "knative.dev/eventing-kafka/pkg/channel/distributed/common/env" "knative.dev/eventing-kafka/pkg/common/consumer" @@ -73,7 +73,7 @@ type KafkaDispatcher struct { logger *zap.SugaredLogger } -func NewDispatcher(ctx context.Context, args *KafkaDispatcherArgs) (*KafkaDispatcher, error) { +func NewDispatcher(ctx context.Context, args *KafkaDispatcherArgs, enqueue func(ref types.NamespacedName)) (*KafkaDispatcher, error) { producer, err := sarama.NewSyncProducer(args.Brokers, args.Config.Sarama.Config) if err != nil { @@ -82,7 +82,7 @@ func NewDispatcher(ctx context.Context, args *KafkaDispatcherArgs) (*KafkaDispat dispatcher := &KafkaDispatcher{ dispatcher: eventingchannels.NewMessageDispatcher(logging.FromContext(ctx).Desugar()), - kafkaConsumerFactory: consumer.NewConsumerGroupFactory(args.Brokers, args.Config.Sarama.Config, &consumer.KafkaConsumerGroupOffsetsChecker{}), + kafkaConsumerFactory: consumer.NewConsumerGroupFactory(args.Brokers, args.Config.Sarama.Config, &consumer.KafkaConsumerGroupOffsetsChecker{}, enqueue), channelSubscriptions: make(map[types.NamespacedName]*KafkaSubscription), subsConsumerGroups: make(map[types.UID]sarama.ConsumerGroup), subscriptions: make(map[types.UID]Subscription), @@ -297,7 +297,7 @@ func (d *KafkaDispatcher) subscribe(ctx context.Context, channelRef types.Namesp } d.logger.Debugw("Starting consumer group", zap.Any("channelRef", channelRef), zap.Any("subscription", sub.UID), zap.String("topic", topicName), zap.String("consumer group", groupID)) - consumerGroup, err := d.kafkaConsumerFactory.StartConsumerGroup(ctx, groupID, []string{topicName}, handler) + consumerGroup, err := d.kafkaConsumerFactory.StartConsumerGroup(ctx, groupID, []string{topicName}, handler, channelRef) if err != nil { // we can not create a consumer - logging that, with reason diff --git a/pkg/channel/consolidated/dispatcher/dispatcher_it_test.go b/pkg/channel/consolidated/dispatcher/dispatcher_it_test.go index b58f64e393..a8460bf64a 100644 --- a/pkg/channel/consolidated/dispatcher/dispatcher_it_test.go +++ b/pkg/channel/consolidated/dispatcher/dispatcher_it_test.go @@ -33,6 +33,7 @@ import ( "github.com/cloudevents/sdk-go/v2/test" "github.com/stretchr/testify/require" "go.uber.org/zap" + "k8s.io/apimachinery/pkg/types" "knative.dev/eventing/pkg/channel/fanout" "knative.dev/eventing/pkg/kncloudevents" "knative.dev/pkg/apis" @@ -82,7 +83,7 @@ func TestDispatcher(t *testing.T) { } // Create the dispatcher. At this point, if Kafka is not up, this thing fails - dispatcher, err := NewDispatcher(context.Background(), &dispatcherArgs) + dispatcher, err := NewDispatcher(context.Background(), &dispatcherArgs, func(ref types.NamespacedName) {}) if err != nil { t.Skipf("no dispatcher: %v", err) } diff --git a/pkg/channel/consolidated/dispatcher/dispatcher_test.go b/pkg/channel/consolidated/dispatcher/dispatcher_test.go index 5245c3d888..b431fca636 100644 --- a/pkg/channel/consolidated/dispatcher/dispatcher_test.go +++ b/pkg/channel/consolidated/dispatcher/dispatcher_test.go @@ -35,12 +35,13 @@ import ( "k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/util/sets" - "knative.dev/eventing-kafka/pkg/channel/consolidated/utils" - "knative.dev/eventing-kafka/pkg/common/consumer" eventingchannels "knative.dev/eventing/pkg/channel" "knative.dev/eventing/pkg/channel/fanout" klogtesting "knative.dev/pkg/logging/testing" _ "knative.dev/pkg/system/testing" + + "knative.dev/eventing-kafka/pkg/channel/consolidated/utils" + "knative.dev/eventing-kafka/pkg/common/consumer" ) // ----- Mocks @@ -50,7 +51,7 @@ type mockKafkaConsumerFactory struct { createErr bool } -func (c mockKafkaConsumerFactory) StartConsumerGroup(ctx context.Context, groupID string, topics []string, handler consumer.KafkaConsumerHandler, options ...consumer.SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { +func (c mockKafkaConsumerFactory) StartConsumerGroup(ctx context.Context, groupID string, topics []string, handler consumer.KafkaConsumerHandler, ref types.NamespacedName, options ...consumer.SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { if c.createErr { return nil, errors.New("error creating consumer") } @@ -519,7 +520,7 @@ func TestNewDispatcher(t *testing.T) { Brokers: []string{"localhost:10000"}, TopicFunc: utils.TopicName, } - _, err := NewDispatcher(context.TODO(), args) + _, err := NewDispatcher(context.TODO(), args, func(ref types.NamespacedName) {}) if err == nil { t.Errorf("Expected error want %s, got %s", "message receiver is not set", err) } diff --git a/pkg/channel/consolidated/reconciler/dispatcher/kafkachannel.go b/pkg/channel/consolidated/reconciler/dispatcher/kafkachannel.go index ccebe1dece..439c959edf 100644 --- a/pkg/channel/consolidated/reconciler/dispatcher/kafkachannel.go +++ b/pkg/channel/consolidated/reconciler/dispatcher/kafkachannel.go @@ -106,15 +106,8 @@ func NewController(ctx context.Context, cmw configmap.Watcher) *controller.Impl Config: kafkaConfig.EventingKafka, TopicFunc: utils.TopicName, } - kafkaDispatcher, err := dispatcher.NewDispatcher(ctx, args) - if err != nil { - logger.Fatalw("Unable to create kafka dispatcher", zap.Error(err)) - } - logger.Info("Starting the Kafka dispatcher") - logger.Infow("Kafka broker configuration", zap.Strings("Brokers", kafkaConfig.Brokers)) r := &Reconciler{ - kafkaDispatcher: kafkaDispatcher, kafkaClientSet: kafkaclientsetinjection.Get(ctx), kafkachannelLister: kafkaChannelInformer.Lister(), kafkachannelInformer: kafkaChannelInformer.Informer(), @@ -123,6 +116,15 @@ func NewController(ctx context.Context, cmw configmap.Watcher) *controller.Impl return controller.Options{SkipStatusUpdates: true} }) + kafkaDispatcher, err := dispatcher.NewDispatcher(ctx, args, r.impl.EnqueueKey) + if err != nil { + logger.Fatalw("Unable to create kafka dispatcher", zap.Error(err)) + } + logger.Info("Starting the Kafka dispatcher") + logger.Infow("Kafka broker configuration", zap.Strings("Brokers", kafkaConfig.Brokers)) + + r.kafkaDispatcher = kafkaDispatcher + logger.Info("Setting up event handlers") // Watch for kafka channels. diff --git a/pkg/channel/distributed/dispatcher/controller/kafkachannel.go b/pkg/channel/distributed/dispatcher/controller/kafkachannel.go index e47e0ec323..daa00127e5 100644 --- a/pkg/channel/distributed/dispatcher/controller/kafkachannel.go +++ b/pkg/channel/distributed/dispatcher/controller/kafkachannel.go @@ -232,7 +232,11 @@ func (r Reconciler) reconcile(ctx context.Context, channel *kafkav1beta1.KafkaCh } // Update The ConsumerGroups To Align With Current KafkaChannel Subscribers - subscriptions := r.dispatcher.UpdateSubscriptions(ctx, subscribers) + channelRef := types.NamespacedName{ + Namespace: channel.GetNamespace(), + Name: channel.GetName(), + } + subscriptions := r.dispatcher.UpdateSubscriptions(ctx, channelRef, subscribers) // Update The KafkaChannel Subscribable Status Based On ConsumerGroup Creation Status channel.Status.SubscribableStatus = r.createSubscribableStatus(channel.Spec.Subscribers, subscriptions) diff --git a/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go b/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go index c5c545b415..399f5932cf 100644 --- a/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go +++ b/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go @@ -365,7 +365,7 @@ func (m *MockDispatcher) Shutdown() { m.Called() } -func (m *MockDispatcher) UpdateSubscriptions(_ context.Context, subscriberSpecs []eventingduck.SubscriberSpec) consumer.SubscriberStatusMap { +func (m *MockDispatcher) UpdateSubscriptions(_ context.Context, ref types.NamespacedName, subscriberSpecs []eventingduck.SubscriberSpec) consumer.SubscriberStatusMap { args := m.Called(subscriberSpecs) return args.Get(0).(consumer.SubscriberStatusMap) } diff --git a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher.go b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher.go index f2d74c737d..c41653434f 100644 --- a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher.go +++ b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher.go @@ -65,7 +65,7 @@ func NewSubscriberWrapper(subscriberSpec eventingduck.SubscriberSpec, groupId st type Dispatcher interface { SecretChanged(ctx context.Context, secret *corev1.Secret) Shutdown() - UpdateSubscriptions(ctx context.Context, subscriberSpecs []eventingduck.SubscriberSpec) commonconsumer.SubscriberStatusMap + UpdateSubscriptions(ctx context.Context, channelRef types.NamespacedName, subscriberSpecs []eventingduck.SubscriberSpec) commonconsumer.SubscriberStatusMap } // DispatcherImpl Is A Struct With Configuration & ConsumerGroup State @@ -83,9 +83,9 @@ type DispatcherImpl struct { var _ Dispatcher = &DispatcherImpl{} // NewDispatcher Is The Dispatcher Constructor -func NewDispatcher(dispatcherConfig DispatcherConfig, controlServer controlprotocol.ServerHandler) (Dispatcher, <-chan commonconsumer.ManagerEvent) { +func NewDispatcher(dispatcherConfig DispatcherConfig, controlServer controlprotocol.ServerHandler, enqueue func(ref types.NamespacedName)) (Dispatcher, <-chan commonconsumer.ManagerEvent) { - consumerGroupManager := commonconsumer.NewConsumerGroupManager(dispatcherConfig.Logger, controlServer, dispatcherConfig.Brokers, dispatcherConfig.SaramaConfig, &commonconsumer.NoopConsumerGroupOffsetsChecker{}) + consumerGroupManager := commonconsumer.NewConsumerGroupManager(dispatcherConfig.Logger, controlServer, dispatcherConfig.Brokers, dispatcherConfig.SaramaConfig, &commonconsumer.NoopConsumerGroupOffsetsChecker{}, enqueue) // Create The DispatcherImpl With Specified Configuration dispatcher := &DispatcherImpl{ @@ -123,7 +123,7 @@ func (d *DispatcherImpl) Shutdown() { } // UpdateSubscriptions manages the Dispatcher's Subscriptions to align with new state -func (d *DispatcherImpl) UpdateSubscriptions(ctx context.Context, subscriberSpecs []eventingduck.SubscriberSpec) commonconsumer.SubscriberStatusMap { +func (d *DispatcherImpl) UpdateSubscriptions(ctx context.Context, channelRef types.NamespacedName, subscriberSpecs []eventingduck.SubscriberSpec) commonconsumer.SubscriberStatusMap { if d.SaramaConfig == nil { d.Logger.Error("Dispatcher has no config!") @@ -151,7 +151,7 @@ func (d *DispatcherImpl) UpdateSubscriptions(ctx context.Context, subscriberSpec // Create/Start A New ConsumerGroup With Custom Handler handler := NewHandler(logger, groupId, &subscriberSpec) - err := d.consumerMgr.StartConsumerGroup(ctx, groupId, []string{d.Topic}, handler) + err := d.consumerMgr.StartConsumerGroup(ctx, groupId, []string{d.Topic}, handler, channelRef) if err != nil { // Log & Return Failure diff --git a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go index f529b63a56..e8a368235d 100644 --- a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go +++ b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go @@ -380,7 +380,7 @@ func TestUpdateSubscriptions(t *testing.T) { } // Perform The Test - result := dispatcher.UpdateSubscriptions(ctx, testCase.args.subscriberSpecs) + result := dispatcher.UpdateSubscriptions(ctx, types.NamespacedName{}, testCase.args.subscriberSpecs) close(errorSource) @@ -572,7 +572,7 @@ func createTestDispatcher(t *testing.T, brokers []string, config *sarama.Config) serverHandler.Service.On("SendAndWaitForAck", mock.Anything, mock.Anything).Return(nil) // Create The Dispatcher - dispatcher, events := NewDispatcher(dispatcherConfig, serverHandler) + dispatcher, events := NewDispatcher(dispatcherConfig, serverHandler, func(ref types.NamespacedName) {}) assert.NotNil(t, events) serverHandler.AssertExpectations(t) diff --git a/pkg/common/consumer/consumer_factory.go b/pkg/common/consumer/consumer_factory.go index dabaebf43c..bdc8fb4b24 100644 --- a/pkg/common/consumer/consumer_factory.go +++ b/pkg/common/consumer/consumer_factory.go @@ -22,6 +22,7 @@ import ( "github.com/Shopify/sarama" "go.uber.org/zap" + "k8s.io/apimachinery/pkg/types" "knative.dev/pkg/logging" ) @@ -33,13 +34,14 @@ type consumeFunc func(ctx context.Context, topics []string, handler sarama.Consu // KafkaConsumerGroupFactory creates the ConsumerGroup and start consuming the specified topic type KafkaConsumerGroupFactory interface { - StartConsumerGroup(ctx context.Context, groupID string, topics []string, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) + StartConsumerGroup(ctx context.Context, groupID string, topics []string, handler KafkaConsumerHandler, channelRef types.NamespacedName, options ...SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) } type kafkaConsumerGroupFactoryImpl struct { config *sarama.Config addrs []string offsetsChecker ConsumerGroupOffsetsChecker + enqueue func(ref types.NamespacedName) } type customConsumerGroup struct { @@ -66,7 +68,7 @@ func (c *customConsumerGroup) Close() error { var _ sarama.ConsumerGroup = (*customConsumerGroup)(nil) // StartConsumerGroup creates a new customConsumerGroup and starts a Consume goroutine on it -func (c kafkaConsumerGroupFactoryImpl) StartConsumerGroup(ctx context.Context, groupID string, topics []string, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { +func (c kafkaConsumerGroupFactoryImpl) StartConsumerGroup(ctx context.Context, groupID string, topics []string, handler KafkaConsumerHandler, channelRef types.NamespacedName, options ...SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { logger := logging.FromContext(ctx) consumerGroup, err := c.createConsumerGroup(groupID) @@ -76,7 +78,7 @@ func (c kafkaConsumerGroupFactoryImpl) StartConsumerGroup(ctx context.Context, g } // Start the consumerGroup.Consume function in a separate goroutine - return c.startExistingConsumerGroup(groupID, consumerGroup, consumerGroup.Consume, topics, logger, handler, options...), nil + return c.startExistingConsumerGroup(groupID, consumerGroup, consumerGroup.Consume, topics, logger, handler, channelRef, options...), nil } // createConsumerGroup creates a Sarama ConsumerGroup using the newConsumerGroup wrapper, with the @@ -87,14 +89,7 @@ func (c kafkaConsumerGroupFactoryImpl) createConsumerGroup(groupID string) (sara // startExistingConsumerGroup creates a goroutine that begins a custom Consume loop on the provided ConsumerGroup // This loop is cancelable via the function provided in the returned customConsumerGroup. -func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup( - groupID string, - saramaGroup sarama.ConsumerGroup, - consume consumeFunc, - topics []string, - logger *zap.SugaredLogger, - handler KafkaConsumerHandler, - options ...SaramaConsumerHandlerOption) *customConsumerGroup { +func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup(groupID string, saramaGroup sarama.ConsumerGroup, consume consumeFunc, topics []string, logger *zap.SugaredLogger, handler KafkaConsumerHandler, channelRef types.NamespacedName, options ...SaramaConsumerHandlerOption) *customConsumerGroup { errorCh := make(chan error, 10) releasedCh := make(chan bool) @@ -105,8 +100,15 @@ func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup( // do not proceed until the check is done err := c.offsetsChecker.WaitForOffsetsInitialization(ctx, groupID, topics, logger, c.addrs, c.config) if err != nil { - logger.Errorw("error while checking if offsets are initialized", zap.Any("topics", topics), zap.String("groupId", groupID), zap.Error(err)) + logger.Errorw("error while checking if offsets are initialized", + zap.Any("topics", topics), + zap.String("groupId", groupID), + zap.String("channel", channelRef.String()), + zap.Error(err), + ) errorCh <- err + c.enqueue(channelRef) + return } logger.Debugw("all offsets are initialized", zap.Any("topics", topics), zap.Any("groupID", groupID)) @@ -136,8 +138,8 @@ func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup( return &customConsumerGroup{cancel, errorCh, saramaGroup, releasedCh} } -func NewConsumerGroupFactory(addrs []string, config *sarama.Config, offsetsChecker ConsumerGroupOffsetsChecker) KafkaConsumerGroupFactory { - return kafkaConsumerGroupFactoryImpl{addrs: addrs, config: config, offsetsChecker: offsetsChecker} +func NewConsumerGroupFactory(addrs []string, config *sarama.Config, offsetsChecker ConsumerGroupOffsetsChecker, enqueue func(ref types.NamespacedName)) KafkaConsumerGroupFactory { + return kafkaConsumerGroupFactoryImpl{addrs: addrs, config: config, offsetsChecker: offsetsChecker, enqueue: enqueue} } var _ KafkaConsumerGroupFactory = (*kafkaConsumerGroupFactoryImpl)(nil) diff --git a/pkg/common/consumer/consumer_factory_test.go b/pkg/common/consumer/consumer_factory_test.go index 1cc309de5a..87342ef12a 100644 --- a/pkg/common/consumer/consumer_factory_test.go +++ b/pkg/common/consumer/consumer_factory_test.go @@ -24,6 +24,7 @@ import ( "github.com/Shopify/sarama" "go.uber.org/zap" + "k8s.io/apimachinery/pkg/types" controllertesting "knative.dev/eventing-kafka/pkg/common/commands/resetoffset/controller/testing" commontesting "knative.dev/eventing-kafka/pkg/common/testing" @@ -151,7 +152,7 @@ func TestErrorPropagationCustomConsumerGroup(t *testing.T) { offsetsChecker: &mockConsumerGroupOffsetsChecker{}, } - consumerGroup, err := factory.StartConsumerGroup(ctx, "bla", []string{}, nil) + consumerGroup, err := factory.StartConsumerGroup(ctx, "bla", []string{}, nil, types.NamespacedName{}) if err != nil { t.Errorf("Should not throw error %v", err) } @@ -200,7 +201,7 @@ func TestErrorWhileCreatingNewConsumerGroup(t *testing.T) { addrs: []string{"b1", "b2"}, offsetsChecker: &mockConsumerGroupOffsetsChecker{}, } - _, err := factory.StartConsumerGroup(ctx, "bla", []string{}, nil) + _, err := factory.StartConsumerGroup(ctx, "bla", []string{}, nil, types.NamespacedName{}) if err == nil || err.Error() != "failed" { t.Errorf("Should contain an error with message failed. Got %v", err) @@ -216,7 +217,7 @@ func TestErrorWhileNewConsumerGroup(t *testing.T) { addrs: []string{"b1", "b2"}, offsetsChecker: &mockConsumerGroupOffsetsChecker{}, } - consumerGroup, _ := factory.StartConsumerGroup(ctx, "bla", []string{}, nil) + consumerGroup, _ := factory.StartConsumerGroup(ctx, "bla", []string{}, nil, types.NamespacedName{}) consumerGroup.(*customConsumerGroup).cancel() // Stop the consume loop from spinning after the error is generated err := <-consumerGroup.Errors() diff --git a/pkg/common/consumer/consumer_manager.go b/pkg/common/consumer/consumer_manager.go index 02a2bcc2b7..bf82bbe519 100644 --- a/pkg/common/consumer/consumer_manager.go +++ b/pkg/common/consumer/consumer_manager.go @@ -94,7 +94,7 @@ type ManagerEvent struct { // KafkaConsumerGroupManager keeps track of Sarama consumer groups and handles messages from control-protocol clients type KafkaConsumerGroupManager interface { Reconfigure(brokers []string, config *sarama.Config) *ReconfigureError - StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) error + StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler KafkaConsumerHandler, ref types.NamespacedName, options ...SaramaConsumerHandlerOption) error CloseConsumerGroup(groupId string) error Errors(groupId string) <-chan error IsManaged(groupId string) bool @@ -141,13 +141,13 @@ func (r ReconfigureError) Error() string { var _ error = (*ReconfigureError)(nil) // NewConsumerGroupManager returns a new kafkaConsumerGroupManagerImpl as a KafkaConsumerGroupManager interface -func NewConsumerGroupManager(logger *zap.Logger, serverHandler controlprotocol.ServerHandler, brokers []string, config *sarama.Config, offsetsChecker ConsumerGroupOffsetsChecker) KafkaConsumerGroupManager { +func NewConsumerGroupManager(logger *zap.Logger, serverHandler controlprotocol.ServerHandler, brokers []string, config *sarama.Config, offsetsChecker ConsumerGroupOffsetsChecker, enqueue func(ref types.NamespacedName)) KafkaConsumerGroupManager { manager := &kafkaConsumerGroupManagerImpl{ logger: logger, server: serverHandler, groups: make(groupMap), - factory: &kafkaConsumerGroupFactoryImpl{addrs: brokers, config: config, offsetsChecker: offsetsChecker}, + factory: &kafkaConsumerGroupFactoryImpl{addrs: brokers, config: config, enqueue: enqueue, offsetsChecker: offsetsChecker}, groupLock: sync.RWMutex{}, eventLock: sync.Mutex{}, offsetsChecker: offsetsChecker, @@ -264,7 +264,7 @@ func (m *kafkaConsumerGroupManagerImpl) Reconfigure(brokers []string, config *sa // StartConsumerGroup uses the consumer factory to create a new ConsumerGroup, add it to the list // of managed groups (for start/stop functionality) and start the Consume loop. -func (m *kafkaConsumerGroupManagerImpl) StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler KafkaConsumerHandler, options ...SaramaConsumerHandlerOption) error { +func (m *kafkaConsumerGroupManagerImpl) StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler KafkaConsumerHandler, ref types.NamespacedName, options ...SaramaConsumerHandlerOption) error { logger := logging.FromContext(ctx) groupLogger := m.logger.With(zap.String("GroupId", groupId)) @@ -286,7 +286,7 @@ func (m *kafkaConsumerGroupManagerImpl) StartConsumerGroup(ctx context.Context, } // The only thing we really want from the factory is the cancel function for the customConsumerGroup - customGroup := m.factory.startExistingConsumerGroup(groupId, group, consume, topics, logger, handler, options...) + customGroup := m.factory.startExistingConsumerGroup(groupId, group, consume, topics, logger, handler, ref, options...) managedGrp := createManagedGroup(ctx, m.logger, group, cancel, customGroup.cancel) // Add the Sarama ConsumerGroup we obtained from the factory to the managed group map, diff --git a/pkg/common/consumer/consumer_manager_test.go b/pkg/common/consumer/consumer_manager_test.go index 7b69332bb1..51b7673681 100644 --- a/pkg/common/consumer/consumer_manager_test.go +++ b/pkg/common/consumer/consumer_manager_test.go @@ -27,6 +27,7 @@ import ( "github.com/Shopify/sarama" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "k8s.io/apimachinery/pkg/types" ctrl "knative.dev/control-protocol/pkg" ctrlservice "knative.dev/control-protocol/pkg/service" logtesting "knative.dev/pkg/logging/testing" @@ -38,7 +39,7 @@ import ( func TestNewConsumerGroupManager(t *testing.T) { server := getMockServerHandler() - manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), server, []string{}, &sarama.Config{}, &NoopConsumerGroupOffsetsChecker{}) + manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), server, []string{}, &sarama.Config{}, &NoopConsumerGroupOffsetsChecker{}, func(ref types.NamespacedName) {}) assert.NotNil(t, manager) assert.NotNil(t, server.Router[commands.StopConsumerGroupOpCode]) assert.NotNil(t, server.Router[commands.StartConsumerGroupOpCode]) @@ -156,7 +157,7 @@ func TestStartConsumerGroup(t *testing.T) { } { t.Run(testCase.name, func(t *testing.T) { ctx := context.TODO() - manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), getMockServerHandler(), []string{}, &sarama.Config{}, &NoopConsumerGroupOffsetsChecker{}) + manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), getMockServerHandler(), []string{}, &sarama.Config{}, &NoopConsumerGroupOffsetsChecker{}, func(ref types.NamespacedName) {}) mockGroup := kafkatesting.NewMockConsumerGroup() newConsumerGroup = func(addrs []string, groupID string, config *sarama.Config) (sarama.ConsumerGroup, error) { if testCase.factoryErr { @@ -165,7 +166,7 @@ func TestStartConsumerGroup(t *testing.T) { mockGroup.On("Errors").Return(mockGroup.ErrorChan) return mockGroup, nil } - err := manager.StartConsumerGroup(ctx, "testid", []string{}, nil, nil) + err := manager.StartConsumerGroup(ctx, "testid", []string{}, nil, types.NamespacedName{}) assert.Equal(t, testCase.factoryErr, err != nil) time.Sleep(shortTimeout) // Give the transferErrors routine a chance to call Errors() mockGroup.AssertExpectations(t) @@ -621,7 +622,7 @@ func getManagerWithMockGroup(t *testing.T, groupId string, factoryErr bool) (Kaf mockGroup.On("Errors").Return(make(chan error)) return mockGroup, nil } - manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), serverHandler, []string{}, &sarama.Config{}, &NoopConsumerGroupOffsetsChecker{}) + manager := NewConsumerGroupManager(logtesting.TestLogger(t).Desugar(), serverHandler, []string{}, &sarama.Config{}, &NoopConsumerGroupOffsetsChecker{}, func(ref types.NamespacedName) {}) if groupId != "" { mockGroup, managedGrp := createMockAndManagedGroups(t) manager.(*kafkaConsumerGroupManagerImpl).groups[groupId] = managedGrp diff --git a/pkg/common/consumer/testing/mocks.go b/pkg/common/consumer/testing/mocks.go index 428d66f134..f507cc4bd5 100644 --- a/pkg/common/consumer/testing/mocks.go +++ b/pkg/common/consumer/testing/mocks.go @@ -21,6 +21,8 @@ import ( "github.com/Shopify/sarama" "github.com/stretchr/testify/mock" + "k8s.io/apimachinery/pkg/types" + "knative.dev/eventing-kafka/pkg/common/consumer" ) @@ -33,7 +35,7 @@ type MockKafkaConsumerGroupFactory struct { mock.Mock } -func (c *MockKafkaConsumerGroupFactory) StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler consumer.KafkaConsumerHandler, options ...consumer.SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { +func (c *MockKafkaConsumerGroupFactory) StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler consumer.KafkaConsumerHandler, ref types.NamespacedName, options ...consumer.SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { args := c.Called(ctx, groupId, topics, handler, options) return args.Get(0).(sarama.ConsumerGroup), args.Error(1) } @@ -61,7 +63,7 @@ func (m *MockConsumerGroupManager) Reconfigure(brokers []string, config *sarama. } func (m *MockConsumerGroupManager) StartConsumerGroup(ctx context.Context, groupId string, topics []string, - handler consumer.KafkaConsumerHandler, options ...consumer.SaramaConsumerHandlerOption) error { + handler consumer.KafkaConsumerHandler, _ types.NamespacedName, options ...consumer.SaramaConsumerHandlerOption) error { return m.Called(ctx, groupId, topics, handler, options).Error(0) } diff --git a/pkg/source/adapter/adapter.go b/pkg/source/adapter/adapter.go index d2f5a1c929..90e41c4098 100644 --- a/pkg/source/adapter/adapter.go +++ b/pkg/source/adapter/adapter.go @@ -27,6 +27,7 @@ import ( "time" "golang.org/x/time/rate" + "k8s.io/apimachinery/pkg/types" ctrl "knative.dev/control-protocol/pkg" ctrlnetwork "knative.dev/control-protocol/pkg/network" @@ -140,12 +141,13 @@ func (a *Adapter) Start(ctx context.Context) (err error) { a.saramaConfig = config options := []consumer.SaramaConsumerHandlerOption{consumer.WithSaramaConsumerLifecycleListener(a)} - consumerGroupFactory := consumer.NewConsumerGroupFactory(addrs, config, &consumer.NoopConsumerGroupOffsetsChecker{}) + consumerGroupFactory := consumer.NewConsumerGroupFactory(addrs, config, &consumer.NoopConsumerGroupOffsetsChecker{}, func(ref types.NamespacedName) {}) group, err := consumerGroupFactory.StartConsumerGroup( ctx, a.config.ConsumerGroup, a.config.Topics, a, + types.NamespacedName{Namespace: a.config.Namespace, Name: a.config.Name}, options..., ) if err != nil { From ca60bf4cb2ff19b2ed94bf9ec405decfb3525214 Mon Sep 17 00:00:00 2001 From: Ali Ok Date: Fri, 8 Oct 2021 13:44:01 +0300 Subject: [PATCH 5/7] Fix unit tests and linter issues --- .../dispatcher/controller/kafkachannel_test.go | 6 +++--- .../dispatcher/dispatcher/dispatcher_test.go | 2 +- pkg/common/consumer/consumer_factory.go | 17 ++++++++++------- pkg/common/consumer/consumer_factory_test.go | 12 ------------ pkg/common/consumer/consumer_manager.go | 2 +- pkg/common/consumer/testing/mocks.go | 6 +++--- 6 files changed, 18 insertions(+), 27 deletions(-) diff --git a/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go b/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go index 399f5932cf..f66cb9bf2a 100644 --- a/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go +++ b/pkg/channel/distributed/dispatcher/controller/kafkachannel_test.go @@ -317,7 +317,7 @@ func TestAllCases(t *testing.T) { status consumer.SubscriberStatusMap, ) controller.Reconciler { mockDispatcher := &MockDispatcher{} - mockDispatcher.On("UpdateSubscriptions", mock.Anything).Return(status) + mockDispatcher.On("UpdateSubscriptions", mock.Anything, mock.Anything, mock.Anything).Return(status) return &Reconciler{ logger: logtesting.TestLogger(t).Desugar(), channelKey: kcKey, @@ -365,8 +365,8 @@ func (m *MockDispatcher) Shutdown() { m.Called() } -func (m *MockDispatcher) UpdateSubscriptions(_ context.Context, ref types.NamespacedName, subscriberSpecs []eventingduck.SubscriberSpec) consumer.SubscriberStatusMap { - args := m.Called(subscriberSpecs) +func (m *MockDispatcher) UpdateSubscriptions(ctx context.Context, ref types.NamespacedName, subscriberSpecs []eventingduck.SubscriberSpec) consumer.SubscriberStatusMap { + args := m.Called(ctx, ref, subscriberSpecs) return args.Get(0).(consumer.SubscriberStatusMap) } diff --git a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go index e8a368235d..cd9ae9d80b 100644 --- a/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go +++ b/pkg/channel/distributed/dispatcher/dispatcher/dispatcher_test.go @@ -365,7 +365,7 @@ func TestUpdateSubscriptions(t *testing.T) { if !testCase.wantFailure { mockManager.On("ClearNotifications").Return() for _, id := range testCase.expectStarted { - mockManager.On("StartConsumerGroup", mock.Anything, "kafka."+id, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(testCase.createErr) + mockManager.On("StartConsumerGroup", mock.Anything, "kafka."+id, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(testCase.createErr) } for _, id := range testCase.expectErrors { mockManager.On("Errors", "kafka."+id).Return((<-chan error)(errorSource)) diff --git a/pkg/common/consumer/consumer_factory.go b/pkg/common/consumer/consumer_factory.go index bdc8fb4b24..4ef170cdff 100644 --- a/pkg/common/consumer/consumer_factory.go +++ b/pkg/common/consumer/consumer_factory.go @@ -89,7 +89,15 @@ func (c kafkaConsumerGroupFactoryImpl) createConsumerGroup(groupID string) (sara // startExistingConsumerGroup creates a goroutine that begins a custom Consume loop on the provided ConsumerGroup // This loop is cancelable via the function provided in the returned customConsumerGroup. -func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup(groupID string, saramaGroup sarama.ConsumerGroup, consume consumeFunc, topics []string, logger *zap.SugaredLogger, handler KafkaConsumerHandler, channelRef types.NamespacedName, options ...SaramaConsumerHandlerOption) *customConsumerGroup { +func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup( + groupID string, + saramaGroup sarama.ConsumerGroup, + consume consumeFunc, + topics []string, + logger *zap.SugaredLogger, + handler KafkaConsumerHandler, + channelRef types.NamespacedName, + options ...SaramaConsumerHandlerOption) *customConsumerGroup { errorCh := make(chan error, 10) releasedCh := make(chan bool) @@ -100,12 +108,7 @@ func (c kafkaConsumerGroupFactoryImpl) startExistingConsumerGroup(groupID string // do not proceed until the check is done err := c.offsetsChecker.WaitForOffsetsInitialization(ctx, groupID, topics, logger, c.addrs, c.config) if err != nil { - logger.Errorw("error while checking if offsets are initialized", - zap.Any("topics", topics), - zap.String("groupId", groupID), - zap.String("channel", channelRef.String()), - zap.Error(err), - ) + logger.Errorw("error while checking if offsets are initialized", zap.Any("topics", topics), zap.String("groupId", groupID), zap.String("channel", channelRef.String()), zap.Error(err)) errorCh <- err c.enqueue(channelRef) return diff --git a/pkg/common/consumer/consumer_factory_test.go b/pkg/common/consumer/consumer_factory_test.go index 87342ef12a..cbc50736ea 100644 --- a/pkg/common/consumer/consumer_factory_test.go +++ b/pkg/common/consumer/consumer_factory_test.go @@ -101,18 +101,6 @@ func mockedNewSaramaClient(client *controllertesting.MockClient, mustFail bool) } } -func mockedNewSaramaClusterAdmin(clusterAdmin sarama.ClusterAdmin, mustFail bool) func(addrs []string, config *sarama.Config) (sarama.ClusterAdmin, error) { - if !mustFail { - return func(addrs []string, config *sarama.Config) (sarama.ClusterAdmin, error) { - return clusterAdmin, nil - } - } else { - return func(addrs []string, config *sarama.Config) (sarama.ClusterAdmin, error) { - return nil, errors.New("failed") - } - } -} - func mockedNewSaramaClusterAdminFromClient(clusterAdmin sarama.ClusterAdmin, mustFail bool) func(client sarama.Client) (sarama.ClusterAdmin, error) { if !mustFail { return func(client sarama.Client) (sarama.ClusterAdmin, error) { diff --git a/pkg/common/consumer/consumer_manager.go b/pkg/common/consumer/consumer_manager.go index bf82bbe519..882e4143da 100644 --- a/pkg/common/consumer/consumer_manager.go +++ b/pkg/common/consumer/consumer_manager.go @@ -147,7 +147,7 @@ func NewConsumerGroupManager(logger *zap.Logger, serverHandler controlprotocol.S logger: logger, server: serverHandler, groups: make(groupMap), - factory: &kafkaConsumerGroupFactoryImpl{addrs: brokers, config: config, enqueue: enqueue, offsetsChecker: offsetsChecker}, + factory: &kafkaConsumerGroupFactoryImpl{addrs: brokers, config: config, offsetsChecker: offsetsChecker, enqueue: enqueue}, groupLock: sync.RWMutex{}, eventLock: sync.Mutex{}, offsetsChecker: offsetsChecker, diff --git a/pkg/common/consumer/testing/mocks.go b/pkg/common/consumer/testing/mocks.go index f507cc4bd5..80767b5ac4 100644 --- a/pkg/common/consumer/testing/mocks.go +++ b/pkg/common/consumer/testing/mocks.go @@ -36,7 +36,7 @@ type MockKafkaConsumerGroupFactory struct { } func (c *MockKafkaConsumerGroupFactory) StartConsumerGroup(ctx context.Context, groupId string, topics []string, handler consumer.KafkaConsumerHandler, ref types.NamespacedName, options ...consumer.SaramaConsumerHandlerOption) (sarama.ConsumerGroup, error) { - args := c.Called(ctx, groupId, topics, handler, options) + args := c.Called(ctx, groupId, topics, handler, ref, options) return args.Get(0).(sarama.ConsumerGroup), args.Error(1) } @@ -63,8 +63,8 @@ func (m *MockConsumerGroupManager) Reconfigure(brokers []string, config *sarama. } func (m *MockConsumerGroupManager) StartConsumerGroup(ctx context.Context, groupId string, topics []string, - handler consumer.KafkaConsumerHandler, _ types.NamespacedName, options ...consumer.SaramaConsumerHandlerOption) error { - return m.Called(ctx, groupId, topics, handler, options).Error(0) + handler consumer.KafkaConsumerHandler, channelRef types.NamespacedName, options ...consumer.SaramaConsumerHandlerOption) error { + return m.Called(ctx, groupId, topics, handler, channelRef, options).Error(0) } func (m *MockConsumerGroupManager) CloseConsumerGroup(groupId string) error { From 1587718bd511c734068fdb78d6bbd0e84f2d1114 Mon Sep 17 00:00:00 2001 From: Ali Ok Date: Mon, 11 Oct 2021 09:51:14 +0300 Subject: [PATCH 6/7] Address comments --- pkg/channel/consolidated/dispatcher/dispatcher.go | 3 ++- pkg/common/consumer/consumergroup_offsets_checker.go | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/pkg/channel/consolidated/dispatcher/dispatcher.go b/pkg/channel/consolidated/dispatcher/dispatcher.go index 48335ae21d..b6ce4bb526 100644 --- a/pkg/channel/consolidated/dispatcher/dispatcher.go +++ b/pkg/channel/consolidated/dispatcher/dispatcher.go @@ -72,7 +72,8 @@ type KafkaDispatcher struct { topicFunc TopicFunc logger *zap.SugaredLogger } - +// NewDispatcher creates a new dispatcher struct. enqueue argument is a function that is used to +// requeue a KafkaChannel instance via the reconciler which is used when creating the consumer. func NewDispatcher(ctx context.Context, args *KafkaDispatcherArgs, enqueue func(ref types.NamespacedName)) (*KafkaDispatcher, error) { producer, err := sarama.NewSyncProducer(args.Brokers, args.Config.Sarama.Config) diff --git a/pkg/common/consumer/consumergroup_offsets_checker.go b/pkg/common/consumer/consumergroup_offsets_checker.go index bb94a6334e..0a511de2fd 100644 --- a/pkg/common/consumer/consumergroup_offsets_checker.go +++ b/pkg/common/consumer/consumergroup_offsets_checker.go @@ -52,7 +52,7 @@ func (k *KafkaConsumerGroupOffsetsChecker) WaitForOffsetsInitialization(ctx cont } defer clusterAdmin.Close() - check := func() (bool, error) { + checkIfAllOffsetsInitialized := func() (bool, error) { if initialized, err := offset.CheckIfAllOffsetsInitialized(client, clusterAdmin, topics, groupID); err == nil { if initialized { return true, nil @@ -64,7 +64,7 @@ func (k *KafkaConsumerGroupOffsetsChecker) WaitForOffsetsInitialization(ctx cont return false, fmt.Errorf("error checking if offsets are initialized. stopping trying. %w", err) } } - err = wait.PollImmediate(OffsetCheckRetryInterval, OffsetCheckRetryTimeout, check) + err = wait.PollImmediate(OffsetCheckRetryInterval, OffsetCheckRetryTimeout, checkIfAllOffsetsInitialized) if err != nil { return fmt.Errorf("failed to check if offsets are initialized %w", err) } From 598969ba882296fadca640f9af9c926a08ce94b0 Mon Sep 17 00:00:00 2001 From: Ali Ok Date: Mon, 11 Oct 2021 10:15:51 +0300 Subject: [PATCH 7/7] Fix codestyle --- pkg/channel/consolidated/dispatcher/dispatcher.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/channel/consolidated/dispatcher/dispatcher.go b/pkg/channel/consolidated/dispatcher/dispatcher.go index b6ce4bb526..bd1e7cf2af 100644 --- a/pkg/channel/consolidated/dispatcher/dispatcher.go +++ b/pkg/channel/consolidated/dispatcher/dispatcher.go @@ -72,6 +72,7 @@ type KafkaDispatcher struct { topicFunc TopicFunc logger *zap.SugaredLogger } + // NewDispatcher creates a new dispatcher struct. enqueue argument is a function that is used to // requeue a KafkaChannel instance via the reconciler which is used when creating the consumer. func NewDispatcher(ctx context.Context, args *KafkaDispatcherArgs, enqueue func(ref types.NamespacedName)) (*KafkaDispatcher, error) {