diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml new file mode 100644 index 0000000..8413181 --- /dev/null +++ b/.github/workflows/go.yml @@ -0,0 +1,65 @@ +name: Go + +on: + push: + branches: [main] + tags: ['v*'] + pull_request: + branches: ['*'] + +permissions: + contents: read + +jobs: + build: + runs-on: ubuntu-latest + strategy: + matrix: + go: ["1.22.x"] + include: + - go: 1.22.x + steps: + - name: Checkout code + uses: actions/checkout@v4 + + - name: Setup Go + uses: actions/setup-go@v5 + with: + go-version: ${{ matrix.go }} + cache-dependency-path: '**/go.sum' + + - name: Download Dependencies + run: | + go mod download + + - name: Test + run: make cover + + - name: Upload coverage reports to Codecov + uses: codecov/codecov-action@v4 + with: + token: ${{ secrets.CODECOV_TOKEN }} + + lint: + name: Lint + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v4 + name: Check out repository + - uses: actions/setup-go@v5 + name: Set up Go + with: + go-version: 1.22.x + cache: false # managed by golangci-lint + + - uses: golangci/golangci-lint-action@v6 + name: Install golangci-lint + with: + version: latest + # Hack: Use the official action to download, but not run. + # make lint below will handle actually running the linter. + args: --help + + - run: make lint + name: Lint \ No newline at end of file diff --git a/.gitignore b/.gitignore index 6f6f5e6..6b401df 100644 --- a/.gitignore +++ b/.gitignore @@ -20,3 +20,23 @@ # Go workspace file go.work go.work.sum + +.vscode/ +.idea/ +.run/ +zk-multiple-kafka-multiple/ +*.out +*.res +*.lsif +*.prof +coverage.tmp.xml +coverage.xml +cover.html +cover.out + +/example/worker/worker +/example/consumer/consumer +/example/sandbox + +testdata/fuzz + diff --git a/AUTHORS b/AUTHORS new file mode 100644 index 0000000..c4d8925 --- /dev/null +++ b/AUTHORS @@ -0,0 +1 @@ +Stewart Boyd \ No newline at end of file diff --git a/Dockerfile b/Dockerfile new file mode 100644 index 0000000..f140a23 --- /dev/null +++ b/Dockerfile @@ -0,0 +1,16 @@ +FROM golang:1.22 AS build + +ENV CGO_ENABLED=1 +ENV GOPROXY=https://proxy.golang.org\|https://artifactory.zgtools.net/artifactory/api/go/devex-go\|direct +ENV GONOSUMDB=*gitlab.zgtools.net* + +WORKDIR /go/src/zkafka +COPY . . + +RUN go mod download +RUN go build -o zkafka + +FROM debian +COPY --from=build /go/src/zkafka / +ENTRYPOINT ["/zkafka"] + diff --git a/Makefile b/Makefile new file mode 100644 index 0000000..1396851 --- /dev/null +++ b/Makefile @@ -0,0 +1,23 @@ +.PHONY: test-no-setup +test-no-setup: + ./coverage.sh + +.PHONY: setup-test +setup-test: + docker compose -p $$RANDOM -f ./example/docker-compose.yaml up -d + +.PHONY: test-local +test-local: setup-test test-no-setup + +.PHONY: cover +cover: + go test -v ./... -count=1 -coverprofile=cover.out -covermode atomic && \ + go tool cover -html=cover.out -o cover.html + +.PHONY: example-producer +example-producer: + go run example/producer/producer.go + +.PHONY: example-worker +example-worker: + go run example/worker/worker.go \ No newline at end of file diff --git a/README.md b/README.md new file mode 100644 index 0000000..59bbf14 --- /dev/null +++ b/README.md @@ -0,0 +1,129 @@ +# zkafka + +[![License](https://img.shields.io/github/license/zillow/zkafka)](https://github.com/zillow/zkafka/blob/main/LICENSE) +[![GitHub Actions](https://github.com/zillow/zkafka/actions/workflows/go.yml/badge.svg)](https://github.com/zillow/zkafka/actions/workflows/go.yml) +[![Codecov](https://codecov.io/gh/zillow/zkafka/branch/main/graph/badge.svg?token=STRT8T67YP)](https://codecov.io/gh/zillow/zkafka) + + +## Install + +`go get -u github.com/zillow/zkafka` + +## About + +A library built on top of confluent-kafka-go for reading and writing to kafka with limited Schema Registry support. The +library supports at least once message processing. It does so using a commit strategy built off auto commit and manual +offset storage. + +--- +**NOTE** + +confluent-kafka-go is a CGO module, and therefore so is zkafka. When building zkafka, make sure to set +CGO_ENABLED=1. +--- + +There are two quick definitions important to the understanding of the commit strategy + +1. **Commit** - involves communicating with kafka broker and durably persisting offsets on a kafka broker. +2. **Store** - is the action of updating a local store of message offsets which will be persisted during the commit + action + +## Commit Strategy: + +1. *Store* offset of a message for commit after processing +2. *Commit* messages whose offsets have been stored at configurable intervals (`auto.commit.interval.ms`) +3. *Commit* messages whose offsets have been stored when partitions are revoked +(this is implicitly handled by librdkafka. To see this add debug=cgrp in ConsumerTopicConfig, and there'll be COMMIT logs after a rebalance. +If doing this experience, set the `auto.commit.interval.ms` to a large value to avoid confusion between the rebalance commit) +4. *Commit* messages whose offsets have been stored on close of reader +(this is implicitly handled by librdkafka. To see this add debug=cgrp in ConsumerTopicConfig, and there'll be COMMIT logs after the client is closed, but before the client is destroyed) + +Errors returned on processing are still stored. This avoids issues due to poison pill messages (messages which will +never be able to be processed without error) +as well as transient errors blocking future message processing. Use WithOnDone option to register callback for +additional processing of these messages. + +This strategy is based off +of [Kafka Docs - Offset Management](https://docs.confluent.io/platform/current/clients/consumer.html#offset-management) +where a strategy of asynchronous/synchronous commits is suggested to reduced duplicate messages. + +## Work + +zkafka also supports an abstraction built on top of the reader defined in the Work struct (`work.go`). Work introduces +concurrency by way of the configurable option `Speedup(n int)`. This creates n goroutines which process messages as +they are written to the golang channel assigned to that goroutine. Kafka key ordering is preserved (by a mechanism similar to kafka +partitions) whereby a message sharing the same key will always be written to the same channel (internally, this is called a virtual partition). +By default, the number of virtual partitions is equal 1. +Speedup() can be increased beyond the number of assigned physical partitions without concern of data loss on account of the reader tracking in-work message offsets and only +committing the lowest offset to be completed. Additionally, kafka key ordering is preserved even as the number of virtual partitions increases beyond the number of physical assigned +partitions. + +## SchemaRegistry Support: + +There is limited support for schema registry in zkafka. A schemaID can be hardcoded via configuration. No +communication is done with schema registry, but some primitive checks can be conducted if a schemaID is specified via +configuration. + +### Producers + +Producers will include the schemaID in messages written to kafka (without any further verification). + +### Consumers + +Consumers will verify that the message they're consuming has the schemaID specified in configuration +(if it's specified). Be careful here, as backwards compatible schema evolutions would be treated as an error condition +as the new schemaID wouldn't match what's in the configuration. + +## Consumer/Producer Configuration + +See for description of configuration options and their defaults: + +1. [Consumer Configuration](https://docs.confluent.io/platform/current/installation/configuration/consumer-configs.html) +2. [Producer Configurations](https://docs.confluent.io/platform/current/installation/configuration/producer-configs.html) + +These are primarily specified through the TopicConfig struct. TopicConfig includes strongly typed fields which translate +to librdconfig values. To see translation see config.go. An escape hatch is provided for ad hoc config properties via +the AdditionalProperties map. Here config values that don't have a strongly typed version in TopicConfig may be +specified. Not all specified config values will work (for example `enable.auto.commit=false` would not work with this +client because that value is explicitly set to true after reading of the AdditionalProperties map). + +```json5 + +{ + "KafkaTopicConfig": { + "Topic": "KafkaTopicName", + "BootstrapServers": [ + "localhost:9093" + ], + // translates to librdkafka value "bootstrap.servers" + // specify ad hoc configuration values which don't have a strongly typed version in the TopicConfig struct. + "AdditionalProperties": { + "auto.commit.interval.ms": 1000, + "retry.backoff.ms": 10 + } + } +} + +``` + +3. zkafka.ProcessError + +The `zkafka.ProcessError` can be used to control error handling on a per-message basis. Use of this type is entirely optional. The current options exposed through this type are as follows: +1. `DisableDLTWrite`: if true, the message will not be written to a dead letter topic (if one is configured) +2. `DisableCircuitBreaker`: if true, the message will not count as a failed message for purposes of controlling the circuit breaker. + +## Installation + +go get -u gitlab.zgtools.net/devex/archetypes/gomods/zkafka + +## Running Example + +``` +make setup-test + +// +make example-producer + +// +make example-worker +``` diff --git a/carrier.go b/carrier.go new file mode 100644 index 0000000..1355a01 --- /dev/null +++ b/carrier.go @@ -0,0 +1,57 @@ +package zkafka + +import ( + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "go.opentelemetry.io/otel/propagation" +) + +var _ propagation.TextMapCarrier = (*kMsgCarrier)(nil) +var _ propagation.TextMapCarrier = (*msgCarrier)(nil) + +type kMsgCarrier struct { + msg *kafka.Message +} + +func (c *kMsgCarrier) Get(key string) string { + for _, h := range c.msg.Headers { + if h.Key == key { + return string(h.Value) + } + } + return "" +} + +func (c *kMsgCarrier) Keys() []string { + keys := make([]string, 0, len(c.msg.Headers)) + for _, v := range c.msg.Headers { + keys = append(keys, v.Key) + } + return keys +} + +func (c *kMsgCarrier) Set(key, val string) { + addStringAttribute(c.msg, key, []byte(val)) +} + +type msgCarrier struct { + msg *Message +} + +func (c *msgCarrier) Get(key string) string { + for k, v := range c.msg.Headers { + if k == key { + return string(v) + } + } + return "" +} + +func (c *msgCarrier) Keys() []string { + keys := make([]string, 0, len(c.msg.Headers)) + for k := range c.msg.Headers { + keys = append(keys, k) + } + return keys +} + +func (c *msgCarrier) Set(_, _ string) {} diff --git a/carrier_test.go b/carrier_test.go new file mode 100644 index 0000000..3dd6a2d --- /dev/null +++ b/carrier_test.go @@ -0,0 +1,93 @@ +package zkafka + +import ( + "testing" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/stretchr/testify/require" +) + +func Test_msgCarrier_Get(t *testing.T) { + tests := []struct { + name string + msg *Message + key string + want string + }{ + { + name: "", + msg: &Message{Headers: map[string][]byte{"key": []byte("value")}}, + key: "key", + want: "value", + }, + { + name: "", + msg: &Message{Headers: map[string][]byte{"key": {4, 9, 1, 32, 99}}}, + key: "key", + want: "\u0004\t\u0001 c", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + c := &msgCarrier{ + msg: tt.msg, + } + got := c.Get(tt.key) + require.Equal(t, got, tt.want) + }) + } +} + +func Test_msgCarrier_Keys(t *testing.T) { + tests := []struct { + name string + msg *Message + want []string + }{ + { + name: "", + msg: &Message{Headers: map[string][]byte{"key": []byte("value"), "key2": []byte("value2")}}, + want: []string{"key", "key2"}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + c := &msgCarrier{ + msg: tt.msg, + } + got := c.Keys() + require.ElementsMatch(t, got, tt.want) + }) + } +} + +func Test_msgCarrier_Set_IsANoop(t *testing.T) { + msg := &Message{Headers: map[string][]byte{"key": []byte("value"), "key2": []byte("value2")}} + c := &msgCarrier{ + msg: msg, + } + c.Set("key", "dog") + require.Equal(t, c.Get("key"), "value") +} + +func Test_kMsgCarrier_Set(t *testing.T) { + msg := &kafka.Message{Headers: []kafka.Header{ + { + Key: "key", + Value: []byte("value"), + }, + { + Key: "key2", + Value: []byte("value2"), + }, + }} + c := &kMsgCarrier{ + msg: msg, + } + c.Set("key", "dog") + c.Set("hello", "world") + require.Equal(t, "dog", c.Get("key")) + require.Equal(t, "world", c.Get("hello")) + require.Equal(t, "value2", c.Get("key2")) + require.Len(t, c.Keys(), 3) +} diff --git a/changelog.md b/changelog.md new file mode 100644 index 0000000..5335ebe --- /dev/null +++ b/changelog.md @@ -0,0 +1,61 @@ +# Changelog + +All notable changes to this project will be documented in this file. + +This project adheres to Semantic Versioning. + +## 1.0.0 (July 2024) + +Initial release to public github.com + +Internal Releases Below +---- + +## 4.2.0 (June 2024) + +1. Updated otel to v1.27 +1. Updated semconv to v1.25 + +## 4.1.0 (May 2024) +1. Added consumer delay config `ProcessDelayMillis` this allows the consumption of messages, but delays processing until at least the configured delay has passed since the message was written. Useful for intermediate dead letter messages. +1. Added ability to add headers via writer option (WithHeaders). +1. Updated so `linger.ms=0` is the default. +1. Updated so `socket.nagle.disable=true` is the default. +1. Increased `SessionTimeoutMillis` and `MaxPollIntervalMillis` defaults to be greater than `ProcessDelayMillis` so that an inadvertent long running processor doesn't cause a rebalance. +1. Updated confluent.Config's usage of `config.AdditionalProps`. They can now override any setting. Previously, some of the promoted configs couldn't be set via this map. + +## 4.0.0 (April 2024) +1. Updated to be used with zworker.Work which requires a `work.Run` interface not `work.Do`. The difference been + `work.Run` is executed once, and `zkafka` is responsible for continuously looping, whereas `work.Do` would be continually executed + in a loop. +1. Renamed `zkafka.WithOnDoneWithContext` to `zkafka.WithOnDone` and removed original `zkafka.WithOnDone` option (which didn't provide a context.Context arg) +1. Updated `Writer` interface to include `WriteRaw` method. The concrete type has supported it for some time, but was waiting for a major version roll to update the interface. +1. Updated `zkafka.Message` headers to be a `map[string][]byte` instead of `map[string]interface{}`. This is closer to the transport representation and is more convenient and self documenting. +The interface{} type was a holdover from the original implementation and was not necessary. +1. Removed `ExtractHeaderKeys` (reduce surface area of package). Opinionated API (zillow specific) that now resides in zkafkacomproot +1. Added variadic arguments (`...zkafka.WriteOption`) to `kwriter.Write*` methods . This allows future customization in a noninvasive way. +1. Removed `zcommon` dependency. Introduce hooks which can be used toward the same end +1. Changed interface{} -> any +1. Added lifecycle methods `PostRead`, `PreWrite` and `PostFanout` +1. Added `WithConsumerProvider` and `WithProducerProvider` which is useful for full e2e testing. +1. Updated work to remove read messages that won't be worked, from the inwork count. Allows Faster shutdown + +## 3.0.0 (October 2023) + +1. Supports migration from Datadog [statsd](https://www.datadoghq.com/blog/statsd/) to [Panoptes](https://getpanoptes.io/). +2. Removes the `Metrics` interface and related options. Removes `NoopMetrics` struct. Rather than calling metrics classes directly, the user registers lifecycle hooks and calls the metric provider from the hooks. For example, [zkafkacomproot](https://gitlab.zgtools.net/devex/archetypes/gomods/zkafkacomproot) registers hooks that call the zmetric provider. +3. Removes the `RequestContextExtractor` interface. Instead, use the `PreProcessing` lifecycle hook to extract information from the request and add it to the context. The context returned from the `PreProcessing` hook is used for the rest of the request lifecycle. + +## 2.0.0 (July 27th 2023) + +1. Removes the dependency on [opentracing-go](https://github.com/opentracing/opentracing-go). + Opentracing-go was a stale dependency that was no longer receiving updates. The library is now instrumented with [opentelemtry](https://github.com/open-telemetry/opentelemetry-go) + a stable tracing library, that was written with backwards compatability in mind with opentracing-go. +2. Removed `WithTracer(opentracing.Tracer)`. Use `WithTracerProvider` and `WithTextMapPropagator` instead. + +## 1.0.0 (August 2022) + +Updated to account for update in zfmt which changes the values of some of the formatter factory +entry values. + +To see further details on zmt update to V1. See migration guide [here](https://gitlab.zgtools.net/devex/archetypes/gomods/zfmt/-/blob/main/README.md#migration-guide) diff --git a/client.go b/client.go new file mode 100644 index 0000000..3791d43 --- /dev/null +++ b/client.go @@ -0,0 +1,174 @@ +package zkafka + +//go:generate mockgen -package=mock_zkafka -destination=./mocks/mock_client.go -source=./client.go + +import ( + "context" + "fmt" + "sync" + + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/trace" +) + +// ClientProvider is the convenient interface for kafka Client +type ClientProvider interface { + Reader(ctx context.Context, topicConfig ConsumerTopicConfig, opts ...ReaderOption) (Reader, error) + Writer(ctx context.Context, topicConfig ProducerTopicConfig, opts ...WriterOption) (Writer, error) + Close() error +} + +// static type checking for the convenient Writer interface +var _ ClientProvider = (*Client)(nil) + +const instrumentationName = "github.com/zillow/zkafka" + +// Client helps instantiate usable readers and writers +type Client struct { + mu sync.RWMutex + conf Config + readers map[string]*KReader + writers map[string]*KWriter + logger Logger + lifecycle LifecycleHooks + groupPrefix string + tp trace.TracerProvider + p propagation.TextMapPropagator + + // confluent dependencies + producerProvider confluentProducerProvider + consumerProvider confluentConsumerProvider +} + +// NewClient instantiates a kafka client to get readers and writers +func NewClient(conf Config, opts ...Option) *Client { + c := &Client{ + conf: conf, + readers: make(map[string]*KReader), + writers: make(map[string]*KWriter), + logger: NoopLogger{}, + + producerProvider: defaultConfluentProducerProvider{}.NewProducer, + consumerProvider: defaultConfluentConsumerProvider{}.NewConsumer, + } + for _, opt := range opts { + opt(c) + } + return c +} + +// Reader gets a kafka consumer from the provided config, either from cache or from a new instance +func (c *Client) Reader(_ context.Context, topicConfig ConsumerTopicConfig, opts ...ReaderOption) (Reader, error) { + err := getDefaultConsumerTopicConfig(&topicConfig) + if err != nil { + return nil, err + } + c.mu.RLock() + r, exist := c.readers[topicConfig.ClientID] + if exist && !r.isClosed { + c.mu.RUnlock() + return r, nil + } + c.mu.RUnlock() + + c.mu.Lock() + defer c.mu.Unlock() + r, exist = c.readers[topicConfig.ClientID] + if exist && !r.isClosed { + return r, nil + } + + reader, err := newReader(c.conf, topicConfig, c.consumerProvider, c.logger, c.groupPrefix) + if err != nil { + return nil, err + } + // copy settings from client first + reader.lifecycle = c.lifecycle + + // overwrite options if given + for _, opt := range opts { + opt(reader) + } + c.readers[topicConfig.ClientID] = reader + return c.readers[topicConfig.ClientID], nil +} + +// Writer gets a kafka producer from the provided config, either from cache or from a new instance +func (c *Client) Writer(_ context.Context, topicConfig ProducerTopicConfig, opts ...WriterOption) (Writer, error) { + err := getDefaultProducerTopicConfig(&topicConfig) + if err != nil { + return nil, err + } + c.mu.RLock() + w, exist := c.writers[topicConfig.ClientID] + if exist && !w.isClosed { + c.mu.RUnlock() + return w, nil + } + c.mu.RUnlock() + + c.mu.Lock() + defer c.mu.Unlock() + w, exist = c.writers[topicConfig.ClientID] + if exist && !w.isClosed { + return w, nil + } + writer, err := newWriter(c.conf, topicConfig, c.producerProvider) + if err != nil { + return nil, err + } + // copy settings from client first + writer.logger = c.logger + writer.tracer = getTracer(c.tp) + writer.p = c.p + writer.lifecycle = c.lifecycle + + // overwrite options if given + for _, opt := range opts { + opt(writer) + } + c.writers[topicConfig.ClientID] = writer + return c.writers[topicConfig.ClientID], nil +} + +func getFormatter(topicConfig TopicConfig) (zfmt.Formatter, error) { + var fmtter zfmt.Formatter + switch topicConfig.GetFormatter() { + case CustomFmt: + fmtter = &noopFormatter{} + default: + fmtter, _ = zfmt.GetFormatter(topicConfig.GetFormatter(), topicConfig.GetSchemaID()) + } + if fmtter == nil { + return nil, fmt.Errorf("unsupported formatter %s", topicConfig.GetFormatter()) + } + return fmtter, nil +} + +// Close terminates all cached readers and writers gracefully. +func (c *Client) Close() error { + c.mu.Lock() + defer c.mu.Unlock() + if c.logger != nil { + c.logger.Debugw(context.Background(), "Close writers and readers") + } + + var err error + for _, w := range c.writers { + w.Close() + } + for _, r := range c.readers { + if e := r.Close(); e != nil { + err = e + } + } + return err +} + +func getTracer(tp trace.TracerProvider) trace.Tracer { + if tp == nil { + return nil + } + return tp.Tracer(instrumentationName, trace.WithInstrumentationVersion("v1.0.0")) +} diff --git a/client_test.go b/client_test.go new file mode 100644 index 0000000..61d85bd --- /dev/null +++ b/client_test.go @@ -0,0 +1,1179 @@ +package zkafka + +import ( + "context" + "errors" + "fmt" + "runtime/debug" + "sync" + "testing" + "time" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/golang/mock/gomock" + "github.com/google/go-cmp/cmp" + "github.com/google/go-cmp/cmp/cmpopts" + "github.com/stretchr/testify/require" + mock_confluent "github.com/zillow/zkafka/mocks/confluent" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/trace" + "go.opentelemetry.io/otel/trace/noop" +) + +func TestNewClient(t *testing.T) { + type args struct { + conf Config + } + tests := []struct { + name string + args args + want *Client + }{ + { + name: "empty config", + want: &Client{ + readers: make(map[string]*KReader), + writers: make(map[string]*KWriter), + logger: NoopLogger{}, + producerProvider: defaultConfluentProducerProvider{}.NewProducer, + consumerProvider: defaultConfluentConsumerProvider{}.NewConsumer, + }, + }, + { + name: "some config", + args: args{ + conf: Config{ + BootstrapServers: []string{"test"}, + }, + }, + want: &Client{ + conf: Config{ + BootstrapServers: []string{"test"}, + }, + readers: make(map[string]*KReader), + writers: make(map[string]*KWriter), + logger: NoopLogger{}, + producerProvider: defaultConfluentProducerProvider{}.NewProducer, + consumerProvider: defaultConfluentConsumerProvider{}.NewConsumer, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + got := NewClient(tt.args.conf) + require.Equal(t, tt.want.conf, got.conf) + require.Equal(t, tt.want.readers, got.readers) + require.Equal(t, tt.want.writers, got.writers) + require.Equal(t, tt.want.logger, got.logger) + }) + } +} + +func TestClient_WithOptions(t *testing.T) { + defer recoverThenFail(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + type fakeTracerProvider struct { + trace.TracerProvider + Salt int + } + type fakePropagator struct { + propagation.TextMapPropagator + Salt int + } + + tp := fakeTracerProvider{Salt: 123} + p := fakePropagator{Salt: 456} + lprep := func(ctx context.Context, meta LifecyclePreProcessingMeta) (context.Context, error) { + return ctx, errors.New("7") + } + lpostp := func(ctx context.Context, meta LifecyclePostProcessingMeta) error { + return errors.New("8") + } + lposta := func(ctx context.Context, meta LifecyclePostAckMeta) error { + return errors.New("9") + } + + hooks := LifecycleHooks{ + PreProcessing: lprep, + PostProcessing: lpostp, + PostAck: lposta, + } + + c := NewClient(Config{}, + LoggerOption(NoopLogger{}), + WithClientTracerProviderOption(tp), + WithClientTextMapPropagator(p), + WithClientLifecycleHooks(hooks), + KafkaGroupPrefixOption("servicename")) + + require.NotNil(t, c.logger, "WithLogger should set logger successfully") + require.Equal(t, tp, c.tp) + require.Equal(t, p, c.p) + _, err := c.lifecycle.PreProcessing(context.Background(), LifecyclePreProcessingMeta{}) + require.Error(t, err, errors.New("7")) + err = c.lifecycle.PostProcessing(context.Background(), LifecyclePostProcessingMeta{}) + require.Error(t, err, errors.New("8")) + err = c.lifecycle.PostAck(context.Background(), LifecyclePostAckMeta{}) + require.Error(t, err, errors.New("9")) + require.NotEmpty(t, c.groupPrefix, "group prefix should be set") +} + +func TestClient_Reader(t *testing.T) { + type fields struct { + conf Config + readers map[string]*KReader + writers map[string]*KWriter + logger Logger + producerProvider confluentProducerProvider + consumerProvider confluentConsumerProvider + } + type args struct { + ctx context.Context + topicConfig ConsumerTopicConfig + opts []ReaderOption + } + tests := []struct { + name string + fields fields + args args + want *KReader + wantErr bool + }{ + { + name: "create new KReader with overridden Brokers, error from consumer provider", + fields: fields{ + consumerProvider: mockConfluentConsumerProvider{err: true}.NewConsumer, + readers: make(map[string]*KReader), + }, + args: args{ + topicConfig: ConsumerTopicConfig{ + ClientID: "test-id", + GroupID: "group", + Topic: "topic", + BootstrapServers: []string{"remotehost:8080"}, + }, + }, + wantErr: true, + }, + { + name: "create new KReader with bad formatter", + fields: fields{ + consumerProvider: mockConfluentConsumerProvider{err: false}.NewConsumer, + readers: make(map[string]*KReader), + }, + args: args{ + topicConfig: ConsumerTopicConfig{ + ClientID: "test-id", + GroupID: "group", + Topic: "topic", + Formatter: zfmt.FormatterType("nonexistantformatter"), + BootstrapServers: []string{"remotehost:8080"}, + }, + }, + wantErr: true, + }, + { + name: "create new KReader for closed KReader", + fields: fields{ + readers: map[string]*KReader{ + "test-config": {isClosed: true}, + }, + consumerProvider: mockConfluentConsumerProvider{c: MockKafkaConsumer{ID: "stew"}}.NewConsumer, + logger: NoopLogger{}, + }, + args: args{ + ctx: context.TODO(), + topicConfig: ConsumerTopicConfig{ClientID: "test-config", GroupID: "group", Topic: "topic"}, + opts: []ReaderOption{RFormatterOption(&zfmt.AvroFormatter{})}, + }, + want: &KReader{ + consumer: MockKafkaConsumer{ID: "stew"}, + topicConfig: ConsumerTopicConfig{ + ClientID: "test-config", + GroupID: "group", + Topic: "topic", + // some sensible default filled out by the client + Formatter: zfmt.JSONFmt, + ReadTimeoutMillis: ptr(1000), + ProcessTimeoutMillis: ptr(60000), + SessionTimeoutMillis: ptr(61000), + MaxPollIntervalMillis: ptr(61000), + }, + logger: NoopLogger{}, + fmtter: &zfmt.AvroFormatter{}, + }, + wantErr: false, + }, + { + name: "create new KReader for closed KReader with default overrides", + fields: fields{ + readers: map[string]*KReader{ + "test-config": {isClosed: true}, + }, + consumerProvider: mockConfluentConsumerProvider{c: MockKafkaConsumer{ID: "stew"}}.NewConsumer, + logger: NoopLogger{}, + }, + args: args{ + ctx: context.TODO(), + topicConfig: ConsumerTopicConfig{ClientID: "test-config", GroupID: "group", Topic: "topic", ReadTimeoutMillis: ptr(10000), ProcessTimeoutMillis: ptr(10000), SessionTimeoutMillis: ptr(20000), MaxPollIntervalMillis: ptr(21000)}, + opts: []ReaderOption{RFormatterOption(&zfmt.AvroFormatter{})}, + }, + want: &KReader{ + consumer: MockKafkaConsumer{ID: "stew"}, + topicConfig: ConsumerTopicConfig{ + ClientID: "test-config", + GroupID: "group", + Topic: "topic", + // some sensible default filled out by the client + Formatter: zfmt.JSONFmt, + ReadTimeoutMillis: ptr(10000), + ProcessTimeoutMillis: ptr(10000), + SessionTimeoutMillis: ptr(20000), + MaxPollIntervalMillis: ptr(21000), + }, + logger: NoopLogger{}, + fmtter: &zfmt.AvroFormatter{}, + }, + wantErr: false, + }, + + { + name: "invalid configuration should return error", + args: args{ + topicConfig: ConsumerTopicConfig{ClientID: "test"}, + }, + wantErr: true, + }, + { + name: "get from cache", + fields: fields{ + readers: map[string]*KReader{ + "test-config": {}, + }, + }, + args: args{ + topicConfig: ConsumerTopicConfig{ + ClientID: "test-config", + GroupID: "group", + Topic: "topic", + }, + }, + want: &KReader{}, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + c := &Client{ + conf: tt.fields.conf, + readers: tt.fields.readers, + writers: tt.fields.writers, + logger: tt.fields.logger, + + consumerProvider: tt.fields.consumerProvider, + producerProvider: tt.fields.producerProvider, + } + got, err := c.Reader(tt.args.ctx, tt.args.topicConfig, tt.args.opts...) + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + gotReader, ok := got.(*KReader) + if err == nil && ok { + + assertEqual(t, gotReader.topicConfig, tt.want.topicConfig) + a, aOk := tt.want.consumer.(MockKafkaConsumer) + + b, bOk := gotReader.consumer.(MockKafkaConsumer) + if aOk && bOk { + assertEqual(t, a, b, cmpopts.IgnoreUnexported(MockKafkaConsumer{})) + } + assertEqual(t, gotReader.logger, tt.want.logger) + assertEqual(t, gotReader.fmtter, tt.want.fmtter) + } + }) + } +} + +func TestClient_Writer(t *testing.T) { + type fields struct { + conf Config + readers map[string]*KReader + writers map[string]*KWriter + logger Logger + producerProvider confluentProducerProvider + } + type args struct { + ctx context.Context + topicConfig ProducerTopicConfig + opts []WriterOption + } + tests := []struct { + name string + fields fields + args args + want *KWriter + wantErr bool + }{ + { + name: "create new KWriter with overridden Brokers, error from producer provider", + fields: fields{ + producerProvider: mockConfluentProducerProvider{err: true}.NewProducer, + writers: make(map[string]*KWriter), + conf: Config{ + SaslUsername: ptr("test-user"), + SaslPassword: ptr("test-password"), + }, + }, + args: args{ + topicConfig: ProducerTopicConfig{ + ClientID: "test-id", + Topic: "topic", + BootstrapServers: []string{"remotehost:8080"}, + }, + }, + wantErr: true, + }, + { + name: "create new KWriter for closed writer", + fields: fields{ + writers: map[string]*KWriter{ + "test-id": {isClosed: true}, + }, + producerProvider: mockConfluentProducerProvider{}.NewProducer, + logger: NoopLogger{}, + }, + args: args{ + topicConfig: ProducerTopicConfig{ClientID: "test-id", Topic: "topic"}, + opts: []WriterOption{WFormatterOption(&zfmt.ProtobufRawFormatter{})}, + }, + want: &KWriter{ + topicConfig: ProducerTopicConfig{ + ClientID: "test-id", + Topic: "topic", + // some sensible default filled out by the client + Formatter: zfmt.JSONFmt, + NagleDisable: ptr(true), + LingerMillis: 0, + }, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + fmtter: &zfmt.ProtobufRawFormatter{}, + }, + wantErr: false, + }, + { + name: "create new KWriter for closed writer with default overrides", + fields: fields{ + writers: map[string]*KWriter{ + "test-id": {isClosed: true}, + }, + producerProvider: mockConfluentProducerProvider{}.NewProducer, + logger: NoopLogger{}, + }, + args: args{ + topicConfig: ProducerTopicConfig{ClientID: "test-id", Topic: "topic", LingerMillis: 1, NagleDisable: ptr(false)}, + opts: []WriterOption{WFormatterOption(&zfmt.ProtobufRawFormatter{})}, + }, + want: &KWriter{ + topicConfig: ProducerTopicConfig{ + ClientID: "test-id", + Topic: "topic", + // some sensible default filled out by the client + Formatter: zfmt.JSONFmt, + NagleDisable: ptr(false), + LingerMillis: 1, + }, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + fmtter: &zfmt.ProtobufRawFormatter{}, + }, + wantErr: false, + }, + { + name: "invalid configuration should return error", + args: args{ + topicConfig: ProducerTopicConfig{Topic: "topic"}, + }, + wantErr: true, + }, + { + name: "get from cache", + fields: fields{ + writers: map[string]*KWriter{ + "test-id": {}, + }, + }, + args: args{ + topicConfig: ProducerTopicConfig{ClientID: "test-id", Topic: "topic"}, + }, + want: &KWriter{}, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + c := &Client{ + conf: tt.fields.conf, + readers: tt.fields.readers, + writers: tt.fields.writers, + logger: tt.fields.logger, + producerProvider: tt.fields.producerProvider, + } + got, err := c.Writer(tt.args.ctx, tt.args.topicConfig, tt.args.opts...) + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + if got == nil && tt.want == nil { + return + } + gotKWriter := got.(*KWriter) + require.NotNil(t, gotKWriter) + + assertEqual(t, gotKWriter.topicConfig, tt.want.topicConfig) + assertEqual(t, gotKWriter.logger, tt.want.logger) + assertEqual(t, gotKWriter.fmtter, tt.want.fmtter) + }) + } +} + +func TestClient_Close(t *testing.T) { + p := MockKafkaProducer{ID: "d"} + mockConsumer := MockKafkaConsumer{ID: "a", errClose: errors.New("error")} + type fields struct { + Mutex *sync.Mutex + conf Config + readers map[string]*KReader + writers map[string]*KWriter + } + + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r1, err := newReader(Config{}, ConsumerTopicConfig{ + Formatter: zfmt.StringFmt, + }, m, &NoopLogger{}, "") + require.NoError(t, err) + r2, err := newReader(Config{}, ConsumerTopicConfig{ + Formatter: zfmt.StringFmt, + }, m, &NoopLogger{}, "") + require.NoError(t, err) + tests := []struct { + name string + wantErr bool + fields fields + }{ + { + name: "no readers/writers => no error", + fields: fields{}, + }, + { + name: "with readers/writers => no error", + wantErr: true, + fields: fields{ + readers: map[string]*KReader{ + "r1": r1, + "r2": r2, + }, + writers: map[string]*KWriter{ + "w1": {producer: p}, + "w2": {producer: p}, + }, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + c := &Client{ + conf: tt.fields.conf, + readers: tt.fields.readers, + writers: tt.fields.writers, + } + err := c.Close() + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + for _, w := range c.writers { + require.True(t, w.isClosed, "clients writer should be closed") + } + for _, reader := range c.readers { + require.True(t, reader.isClosed, "clients reader should be closed") + } + }) + } +} + +func Test_getFormatter(t *testing.T) { + type args struct { + topicConfig TopicConfig + } + tests := []struct { + name string + args args + want zfmt.Formatter + wantErr bool + }{ + { + name: "unsupported empty", + args: args{topicConfig: ConsumerTopicConfig{}}, + wantErr: true, + }, + { + name: "string", + args: args{topicConfig: ConsumerTopicConfig{Formatter: zfmt.FormatterType("string")}}, + want: &zfmt.StringFormatter{}, + wantErr: false, + }, + { + name: "json", + args: args{topicConfig: ConsumerTopicConfig{Formatter: zfmt.FormatterType("json")}}, + want: &zfmt.JSONFormatter{}, + wantErr: false, + }, + { + name: "protocol buffer", + args: args{topicConfig: ConsumerTopicConfig{Formatter: zfmt.FormatterType("proto_raw")}}, + want: &zfmt.ProtobufRawFormatter{}, + wantErr: false, + }, + { + name: "apache avro", + args: args{topicConfig: ConsumerTopicConfig{Formatter: zfmt.FormatterType("avro")}}, + want: &zfmt.AvroFormatter{}, + wantErr: false, + }, + { + name: "confluent avro with schema ClientID", + args: args{topicConfig: ProducerTopicConfig{Formatter: zfmt.FormatterType("avro_schema")}}, + want: &zfmt.SchematizedAvroFormatter{}, + wantErr: false, + }, + { + name: "confluent avro with inferred schema ClientID", + args: args{topicConfig: ConsumerTopicConfig{Formatter: zfmt.FormatterType("avro_schema"), SchemaID: 10}}, + want: &zfmt.SchematizedAvroFormatter{SchemaID: 10}, + wantErr: false, + }, + { + name: "confluent json with schema ID", + args: args{topicConfig: ConsumerTopicConfig{Formatter: zfmt.FormatterType("json_schema")}}, + want: &zfmt.SchematizedJSONFormatter{}, + wantErr: false, + }, + { + name: "confluent json with inferred schema ID", + args: args{topicConfig: ConsumerTopicConfig{Formatter: zfmt.FormatterType("json_schema"), SchemaID: 10}}, + want: &zfmt.SchematizedJSONFormatter{SchemaID: 10}, + wantErr: false, + }, + { + name: "confluent json with schema ID", + args: args{topicConfig: ConsumerTopicConfig{Formatter: zfmt.FormatterType("proto_schema_deprecated")}}, + want: &zfmt.SchematizedProtoFormatterDeprecated{}, + wantErr: false, + }, + { + name: "confluent json with inferred schema ID", + args: args{topicConfig: ProducerTopicConfig{Formatter: zfmt.FormatterType("proto_schema_deprecated"), SchemaID: 10}}, + want: &zfmt.SchematizedProtoFormatterDeprecated{SchemaID: 10}, + wantErr: false, + }, + { + name: "unsupported", + args: args{topicConfig: ConsumerTopicConfig{Formatter: zfmt.FormatterType("what"), SchemaID: 10}}, + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + got, err := getFormatter(tt.args.topicConfig) + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + require.Equal(t, tt.want, got) + } + }) + } +} + +func TestClient_ConsumerProviderOptionAllowsForInjectionOfCustomConsumer(t *testing.T) { + defer recoverThenFail(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + cfg := Config{} + ccfg := ConsumerTopicConfig{ + ClientID: "test-id", + GroupID: "group", + Topic: "topic", + BootstrapServers: []string{"remotehost:8080"}, + } + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).MaxTimes(1) + mockConsumer.EXPECT().ReadMessage(gomock.Any()).Return(&kafka.Message{ + Key: []byte("hello"), + }, nil).AnyTimes() + mockConsumer.EXPECT().Close().MaxTimes(1) + + c := NewClient(cfg, + WithConsumerProvider(func(config map[string]any) (KafkaConsumer, error) { + return mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer(nil) + }), + ) + + reader, err := c.Reader(context.Background(), ccfg) + require.NoError(t, err) + + msg, err := reader.Read(context.Background()) + require.NoError(t, err) + require.Equal(t, "hello", msg.Key) +} + +type mockConfluentConsumerProvider struct { + c KafkaConsumer + err bool +} + +func (p mockConfluentConsumerProvider) NewConsumer(_ kafka.ConfigMap) (KafkaConsumer, error) { + err := errors.New("fake error") + if !p.err { + err = nil + } + return p.c, err +} + +type mockConfluentProducerProvider struct { + c KafkaProducer + err bool +} + +func (p mockConfluentProducerProvider) NewProducer(_ kafka.ConfigMap) (KafkaProducer, error) { + err := errors.New("fake error") + if !p.err { + err = nil + } + return p.c, err +} + +var _ KafkaConsumer = (*MockKafkaConsumer)(nil) + +type MockKafkaConsumer struct { + ID string + err error + errClose error +} + +func (m MockKafkaConsumer) SubscribeTopics(_ []string, _ kafka.RebalanceCb) error { + return m.err +} + +func (MockKafkaConsumer) Commit() ([]kafka.TopicPartition, error) { + return nil, nil +} + +func (MockKafkaConsumer) StoreOffsets(_ []kafka.TopicPartition) ([]kafka.TopicPartition, error) { + return nil, nil +} + +func (m MockKafkaConsumer) ReadMessage(_ time.Duration) (*kafka.Message, error) { + return nil, nil +} + +func (m MockKafkaConsumer) Close() error { + return m.errClose +} + +func (m MockKafkaConsumer) Assignment() (partitions []kafka.TopicPartition, err error) { + return nil, nil +} + +func (m MockKafkaConsumer) Seek(partition kafka.TopicPartition, timeoutMs int) error { + return nil +} + +func (m MockKafkaConsumer) AssignmentLost() bool { + return false +} + +type MockKafkaProducer struct { + ID string + err error +} + +func (m MockKafkaProducer) Produce(_ *kafka.Message, _ chan kafka.Event) error { + return m.err +} + +func (m MockKafkaProducer) Close() { +} + +func Test_makeConfig_Consumer(t *testing.T) { + type args struct { + conf Config + topicConfig ConsumerTopicConfig + prefix string + } + tests := []struct { + name string + args args + want kafka.ConfigMap + }{ + { + name: "with transaction", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080", "https://localhost:8081"}, + }, + topicConfig: ConsumerTopicConfig{ + ClientID: "clientid", + GroupID: "group", + Topic: "", + Formatter: "", + SchemaID: 0, + Transaction: true, + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080,https://localhost:8081", + "isolation.level": "read_committed", + "enable.auto.offset.store": false, + "enable.auto.commit": true, + "group.id": "group", + "client.id": "clientid", + }, + }, + { + name: "with DeliveryTimeoutMs and AutoCommitIntervalMs", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080", "https://localhost:8081"}, + }, + topicConfig: ConsumerTopicConfig{ + ClientID: "clientid", + GroupID: "group", + AutoCommitIntervalMs: ptr(200), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080,https://localhost:8081", + "enable.auto.commit": true, + "enable.auto.offset.store": false, + "group.id": "group", + "client.id": "clientid", + "auto.commit.interval.ms": 200, + }, + }, + { + name: "with AdditionalProperties", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + }, + topicConfig: ConsumerTopicConfig{ + ClientID: "clientid", + GroupID: "group", + AutoCommitIntervalMs: ptr(200), + AdditionalProps: map[string]any{ + "stewarts.random.property.not.included.in.topicconfig": 123, + }, + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "enable.auto.commit": true, + "enable.auto.offset.store": false, + "group.id": "group", + "client.id": "clientid", + "auto.commit.interval.ms": 200, + "stewarts.random.property.not.included.in.topicconfig": 123, + }, + }, + { + name: "with AdditionalProperties that are floats (if not handled cause errors in confluent go)", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + prefix: "xxxx", + topicConfig: ConsumerTopicConfig{ + ClientID: "clientid", + GroupID: "group", + AdditionalProps: map[string]any{ + "auto.commit.interval.ms": float32(20), + "linger.ms": float64(5), + }, + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "enable.auto.commit": true, + "enable.auto.offset.store": false, + "group.id": "xxxx.group", + "client.id": "clientid", + "auto.commit.interval.ms": 20, + "linger.ms": 5, + "sasl.mechanism": "SCRAM-SHA-256", + "sasl.password": "password", + "sasl.username": "username", + "security.protocol": "SASL_SSL", + }, + }, + { + name: "with sasl override empty", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + prefix: "xxxx", + topicConfig: ConsumerTopicConfig{ + ClientID: "clientid", + GroupID: "group", + SaslUsername: ptr(""), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "enable.auto.commit": true, + "enable.auto.offset.store": false, + "group.id": "xxxx.group", + "client.id": "clientid", + }, + }, + { + name: "with sasl override override name", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + prefix: "xxxx", + topicConfig: ConsumerTopicConfig{ + ClientID: "clientid", + GroupID: "group", + SaslUsername: ptr("usernameOverride"), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "enable.auto.commit": true, + "enable.auto.offset.store": false, + "group.id": "xxxx.group", + "client.id": "clientid", + "sasl.mechanism": "SCRAM-SHA-256", + "sasl.password": "password", + "sasl.username": "usernameOverride", + "security.protocol": "SASL_SSL", + }, + }, + { + name: "with sasl override override pwd", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + prefix: "xxxx", + topicConfig: ConsumerTopicConfig{ + ClientID: "clientid", + GroupID: "group", + SaslPassword: ptr("passwordOverride"), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "enable.auto.commit": true, + "enable.auto.offset.store": false, + "group.id": "xxxx.group", + "client.id": "clientid", + "sasl.mechanism": "SCRAM-SHA-256", + "sasl.password": "passwordOverride", + "sasl.username": "username", + "security.protocol": "SASL_SSL", + }, + }, + { + name: "with sasl override override both", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + prefix: "xxxx", + topicConfig: ConsumerTopicConfig{ + ClientID: "clientid", + GroupID: "group", + SaslUsername: ptr("usernameOverride"), + SaslPassword: ptr("passwordOverride"), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "enable.auto.commit": true, + "enable.auto.offset.store": false, + "group.id": "xxxx.group", + "client.id": "clientid", + "sasl.mechanism": "SCRAM-SHA-256", + "sasl.password": "passwordOverride", + "sasl.username": "usernameOverride", + "security.protocol": "SASL_SSL", + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + got := makeConsumerConfig(tt.args.conf, tt.args.topicConfig, tt.args.prefix) + assertEqual(t, got, tt.want) + }) + } +} + +func Test_makeConfig_Producer(t *testing.T) { + type args struct { + conf Config + topicConfig ProducerTopicConfig + } + tests := []struct { + name string + args args + want kafka.ConfigMap + }{ + { + name: "with transaction", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080", "https://localhost:8081"}, + }, + topicConfig: ProducerTopicConfig{ + ClientID: "clientid", + Topic: "", + Formatter: "", + SchemaID: 0, + Transaction: true, + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080,https://localhost:8081", + "enable.idempotence": true, + "request.required.acks": -1, + "max.in.flight.requests.per.connection": 1, + "client.id": "clientid", + "linger.ms": 0, + }, + }, + { + name: "with DeliveryTimeoutMs and AutoCommitIntervalMs", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080", "https://localhost:8081"}, + }, + topicConfig: ProducerTopicConfig{ + ClientID: "clientid", + DeliveryTimeoutMs: ptr(100), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080,https://localhost:8081", + "client.id": "clientid", + "delivery.timeout.ms": 100, + "enable.idempotence": true, + "linger.ms": 0, + }, + }, + { + name: "with AdditionalProperties", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + }, + topicConfig: ProducerTopicConfig{ + ClientID: "clientid", + DeliveryTimeoutMs: ptr(100), + AdditionalProps: map[string]any{ + "stewarts.random.property.not.included.in.topicconfig": 123, + }, + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "enable.idempotence": true, + "client.id": "clientid", + "delivery.timeout.ms": 100, + "stewarts.random.property.not.included.in.topicconfig": 123, + "linger.ms": 0, + }, + }, + { + name: "with AdditionalProperties that are floats (if not handled cause errors in confluent go)", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + topicConfig: ProducerTopicConfig{ + ClientID: "clientid", + DeliveryTimeoutMs: ptr(100), + EnableIdempotence: ptr(false), + RequestRequiredAcks: ptr("all"), + AdditionalProps: map[string]any{ + "auto.commit.interval.ms": float32(20), + "linger.ms": float64(5), + }, + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "client.id": "clientid", + "enable.idempotence": false, + "delivery.timeout.ms": 100, + "auto.commit.interval.ms": 20, + "linger.ms": 5, + "request.required.acks": "all", + "sasl.mechanism": "SCRAM-SHA-256", + "sasl.password": "password", + "sasl.username": "username", + "security.protocol": "SASL_SSL", + }, + }, + { + name: "with sasl override empty", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + topicConfig: ProducerTopicConfig{ + ClientID: "clientid", + SaslUsername: ptr(""), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "client.id": "clientid", + "enable.idempotence": true, + "linger.ms": 0, + }, + }, + { + name: "with sasl override override name", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + topicConfig: ProducerTopicConfig{ + ClientID: "clientid", + SaslUsername: ptr("usernameOverride"), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "client.id": "clientid", + "enable.idempotence": true, + "sasl.mechanism": "SCRAM-SHA-256", + "sasl.password": "password", + "sasl.username": "usernameOverride", + "security.protocol": "SASL_SSL", + "linger.ms": 0, + }, + }, + { + name: "with sasl override override pwd", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + topicConfig: ProducerTopicConfig{ + ClientID: "clientid", + SaslPassword: ptr("passwordOverride"), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "client.id": "clientid", + "enable.idempotence": true, + "sasl.mechanism": "SCRAM-SHA-256", + "sasl.password": "passwordOverride", + "sasl.username": "username", + "security.protocol": "SASL_SSL", + "linger.ms": 0, + }, + }, + { + name: "with sasl override override both", + args: args{ + conf: Config{ + BootstrapServers: []string{"http://localhost:8080"}, + SaslUsername: ptr("username"), + SaslPassword: ptr("password"), + }, + topicConfig: ProducerTopicConfig{ + ClientID: "clientid", + SaslUsername: ptr("usernameOverride"), + SaslPassword: ptr("passwordOverride"), + }, + }, + want: kafka.ConfigMap{ + "bootstrap.servers": "http://localhost:8080", + "client.id": "clientid", + "enable.idempotence": true, + "sasl.mechanism": "SCRAM-SHA-256", + "sasl.password": "passwordOverride", + "sasl.username": "usernameOverride", + "security.protocol": "SASL_SSL", + "linger.ms": 0, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + got := makeProducerConfig(tt.args.conf, tt.args.topicConfig) + assertEqual(t, got, tt.want) + }) + } +} + +func assertEqual(t *testing.T, got, want any, opts ...cmp.Option) { + t.Helper() + if diff := cmp.Diff(got, want, opts...); diff != "" { + diff = fmt.Sprintf("\ngot: -\nwant: +\n%s", diff) + t.Fatal(diff) + } +} + +func recoverThenFail(t *testing.T) { + if r := recover(); r != nil { + fmt.Print(string(debug.Stack())) + t.Fatal(r) + } +} diff --git a/commitmgr.go b/commitmgr.go new file mode 100644 index 0000000..2af24a4 --- /dev/null +++ b/commitmgr.go @@ -0,0 +1,206 @@ +package zkafka + +import ( + "context" + "sync" + "sync/atomic" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" +) + +// topicCommitMgr manages each subscribed topics commit manager. +type topicCommitMgr struct { + mtx *sync.RWMutex + topicToCommitMgr map[string]*commitMgr +} + +func newTopicCommitMgr() *topicCommitMgr { + return &topicCommitMgr{ + mtx: &sync.RWMutex{}, + topicToCommitMgr: map[string]*commitMgr{}} +} + +// get returns a topics commit manager in a thread safe way +func (t *topicCommitMgr) get(topicName string) *commitMgr { + t.mtx.RLock() + c, found := t.topicToCommitMgr[topicName] + t.mtx.RUnlock() + if !found { + t.mtx.Lock() + defer t.mtx.Unlock() + c, found = t.topicToCommitMgr[topicName] + if found { + return c + } + c = newCommitMgr() + t.topicToCommitMgr[topicName] = c + } + return c +} + +// commitMgr manages inWork and completed commits. Its main responsibility is determining, through TryPop, the largest completed +// offset that can be safely committed (no outstanding inWork offsets with lower offset numbers). It manages these collections of offsets +// on a per-partition basis. +type commitMgr struct { + // mtx synchronizes access to per partition maps (partitionToMtx, partitionToInWork...) + mtx *sync.RWMutex + // partitionToMtx is a map of per partition managed mutexes. Used to synchronize mutations to offsetHeaps + partitionToMtx map[int32]*sync.Mutex + // partitionToInWork uses partition index as a key and has a heap as a value. It's responsible for allowing quick determination + // of the smallest inwork offset + partitionToInWork map[int32]*offsetHeap + // partitionToCompleted uses partition index as a key and has a heap as a value. It's responsible for allowing quick determination + // of the smallest completed offset + partitionToCompleted map[int32]*offsetHeap + // inWorkCount is a count of inflight work. Incremented when work is pushed to heap. Decremented when work is completed + inWorkCount int64 +} + +func newCommitMgr() *commitMgr { + return &commitMgr{ + mtx: &sync.RWMutex{}, + partitionToMtx: map[int32]*sync.Mutex{}, + partitionToInWork: map[int32]*offsetHeap{}, + partitionToCompleted: map[int32]*offsetHeap{}, + } +} + +// PushInWork pushes an offset to one of the managed inwork heaps. +func (c *commitMgr) PushInWork(tp kafka.TopicPartition) { + m := c.mutex(tp.Partition) + m.Lock() + heap := c.getInWorkHeap(tp.Partition) + heap.Push(tp) + atomic.AddInt64(&c.inWorkCount, 1) + m.Unlock() +} + +// RemoveInWork removes an arbitrary partition from the heap (not necessarily the minimum). +// If the pop is successful (the partition is found) the in work count is decremented +func (c *commitMgr) RemoveInWork(tp kafka.TopicPartition) { + m := c.mutex(tp.Partition) + m.Lock() + heap := c.getInWorkHeap(tp.Partition) + if heap.SeekPop(tp) != nil { + atomic.AddInt64(&c.inWorkCount, -1) + } + m.Unlock() +} + +// PushCompleted pushes an offset to one of the managed completed heaps +func (c *commitMgr) PushCompleted(tp kafka.TopicPartition) { + m := c.mutex(tp.Partition) + m.Lock() + heap := c.getCompletedHeap(tp.Partition) + heap.Push(tp) + atomic.AddInt64(&c.inWorkCount, -1) + m.Unlock() +} + +func (c *commitMgr) InWorkCount() int64 { + return atomic.LoadInt64(&c.inWorkCount) +} + +// TryPop returns the largest shared offset between inWork and completed. If +// none such offset exists nil is returned. +// TryPop is thread safe +func (c *commitMgr) TryPop(_ context.Context, partition int32) *kafka.TopicPartition { + m := c.mutex(partition) + m.Lock() + defer m.Unlock() + var commitOffset *kafka.TopicPartition + for { + inWorkPeek, err := c.peekInWork(partition) + if err != nil { + break + } + completedPeek, err := c.peekCompleted(partition) + if err != nil { + break + } + + if completedPeek.Offset == inWorkPeek.Offset { + _ = c.popCompleted(partition) + _ = c.popInWork(partition) + commitOffset = &completedPeek + } else { + break + } + } + return commitOffset +} + +// mutex returns a per partition mutex used for managing offset heaps at a per partition granularity +func (c *commitMgr) mutex(partition int32) *sync.Mutex { + c.mtx.RLock() + mtx, found := c.partitionToMtx[partition] + c.mtx.RUnlock() + if !found { + c.mtx.Lock() + defer c.mtx.Unlock() + mtx, found = c.partitionToMtx[partition] + if found { + return mtx + } + mtx = &sync.Mutex{} + c.partitionToMtx[partition] = mtx + } + return mtx +} + +// getInWorkHeap returns the in work offsetHeap for a particular partition in a thread safe way +func (c *commitMgr) getInWorkHeap(partition int32) *offsetHeap { + c.mtx.RLock() + h, found := c.partitionToInWork[partition] + c.mtx.RUnlock() + if !found { + c.mtx.Lock() + defer c.mtx.Unlock() + h, found = c.partitionToInWork[partition] + if found { + return h + } + h = &offsetHeap{} + c.partitionToInWork[partition] = h + } + return h +} + +// getCompletedHeap returns the completed offsetHeap for a particular partition in a thread safe way +func (c *commitMgr) getCompletedHeap(partition int32) *offsetHeap { + c.mtx.RLock() + h, found := c.partitionToCompleted[partition] + c.mtx.RUnlock() + if !found { + c.mtx.Lock() + defer c.mtx.Unlock() + h, found = c.partitionToCompleted[partition] + if found { + return h + } + h = &offsetHeap{} + c.partitionToCompleted[partition] = h + } + return h +} + +func (c *commitMgr) popInWork(partition int32) kafka.TopicPartition { + heap := c.getInWorkHeap(partition) + out := heap.Pop() + return out +} + +func (c *commitMgr) popCompleted(partition int32) kafka.TopicPartition { + heap := c.getCompletedHeap(partition) + out := heap.Pop() + return out +} + +func (c *commitMgr) peekInWork(partition int32) (kafka.TopicPartition, error) { + heap := c.getInWorkHeap(partition) + return heap.Peek() +} +func (c *commitMgr) peekCompleted(partition int32) (kafka.TopicPartition, error) { + heap := c.getCompletedHeap(partition) + return heap.Peek() +} diff --git a/commitmgr_test.go b/commitmgr_test.go new file mode 100644 index 0000000..93e3bd3 --- /dev/null +++ b/commitmgr_test.go @@ -0,0 +1,290 @@ +package zkafka + +import ( + "context" + "strconv" + "sync" + "testing" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/stretchr/testify/require" +) + +func Test_commitMgr_GetCommitOffset_ShouldReturnProperCandidateCommitOffsetsAndUpdateInternalDataStructs(t *testing.T) { + defer recoverThenFail(t) + mgr := newCommitMgr() + offset1 := kafka.TopicPartition{ + Partition: 1, + Offset: 1, + } + offset2 := kafka.TopicPartition{ + Partition: 1, + Offset: 2, + } + offset3 := kafka.TopicPartition{ + Partition: 1, + Offset: 3, + } + mgr.PushInWork(offset3) + mgr.PushInWork(offset2) + mgr.PushInWork(offset1) + mgr.PushCompleted(offset3) + got := mgr.TryPop(context.Background(), 1) + require.Nil(t, got, "expected nil since there are smaller inwork messages than the lowest completed msg.") + + mgr.PushCompleted(offset2) + got = mgr.TryPop(context.Background(), 1) + require.Nil(t, got, "expected nil since there are smaller inwork messages than the lowest completed msg.") + + mgr.PushCompleted(offset1) + got = mgr.TryPop(context.Background(), 1) + require.NotNil(t, got, "expected commitOffset result to be the largest shared offset between inwork and completed.") + require.Equal(t, offset3, *got, "expected commitOffset result to be the largest shared offset between inwork and completed.") + + require.Len(t, mgr.partitionToCompleted, 1, "expect only 1 map entry for the single partition touched") + require.Len(t, mgr.partitionToInWork, 1, "expect only 1 map entry for the single partition touched") + + require.Empty(t, mgr.partitionToCompleted[1].data, "expected last get to empty heap") + require.Empty(t, mgr.partitionToInWork[1].data, "expected last get to empty heap") + require.Empty(t, mgr.inWorkCount, "expected inWorkCount to be empty") +} + +func Test_commitMgr_GetCommitOffset_ShouldReturnProperCandidateCommitOffsetsAndUpdateInternalDataStructs_WithLargeHeaps(t *testing.T) { + defer recoverThenFail(t) + mgr := newCommitMgr() + var offsets []kafka.TopicPartition + maxCount := 10000 + + var partition int32 = 1 + for i := 0; i < maxCount; i++ { + offsets = append(offsets, kafka.TopicPartition{ + Partition: partition, + Offset: kafka.Offset(i), + }) + mgr.PushInWork(offsets[i]) + } + for i := maxCount - 1; i > 0; i-- { + offsets = append(offsets, kafka.TopicPartition{ + Partition: partition, + Offset: kafka.Offset(i), + }) + mgr.PushCompleted(offsets[i]) + got := mgr.TryPop(context.Background(), partition) + require.Nil(t, got, "expected nil commit since offset=0 has yet to be committed") + } + + mgr.PushCompleted(kafka.TopicPartition{Partition: partition, Offset: 0}) + expectedLast := kafka.TopicPartition{Partition: partition, Offset: kafka.Offset(maxCount - 1)} + + got := mgr.TryPop(context.Background(), partition) + require.NotNil(t, got, "expected commit message") + require.Equal(t, expectedLast, *got, "expected commit message") + + require.Len(t, mgr.partitionToCompleted, 1, "expect only 1 map entry for the single partition touched") + require.Len(t, mgr.partitionToInWork, 1, "expect only 1 map entry for the single partition touched") + + require.Empty(t, mgr.partitionToCompleted[1].data, "expected last get to empty heap") + require.Empty(t, mgr.partitionToInWork[1].data, "expected last get to empty heap") + require.Empty(t, mgr.inWorkCount, "expected inWorkCount to be empty") +} + +func Test_commitMgr_GetCommitOffset_ShouldHandleMultiplePartitions(t *testing.T) { + defer recoverThenFail(t) + mgr := newCommitMgr() + offset1_1 := kafka.TopicPartition{ + Partition: 1, + Offset: 1, + } + offset3_1 := kafka.TopicPartition{ + Partition: 3, + Offset: 1, + } + offset1_2 := kafka.TopicPartition{ + Partition: 1, + Offset: 2, + } + mgr.PushInWork(offset1_1) + mgr.PushInWork(offset3_1) + mgr.PushInWork(offset1_2) + mgr.PushCompleted(offset1_2) + got := mgr.TryPop(context.Background(), 1) + require.Nil(t, got, "expected nil since there are smaller inwork messages than the lowest completed msg for this partition") + + mgr.PushCompleted(offset3_1) + got = mgr.TryPop(context.Background(), 3) + require.NotNil(t, got) + require.Equal(t, offset3_1, *got) + + mgr.PushCompleted(offset1_1) + got = mgr.TryPop(context.Background(), 1) + require.Equal(t, offset1_2, *got, "expected commitOffset result to be the largest shared offset between inwork and completed") + + require.Len(t, mgr.partitionToCompleted, 2, "expect only 2 map entry for the single partition touched") + require.Len(t, mgr.partitionToInWork, 2, "expect only 2 map entry for the single partition touched") + + require.Len(t, mgr.partitionToInWork, 2, "expect only 2 map entry for the single partition touched") + require.Empty(t, mgr.partitionToCompleted[1].data, 2, "expected last get to empty heap") + require.Empty(t, mgr.partitionToInWork[1].data, 2, "expected last get to empty heap") + require.Empty(t, mgr.partitionToCompleted[3].data, 2, "expected last get to empty heap") + require.Empty(t, mgr.partitionToInWork[3].data, 2, "expected last get to empty heap") + require.Equal(t, int64(0), mgr.inWorkCount, "expected inWorkCount to be empty") +} + +// Test_commitMgr_RemoveInWork_CanBeUsedToUpdateInWorkInCommitManager +// RemoveInWork is used in clean up situations. This test shows that offsets that +// are added as inwork can be removed because they are "completed" (typical usage) +// or can be "removed" explicitly (`RemoveInWork` call). +// +// Assertions are made to the inworkcount as well as by using `TryPop` which should only return offsets +// that have had predecessor contiguously committed. +func Test_commitMgr_RemoveInWork_CanBeUsedToUpdateInWorkInCommitManager(t *testing.T) { + defer recoverThenFail(t) + mgr := newCommitMgr() + offset1_1 := kafka.TopicPartition{ + Partition: 1, + Offset: 1, + } + offset1_2 := kafka.TopicPartition{ + Partition: 1, + Offset: 2, + } + offset1_3 := kafka.TopicPartition{ + Partition: 1, + Offset: 3, + } + mgr.PushInWork(offset1_1) + mgr.PushInWork(offset1_2) + + require.Equal(t, mgr.InWorkCount(), int64(2)) + + got := mgr.TryPop(context.Background(), 1) + require.Nil(t, got, "expected nil since offsets are in work but not completed") + + mgr.PushCompleted(offset1_1) + got = mgr.TryPop(context.Background(), 1) + require.NotNil(t, got) + require.Equal(t, offset1_1, *got) + require.Equal(t, mgr.InWorkCount(), int64(1)) + + mgr.RemoveInWork(offset1_3) + require.Equal(t, mgr.InWorkCount(), int64(1), "We attempted to remove an offset that wasn't in work. This shouldn't happen, but we show that its non impacting on data structure") + + mgr.RemoveInWork(offset1_2) + require.Equal(t, mgr.InWorkCount(), int64(0), "Expect in work count to drop since offset was added as 'inwork' and then removed") + + got = mgr.TryPop(context.Background(), 1) + require.Nil(t, got, "Expected nil since all work has been completed or removed") +} + +func Test_commitMgr_PerPartitionDataStructuresBuiltUpConcurrentlyCorrectly(t *testing.T) { + defer recoverThenFail(t) + mgr := newCommitMgr() + + wg := sync.WaitGroup{} + partitionCount := 1000 + for i := 0; i < partitionCount; i++ { + wg.Add(1) + go func(partition int) { + mgr.PushInWork(kafka.TopicPartition{Partition: int32(partition)}) + mgr.PushCompleted(kafka.TopicPartition{Partition: int32(partition)}) + wg.Done() + }(i) + } + wg.Wait() + require.Len(t, mgr.partitionToMtx, partitionCount, "expected partitionToMtx to be have one per visited partition") + require.Len(t, mgr.partitionToInWork, partitionCount, "expected partitionToInWork to be have one per visited partition") + require.Len(t, mgr.partitionToCompleted, partitionCount, "expected partitionToCompleted to be have one per visited partition") + require.Equal(t, int64(0), mgr.inWorkCount, "expected inWorkCount to be empty") +} + +// Test_commitMgr_mutex_ShouldReturnReferenceToSameMutexForSamePartition tests for a race condition (based on bugfound) +// where two goroutines calling this method received distinct mutexes (which isn't correct for syncronization purposes). +// Try a large amount of times to access the mutex for a particular partition. Always should return same pointer +func Test_commitMgr_mutex_ShouldReturnReferenceToSameMutexForSamePartition(t *testing.T) { + defer recoverThenFail(t) + mgr := newCommitMgr() + mgr.mutex(0) + for i := 0; i < 10000; i++ { + wg := sync.WaitGroup{} + wg.Add(2) + var m1 *sync.Mutex + var m2 *sync.Mutex + go func() { + m1 = mgr.mutex(int32(i)) + wg.Done() + }() + go func() { + m2 = mgr.mutex(int32(i)) + wg.Done() + }() + wg.Wait() + require.Same(t, m1, m2, "mutex pointers should be shared for same partition") + } +} + +func Test_commitMgr_mutex_ShouldReturnReferenceToSameCompletedHeapForSamePartition(t *testing.T) { + defer recoverThenFail(t) + mgr := newCommitMgr() + mgr.mutex(0) + for i := 0; i < 10000; i++ { + wg := sync.WaitGroup{} + wg.Add(2) + var h1 *offsetHeap + var h2 *offsetHeap + go func() { + h1 = mgr.getCompletedHeap(int32(i)) + wg.Done() + }() + go func() { + h2 = mgr.getCompletedHeap(int32(i)) + wg.Done() + }() + wg.Wait() + require.Same(t, h1, h2, "mutex pointers should be shared for same partition") + } +} + +func Test_commitMgr_mutex_ShouldReturnReferenceToSameInWorkHeapForSamePartition(t *testing.T) { + defer recoverThenFail(t) + mgr := newCommitMgr() + mgr.mutex(0) + for i := 0; i < 10000; i++ { + wg := sync.WaitGroup{} + wg.Add(2) + var h1 *offsetHeap + var h2 *offsetHeap + go func() { + h1 = mgr.getInWorkHeap(int32(i)) + wg.Done() + }() + go func() { + h2 = mgr.getInWorkHeap(int32(i)) + wg.Done() + }() + wg.Wait() + require.Same(t, h1, h2, "mutex pointers should be shared for same partition") + } +} + +func Test_topicCommitMgr_mutex_ShouldReturnReferenceToSameCommitMgrForSameTopic(t *testing.T) { + defer recoverThenFail(t) + mgr := newTopicCommitMgr() + loopCount := 10000 + for i := 0; i < loopCount; i++ { + wg := sync.WaitGroup{} + wg.Add(2) + var h1 *commitMgr + var h2 *commitMgr + go func() { + h1 = mgr.get(strconv.Itoa(i)) + wg.Done() + }() + go func() { + h2 = mgr.get(strconv.Itoa(i)) + wg.Done() + }() + wg.Wait() + require.Same(t, h1, h2, "mutex pointers should be shared for same partition") + } + require.Len(t, mgr.topicToCommitMgr, loopCount, "expected managed topics under topicCommitManager to equal number of loops") +} diff --git a/config.go b/config.go new file mode 100644 index 0000000..a87844a --- /dev/null +++ b/config.go @@ -0,0 +1,421 @@ +package zkafka + +import ( + "errors" + "fmt" + "strings" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +const ( + // librdkafka configuration keys. For full definitions visit https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md + clientID = "client.id" + groupID = "group.id" + isolationLevel = "isolation.level" + enableIdempotence = "enable.idempotence" + bootstrapServers = "bootstrap.servers" + saslUsername = "sasl.username" + saslPassword = "sasl.password" + saslMechanism = "sasl.mechanism" + securityProtocol = "security.protocol" + requestRequiredAcks = "request.required.acks" + maxInFlightRequestsPerConnection = "max.in.flight.requests.per.connection" + deliveryTimeoutMs = "delivery.timeout.ms" + enableAutoCommit = "enable.auto.commit" + autoCommitIntervalMs = "auto.commit.interval.ms" + enableAutoOffsetStore = "enable.auto.offset.store" + maxPollIntervalMs = "max.poll.interval.ms" + sessionTimeoutMs = "session.timeout.ms" + lingerMs = "linger.ms" + socketNagleDisable = "socket.nagle.disable" +) + +// Config holds configuration to create underlying kafka client +type Config struct { + // BootstrapServers is a list of broker addresses + BootstrapServers []string + + // SaslUsername and SaslPassword for accessing Kafka Cluster + SaslUsername *string + SaslPassword *string + + // CAFile, KeyFile, CertFile are used to enable TLS with valid configuration + // If not defined, TLS with InsecureSkipVerify=false is used. + CAFile string + KeyFile string + CertFile string +} + +// ConsumerTopicConfig holds configuration to create reader for a kafka topic +type ConsumerTopicConfig struct { + // ClientID is required and should be unique. This is used as a cache key for the client + ClientID string + + // GroupID is required for observability per ZG Kafka Best Practices + // http://analytics.pages.zgtools.net/data-engineering/data-infra/streamz/docs/#/guides/kafka-guidelines?id=observability + // The convention is [team_name]/[service]/[group], e.g. concierge/search/index-reader + GroupID string + + // Topic is the name of the topic to be consumed. At least one should be specified between the Topic and Topics attributes + Topic string + + // Topics are the names of the topics to be consumed. At least one should be specified between the Topic and Topics attributes + Topics []string + + // BootstrapServers are the addresses of the possible brokers to be connected to. + // If not defined, Reader and Writer will attempt to use the brokers defined by the client + BootstrapServers []string + + // AutoCommitIntervalMs is a setting which indicates how often offsets will be committed to the kafka broker. + AutoCommitIntervalMs *int + + // AdditionalProps is defined as an escape hatch to specify properties not specified as strongly typed fields. + // The values here will be overwritten by the values of TopicConfig fields if specified there as well. + AdditionalProps map[string]interface{} + + // Formatter is json if not defined + Formatter zfmt.FormatterType + + // SchemaID defines the schema registered with Confluent Schema Registry + // Default value is 0, and it implies that both Writer and Reader do not care about schema validation + // and should encode/decode the message based on data type provided. + // Currently, this only works with SchematizedAvroFormatter + SchemaID int + + // Enable kafka transaction, default to false + Transaction bool + + // ReadTimeoutMillis specifies how much time, in milliseconds, before a kafka read times out (and error is returned) + ReadTimeoutMillis *int + + // ProcessTimeoutMillis specifies how much time, in milliseconds, + // is given to process a particular message before cancellation is calls. + // Default to 1 minute + ProcessTimeoutMillis *int + + // SessionTimeoutMillis specifies how much time, in milliseconds, + // is given by the broker, where in the absence of a heartbeat being successfully received from the consumer + // group member, the member is considered failed (and a rebalance is initiated). + // Defaults to 1 minute 1 second (just over default `ProcessTimeoutMillis`) + SessionTimeoutMillis *int + + // MaxPollIntervalMillis specifies how much time, in milliseconds, + // is given by the broker, where in the absence of `Read`/`Poll` being called by a consumer, the member is considered failed (and a rebalance is initiated). + // Defaults to 1 minute 1 second (just over default `ProcessTimeoutMillis`) + MaxPollIntervalMillis *int + + // ProcessDelayMillis specifies how much time, in milliseconds, + // a virtual partition processor should pause prior to calling processor. + // The specified duration represents the maximum pause a processor will execute. The virtual partition processor + // uses the message's timestamp and its local estimate of `now` to determine + // the observed delay. If the observed delay is less than the amount configured here, + // an additional pause is executed. + ProcessDelayMillis *int + + // SaslUsername and SaslPassword for accessing Kafka Cluster + SaslUsername *string + SaslPassword *string + + // DeadLetterTopicConfig allows you to specify a topic for which to write messages which failed during processing to + DeadLetterTopicConfig *ProducerTopicConfig +} + +func (p ConsumerTopicConfig) GetFormatter() zfmt.FormatterType { + return p.Formatter +} + +func (p ConsumerTopicConfig) GetSchemaID() int { + return p.SchemaID +} + +// topics returns a logical slice of the topics specified in the configuration, +// a combination of the singular Topic and enumerable Topics. It removes any empty topicNames +func (p ConsumerTopicConfig) topics() []string { + topics := make([]string, 0, len(p.Topics)+1) + if p.Topic != "" { + topics = append(topics, p.Topic) + } + for _, t := range p.Topics { + if t == "" { + continue + } + topics = append(topics, t) + } + return topics +} + +// ProducerTopicConfig holds configuration to create writer to kafka topic +type ProducerTopicConfig struct { + // ClientID is required and should be unique. This is used as a cache key for the client + ClientID string + + // Topic is required + Topic string + + // BootstrapServers are the addresses of the possible brokers to be connected to. + // If not defined, Reader and Writer will attempt to use the brokers defined by the client + BootstrapServers []string + + // DeliveryTimeoutMs is a librdkafka setting. Local message timeout. + // This value is only enforced locally and limits the time a produced message waits for successful delivery. + // A time of 0 is infinite. This is the maximum time librdkafka may use to deliver a message (including retries). + // Delivery error occurs when either the retry count or the message timeout are exceeded. + // See defaults in librdkafka configuration + DeliveryTimeoutMs *int + + // AdditionalProps is defined as an escape hatch to specify properties not specified as strongly typed fields. + // The values here will be overwritten by the values of TopicConfig fields if specified there as well. + AdditionalProps map[string]interface{} + + // Formatter is json if not defined + Formatter zfmt.FormatterType + + // SchemaID defines the schema registered with Confluent Schema Registry + // Default value is 0, and it implies that both Writer and Reader do not care about schema validation + // and should encode/decode the message based on data type provided. + // Currently, this only works with SchematizedAvroFormatter + SchemaID int + + // Enable kafka transaction, default to false + Transaction bool + + // RequestRequiredAcks indicates the number of acknowledgments the leader broker must receieve from In Sync Replica (ISR) brokers before responding + // to the request (0=Broker does not send any response to client, -1 or all=broker blocks until all ISRs commit) + RequestRequiredAcks *string + + // EnableIdempotence When set to true, the producer will ensure that messages are successfully produced exactly once and in the original produce order. + // Default to true + EnableIdempotence *bool + + // LingerMillis specifies the delay, in milliseconds, to wait for messages in the producer to accumulate before constructing message batches. + LingerMillis int + + NagleDisable *bool + + // SaslUsername and SaslPassword for accessing Kafka Cluster + SaslUsername *string + SaslPassword *string +} + +func (p ProducerTopicConfig) GetFormatter() zfmt.FormatterType { + return p.Formatter +} + +func (p ProducerTopicConfig) GetSchemaID() int { + return p.SchemaID +} + +type TopicConfig interface { + GetFormatter() zfmt.FormatterType + GetSchemaID() int +} + +func getDefaultConsumerTopicConfig(topicConfig *ConsumerTopicConfig) error { + if topicConfig.ClientID == "" { + return errors.New("invalid config, ClientID must not be empty") + } + if topicConfig.GroupID == "" { + return errors.New("invalid config, group name cannot be empty") + } + if len(topicConfig.topics()) == 0 { + return errors.New("invalid config, missing topic name") + } + + if string(topicConfig.Formatter) == "" { + // default to json formatter + topicConfig.Formatter = zfmt.JSONFmt + } + + const defaultProcessTimeoutMillis = 60 * 1000 + if topicConfig.ProcessTimeoutMillis == nil || *topicConfig.ProcessTimeoutMillis == 0 { + topicConfig.ProcessTimeoutMillis = ptr(defaultProcessTimeoutMillis) + } + const defaultSessionTimeoutMillis = 61 * 1000 + if topicConfig.SessionTimeoutMillis == nil || *topicConfig.SessionTimeoutMillis <= 0 { + topicConfig.SessionTimeoutMillis = ptr(defaultSessionTimeoutMillis) + } + const defaultMaxPollTimeoutMillis = 61 * 1000 + if topicConfig.MaxPollIntervalMillis == nil || *topicConfig.MaxPollIntervalMillis <= 0 { + topicConfig.MaxPollIntervalMillis = ptr(defaultMaxPollTimeoutMillis) + } + + var defaultReadTimeoutMillis = 1000 + if topicConfig.ReadTimeoutMillis == nil || *topicConfig.ReadTimeoutMillis <= 0 { + topicConfig.ReadTimeoutMillis = &defaultReadTimeoutMillis + } + + return nil +} + +func getDefaultProducerTopicConfig(topicConfig *ProducerTopicConfig) error { + if topicConfig.ClientID == "" { + return errors.New("invalid config, ClientID must not be empty") + } + if topicConfig.Topic == "" { + return errors.New("invalid config, missing topic name") + } + if topicConfig.NagleDisable == nil { + topicConfig.NagleDisable = ptr(true) + } + + if string(topicConfig.Formatter) == "" { + // default to json formatter + topicConfig.Formatter = zfmt.JSONFmt + } + + return nil +} + +// makeConsumerConfig creates a kafka configMap from the specified strongly typed Config and TopicConfig. +// TopicConfig specifies a way to specify config values that aren't strongly typed via AdditionalProps field. +// Those values are overwritten if specified in strongly typed TopicConfig fields. +func makeConsumerConfig(conf Config, topicConfig ConsumerTopicConfig, prefix string) kafka.ConfigMap { + configMap := kafka.ConfigMap{} + + configMap[clientID] = topicConfig.ClientID + grp := topicConfig.GroupID + if prefix != "" { + grp = fmt.Sprintf("%s.%s", prefix, grp) + } + configMap[groupID] = grp + configMap[enableAutoCommit] = true + + if topicConfig.MaxPollIntervalMillis != nil { + configMap[maxPollIntervalMs] = *topicConfig.MaxPollIntervalMillis + } + + if topicConfig.SessionTimeoutMillis != nil { + configMap[sessionTimeoutMs] = *topicConfig.SessionTimeoutMillis + } + + // per https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md#auto-offset-commit we can control which offsets + // are eligible for commit by storing this to false and calling rd_kafka_offsets_store ourselves (via StoreOffsets) + configMap[enableAutoOffsetStore] = false + + if topicConfig.AutoCommitIntervalMs != nil { + configMap[autoCommitIntervalMs] = *topicConfig.AutoCommitIntervalMs + } + + // specific settings to enable transaction API (this is actually a poor man's transaction, you still need some library help + // which isn't currently implemented in lib) + if topicConfig.Transaction { + configMap[isolationLevel] = "read_committed" + } + + // overwrite BootstrapServers if defined per topic config + addresses := conf.BootstrapServers + if len(topicConfig.BootstrapServers) != 0 { + addresses = topicConfig.BootstrapServers + } + configMap[bootstrapServers] = strings.Join(addresses, ",") + + saslUname := conf.SaslUsername + if topicConfig.SaslUsername != nil { + saslUname = topicConfig.SaslUsername + } + + saslPwd := conf.SaslPassword + if topicConfig.SaslPassword != nil { + saslPwd = topicConfig.SaslPassword + } + + if saslUname != nil && saslPwd != nil && len(*saslUname) > 0 && len(*saslPwd) > 0 { + configMap[securityProtocol] = "SASL_SSL" + configMap[saslMechanism] = "SCRAM-SHA-256" + configMap[saslUsername] = *saslUname + configMap[saslPassword] = *saslPwd + } + + for key, val := range topicConfig.AdditionalProps { + // confluent-kafka-go does some limited type checking and doesn't allow for floats64. + // We'll convert these to int and store them in the map + switch v := val.(type) { + case float64: + configMap[key] = kafka.ConfigValue(int(v)) + case float32: + configMap[key] = kafka.ConfigValue(int(v)) + default: + configMap[key] = kafka.ConfigValue(v) + } + } + return configMap +} + +// makeProducerConfig creates a kafka configMap from the specified strongly typed Config and TopicConfig. +// TopicConfig specifies a way to specify config values that aren't strongly typed via AdditionalProps field. +// Those values are overwritten if specified in strongly typed TopicConfig fields. +func makeProducerConfig(conf Config, topicConfig ProducerTopicConfig) kafka.ConfigMap { + configMap := kafka.ConfigMap{} + + configMap[clientID] = topicConfig.ClientID + + if topicConfig.RequestRequiredAcks != nil { + configMap[requestRequiredAcks] = *topicConfig.RequestRequiredAcks + } + configMap[enableIdempotence] = true + if topicConfig.EnableIdempotence != nil { + configMap[enableIdempotence] = *topicConfig.EnableIdempotence + } + + if topicConfig.DeliveryTimeoutMs != nil { + configMap[deliveryTimeoutMs] = *topicConfig.DeliveryTimeoutMs + } + + if topicConfig.LingerMillis >= 0 { + configMap[lingerMs] = topicConfig.LingerMillis + } + + if topicConfig.NagleDisable != nil { + configMap[socketNagleDisable] = *topicConfig.NagleDisable + } + + // specific settings to enable transaction API (this is actually a poor man's transaction, you still need some library help + // which isn't currently implemented in lib) + if topicConfig.Transaction { + configMap[enableIdempotence] = true + + configMap[requestRequiredAcks] = -1 + + configMap[maxInFlightRequestsPerConnection] = 1 + } + + // overwrite BootstrapServers if defined per topic config + addresses := conf.BootstrapServers + if len(topicConfig.BootstrapServers) != 0 { + addresses = topicConfig.BootstrapServers + } + configMap[bootstrapServers] = strings.Join(addresses, ",") + + saslUname := conf.SaslUsername + if topicConfig.SaslUsername != nil { + saslUname = topicConfig.SaslUsername + } + + saslPwd := conf.SaslPassword + if topicConfig.SaslPassword != nil { + saslPwd = topicConfig.SaslPassword + } + + if saslUname != nil && saslPwd != nil && len(*saslUname) > 0 && len(*saslPwd) > 0 { + configMap[securityProtocol] = "SASL_SSL" + configMap[saslMechanism] = "SCRAM-SHA-256" + configMap[saslUsername] = *saslUname + configMap[saslPassword] = *saslPwd + } + + for key, val := range topicConfig.AdditionalProps { + // confluent-kafka-go does some limited type checking and doesn't allow for floats64. + // We'll convert these to int and store them in the map + switch v := val.(type) { + case float64: + configMap[key] = kafka.ConfigValue(int(v)) + case float32: + configMap[key] = kafka.ConfigValue(int(v)) + default: + configMap[key] = kafka.ConfigValue(v) + } + } + return configMap +} diff --git a/config_test.go b/config_test.go new file mode 100644 index 0000000..bfaf680 --- /dev/null +++ b/config_test.go @@ -0,0 +1,290 @@ +package zkafka + +import ( + "testing" + + "github.com/google/go-cmp/cmp/cmpopts" + "github.com/stretchr/testify/require" + + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +func Test_getDefaultConsumerTopicConfig(t *testing.T) { + type args struct { + conf *ConsumerTopicConfig + } + tests := []struct { + name string + args args + wantErr bool + expectedTopics []string + }{ + { + name: "missing required field (GroupID) => error", + args: args{conf: &ConsumerTopicConfig{ + Topic: "test_topic", + ClientID: "test", + }}, + wantErr: true, + }, + { + name: "missing required field (Topic) => error", + args: args{conf: &ConsumerTopicConfig{ + GroupID: "test_group", + ClientID: "test", + }}, + wantErr: true, + }, + { + name: "missing required non empty fields (Topic and or Topics) => error", + args: args{conf: &ConsumerTopicConfig{ + GroupID: "test_group", + ClientID: "test", + Topics: []string{"", "", ""}, + }}, + wantErr: true, + }, + { + name: "missing required field (ClientID) => error", + args: args{conf: &ConsumerTopicConfig{ + GroupID: "test_group", + Topic: "test_topic", + }}, + wantErr: true, + }, + { + name: "has minimum required fields => no error", + args: args{conf: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + }}, + wantErr: false, + expectedTopics: []string{"test_topic"}, + }, + { + name: "has minimum required fields (with multitopic subscription) => no error", + args: args{conf: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "hello", + Topics: []string{"test_topic", "tt"}, + }}, + wantErr: false, + expectedTopics: []string{"hello", "test_topic", "tt"}, + }, + { + name: "has minimum required fields (with a 0 process timeout) => no error", + args: args{conf: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + ReadTimeoutMillis: ptr(0), + }}, + wantErr: false, + expectedTopics: []string{"test_topic"}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + err := getDefaultConsumerTopicConfig(tt.args.conf) + if tt.wantErr { + require.Error(t, err) + return + } else { + require.NoError(t, err) + } + var defaultReadTimeoutMillis = 1000 + var defaultProcessTimeoutMillis = 60000 + var defaultSessionTimeoutMillis = 61000 + var defaultMaxPollIntervalMillis = 61000 + expectedConfig := &ConsumerTopicConfig{ + ClientID: tt.args.conf.ClientID, + GroupID: tt.args.conf.GroupID, + Formatter: zfmt.JSONFmt, + ReadTimeoutMillis: &defaultReadTimeoutMillis, + ProcessTimeoutMillis: &defaultProcessTimeoutMillis, + SessionTimeoutMillis: &defaultSessionTimeoutMillis, + MaxPollIntervalMillis: &defaultMaxPollIntervalMillis, + } + ignoreOpts := cmpopts.IgnoreFields(ConsumerTopicConfig{}, "Topic", "Topics") + assertEqual(t, tt.args.conf, expectedConfig, ignoreOpts) + assertEqual(t, tt.args.conf.topics(), tt.expectedTopics) + }) + } +} + +func Test_getDefaultProducerTopicConfig(t *testing.T) { + type args struct { + conf *ProducerTopicConfig + } + tests := []struct { + name string + args args + wantErr bool + }{ + { + name: "missing required field (Topic) => error", + args: args{conf: &ProducerTopicConfig{ + ClientID: "test", + }}, + wantErr: true, + }, + { + name: "has minimum required fields => no error", + args: args{conf: &ProducerTopicConfig{ + ClientID: "test", + Topic: "test_topic", + }, + }, + wantErr: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + err := getDefaultProducerTopicConfig(tt.args.conf) + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + if tt.wantErr { + return + } + expectedConfig := &ProducerTopicConfig{ + ClientID: tt.args.conf.ClientID, + Topic: tt.args.conf.Topic, + Formatter: zfmt.JSONFmt, + NagleDisable: ptr(true), + } + assertEqual(t, tt.args.conf, expectedConfig) + }) + } +} + +func Test_getDefaultConsumerTopicConfigSpecialCase(t *testing.T) { + type args struct { + conf *ConsumerTopicConfig + } + tests := []struct { + name string + args args + want *ConsumerTopicConfig + }{ + { + name: "has minimum required fields with negative CacheTime => no error (adjusted to default)", + args: args{conf: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + ProcessTimeoutMillis: ptr(60000), + }}, + want: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + Formatter: zfmt.JSONFmt, + ReadTimeoutMillis: ptr(1000), + ProcessTimeoutMillis: ptr(60000), + SessionTimeoutMillis: ptr(61000), + MaxPollIntervalMillis: ptr(61000), + }, + }, + { + name: "has minimum required fields with negative buffer => no error", + args: args{conf: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + }}, + want: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + Formatter: zfmt.JSONFmt, + ReadTimeoutMillis: ptr(1000), + ProcessTimeoutMillis: ptr(60000), + SessionTimeoutMillis: ptr(61000), + MaxPollIntervalMillis: ptr(61000), + }, + }, + { + name: "has minimum required fields with positive ProcessTimeoutMillis", + args: args{conf: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + ReadTimeoutMillis: ptr(100), + AutoCommitIntervalMs: ptr(12000), + ProcessTimeoutMillis: ptr(60000), + SessionTimeoutMillis: ptr(61000), + MaxPollIntervalMillis: ptr(61000), + }}, + want: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + Formatter: zfmt.JSONFmt, + ReadTimeoutMillis: ptr(100), + AutoCommitIntervalMs: ptr(12000), + ProcessTimeoutMillis: ptr(60000), + SessionTimeoutMillis: ptr(61000), + MaxPollIntervalMillis: ptr(61000), + }, + }, + { + name: "has minimum required fields with positive AutoCommitIntervalMs", + args: args{conf: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + AutoCommitIntervalMs: ptr(123), + ProcessTimeoutMillis: ptr(60000), + }}, + want: &ConsumerTopicConfig{ + ClientID: "test", + GroupID: "test_group", + Topic: "test_topic", + Formatter: zfmt.JSONFmt, + ReadTimeoutMillis: ptr(1000), + AutoCommitIntervalMs: ptr(123), + ProcessTimeoutMillis: ptr(60000), + SessionTimeoutMillis: ptr(61000), + MaxPollIntervalMillis: ptr(61000), + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + err := getDefaultConsumerTopicConfig(tt.args.conf) + require.NoError(t, err) + assertEqual(t, tt.args.conf, tt.want) + }) + } +} + +func Test_getDefaultProducerTopicConfigSpecialCase(t *testing.T) { + type args struct { + conf *ProducerTopicConfig + } + var tests []struct { + name string + args args + want *ProducerTopicConfig + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + err := getDefaultProducerTopicConfig(tt.args.conf) + require.NoError(t, err) + assertEqual(t, tt.args.conf, tt.want) + }) + } +} diff --git a/confluent.go b/confluent.go new file mode 100644 index 0000000..a4554bf --- /dev/null +++ b/confluent.go @@ -0,0 +1,23 @@ +package zkafka + +import ( + "github.com/confluentinc/confluent-kafka-go/v2/kafka" +) + +type confluentConsumerProvider func(configMap kafka.ConfigMap) (KafkaConsumer, error) + +type defaultConfluentConsumerProvider struct { +} + +func (p defaultConfluentConsumerProvider) NewConsumer(configMap kafka.ConfigMap) (KafkaConsumer, error) { + return kafka.NewConsumer(&configMap) +} + +type confluentProducerProvider func(configMap kafka.ConfigMap) (KafkaProducer, error) + +type defaultConfluentProducerProvider struct { +} + +func (p defaultConfluentProducerProvider) NewProducer(configMap kafka.ConfigMap) (KafkaProducer, error) { + return kafka.NewProducer(&configMap) +} diff --git a/coption.go b/coption.go new file mode 100644 index 0000000..814e907 --- /dev/null +++ b/coption.go @@ -0,0 +1,84 @@ +package zkafka + +import ( + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/trace" +) + +// Option is a function that modify the client configurations +type Option func(*Client) + +// LoggerOption applies logger to the client and to all writers/readers which are created +// after this call. +func LoggerOption(logger Logger) Option { + return func(c *Client) { + if logger != nil { + c.logger = logger + } + } +} + +// WithClientTracerProviderOption applies an otel tracer provider to the client and to all writers/readers which are created +func WithClientTracerProviderOption(tp trace.TracerProvider) Option { + return func(c *Client) { + if tp != nil { + c.tp = tp + } + } +} + +// WithClientTextMapPropagator applies an otel p to the client and to all writers/readers which are created +func WithClientTextMapPropagator(p propagation.TextMapPropagator) Option { + return func(c *Client) { + c.p = p + } +} + +// KafkaGroupPrefixOption creates a groupPrefix which will be added to all client and producer groupID +// strings if created after this option is added +func KafkaGroupPrefixOption(prefix string) Option { + return func(c *Client) { + c.groupPrefix = prefix + } +} + +func WithClientLifecycleHooks(h LifecycleHooks) Option { + return func(c *Client) { + c.lifecycle = h + } +} + +// WithConsumerProvider allows for the specification of a factory which is responsible for returning +// a KafkaConsumer given a config map. +func WithConsumerProvider(provider func(config map[string]any) (KafkaConsumer, error)) Option { + return func(c *Client) { + c.consumerProvider = func(kConfigMap kafka.ConfigMap) (KafkaConsumer, error) { + if provider == nil { + return nil, nil + } + configMap := map[string]any{} + for k, v := range kConfigMap { + configMap[k] = v + } + return provider(configMap) + } + } +} + +// WithProducerProvider allows for the specification of a factory which is responsible for returning +// a KafkaProducer given a config map. +func WithProducerProvider(provider func(config map[string]any) (KafkaProducer, error)) Option { + return func(c *Client) { + c.producerProvider = func(kConfigMap kafka.ConfigMap) (KafkaProducer, error) { + if provider == nil { + return nil, nil + } + configMap := map[string]any{} + for k, v := range kConfigMap { + configMap[k] = v + } + return provider(configMap) + } + } +} diff --git a/coverage.sh b/coverage.sh new file mode 100755 index 0000000..0934cac --- /dev/null +++ b/coverage.sh @@ -0,0 +1,52 @@ +#!/usr/bin/env bash + +# golang packages that will be used for either testing or will be assessed for coverage +pck1=github.com/zillow/zkafka +pck2=$pck1/test + +topdir=$(pwd) + +# binary result filepaths +root_res=$topdir/root.res +source_res=$topdir/source.res + +# go cover formatted files +root_out=$topdir/root.out +source_out=$topdir/source.out + +omni_out=$topdir/omni.out + +function quit() { + echo "Error in coverage.sh. Stopping processing"; + exit; +} +# change to example directory for execution (because it uses hardcoded filepaths, and the testable +# examples don't work when executed outside of that directory +go test -c -coverpkg=$pck1 -covermode=atomic -o "$root_res" -tags integration $pck1 +# convert binary to go formatted +go tool test2json -t "$root_res" -test.v -test.coverprofile "$root_out" + +go test -c -coverpkg=$pck1 -covermode=atomic -o "$source_res" -tags integration $pck2 +go tool test2json -t "$source_res" -test.v -test.coverprofile "$source_out" + +# delete aggregate file +rm "$omni_out" + +# combine the results (the first line should be omitted on subsequent appends) +cat "$root_out" >> "$omni_out" +tail -n+2 "$source_out" >> "$omni_out" + +# print aggregated results +go tool cover -func="$omni_out" + +# we need to create a cobertura.xml file (this is used for code coverage visualization) +# download the tool to convert gocover into covertura +go install github.com/boumenot/gocover-cobertura@latest +gocover-cobertura < $omni_out > coverage.tmp.xml +# the cobertura generated file has two issues. Its source array doesn't include the curdir and +# the class node filepaths aren't relative to the root. +# We'll run two commands +# 1. Remove the prefixed go.mod package name from filenames inside of the cobertura with a brute force replace with empty string +# 2. Add the workingdirectory to the sources array using a find replace (search for sources node, and replace with sources node but new workdir source nod) +pkg=gitlab.zgtools.net/devex/archetypes/gomods/zkafka +sed "s|$pkg/||" coverage.tmp.xml | sed "s||\n$(pwd)|"> coverage.xml \ No newline at end of file diff --git a/example/consumer/consumer.go b/example/consumer/consumer.go new file mode 100644 index 0000000..53747fe --- /dev/null +++ b/example/consumer/consumer.go @@ -0,0 +1,75 @@ +package main + +import ( + "context" + "fmt" + "log" + + "github.com/zillow/zkafka" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +func main() { + // configure broker connectivity options with zkafka.Config + cfg := zkafka.Config{ + BootstrapServers: []string{"localhost:9093"}, + } + + // configure consumer options with zkafka.ConsumerTopicConfig. See zkafka for full option values + topicConfig := zkafka.ConsumerTopicConfig{ + ClientID: "xxx", + GroupID: "golang-example-consumer-3", + Topic: "two-multi-partition", + // defaults to 1 second. Use 10 seconds for example to give time to establish connection + ReadTimeoutMillis: ptr(10000), + // Specify the formatter used to deserialize the contents of kafka message + Formatter: zfmt.JSONFmt, + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + } + + ctx := context.Background() + // Create a reader using Config and ConsumerTopicConfig + reader, err := zkafka.NewClient(cfg).Reader(ctx, topicConfig) + if err != nil { + log.Fatal(err) + } + for { + // Poll for 1 message for up to 10 seconds. Return nil if no messages available. + // To continually poll a kafka message, we suggest using zkafka in conjunction with zwork. This offers a + // good mechanism for processing messages using stateless messaging semantics using golang + msg, err := reader.Read(ctx) + if err != nil { + log.Fatal(err) + } + if msg == nil { + log.Printf("No messages available") + return + } + + // zkafka.Message (the type of msg variable) wraps a kafka message exposing higher level APIs for interacting with the data. + // This includes a decode method for easily + // deserializing the kafka value byte array. In this case, we're using the JSONFmt (specified in TopicConfig). + item := DummyEvent{} + if err = msg.Decode(&item); err != nil { + log.Fatal(err) + } + + // print out the contents of kafka message used to hydrate the DummyEvent struct + fmt.Printf("dummy event %+v\n", item) + + // call msg.Done to commit the work with the kafka broker + msg.Done() + } +} + +// DummyEvent is a deserializable struct for producing/consuming kafka message values. +type DummyEvent struct { + Name string `json:"name"` + Age int `json:"age"` +} + +func ptr[T any](v T) *T { + return &v +} diff --git a/example/docker-compose.yaml b/example/docker-compose.yaml new file mode 100644 index 0000000..04d2e00 --- /dev/null +++ b/example/docker-compose.yaml @@ -0,0 +1,10 @@ +version: '2.1' + +services: + kafka3: + image: stewartboyd1988/docker-kafka-kraft + container_name: kafka3 + ports: + - "29092:29092" + - "9092:9092" + - "9093:9093" diff --git a/example/producer/producer.go b/example/producer/producer.go new file mode 100644 index 0000000..1196455 --- /dev/null +++ b/example/producer/producer.go @@ -0,0 +1,45 @@ +package main + +import ( + "context" + "log" + "math/rand" + "time" + + "github.com/google/uuid" + "github.com/zillow/zkafka" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +func main() { + ctx := context.Background() + writer, err := zkafka.NewClient(zkafka.Config{ + BootstrapServers: []string{"localhost:9093"}, + }).Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: "example", + Topic: "two-multi-partition", + Formatter: zfmt.JSONFmt, + }) + if err != nil { + log.Panic(err) + } + for { + event := DummyEvent{ + Name: uuid.NewString(), + Age: rand.Intn(100), + } + + resp, err := writer.Write(ctx, &event) + if err != nil { + log.Panic(err) + } + log.Printf("resp: %+v\n", resp) + time.Sleep(time.Second) + } +} + +// DummyEvent is a deserializable struct for producing/consuming kafka message values. +type DummyEvent struct { + Name string `json:"name"` + Age int `json:"age"` +} diff --git a/example/worker/bench/main.go b/example/worker/bench/main.go new file mode 100644 index 0000000..aa1da3c --- /dev/null +++ b/example/worker/bench/main.go @@ -0,0 +1,64 @@ +package main + +import ( + "context" + "errors" + "fmt" + "log" + "os" + "runtime/pprof" + "time" + + "github.com/golang/mock/gomock" + "github.com/zillow/zkafka" + zkafka_mocks "github.com/zillow/zkafka/mocks" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +func main() { + f, err := os.Create("cpu.prof") + if err != nil { + log.Fatal("could not create CPU profile: ", err) + } + defer f.Close() // error handling omitted for example + if err := pprof.StartCPUProfile(f); err != nil { + log.Fatal("could not start CPU profile: ", err) + } + defer pprof.StopCPUProfile() + + ctrl := gomock.NewController(&testReporter{}) + defer ctrl.Finish() + messageDone := func() { + } + msg := zkafka.GetFakeMessage("1", struct{ name string }{name: "arish"}, &zfmt.JSONFormatter{}, messageDone) + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).Return(msg, nil).AnyTimes() + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Return(r, nil).AnyTimes() + + kwf := zkafka.NewWorkFactory(kcp) + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: "topicName"}, + &kafkaProcessorError{}, + zkafka.Speedup(10), + zkafka.CircuitBreakAfter(100), + zkafka.CircuitBreakFor(30*time.Second), + zkafka.DisableBusyLoopBreaker(), + ) + ctx, c := context.WithTimeout(context.Background(), 2*time.Minute) + defer c() + w.Run(ctx, nil) +} + +type kafkaProcessorError struct{} + +func (p *kafkaProcessorError) Process(_ context.Context, _ *zkafka.Message) error { + fmt.Print(".") + return errors.New("an error occurred during processing") +} + +type testReporter struct{} + +func (t *testReporter) Errorf(format string, args ...any) {} +func (t *testReporter) Fatalf(format string, args ...any) {} diff --git a/example/worker/worker.go b/example/worker/worker.go new file mode 100644 index 0000000..6f4c803 --- /dev/null +++ b/example/worker/worker.go @@ -0,0 +1,103 @@ +package main + +import ( + "bytes" + "context" + "log" + "os" + "os/signal" + "syscall" + "time" + + "github.com/zillow/zkafka" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +// Demonstrates reading from a topic via the zkafka.Work struct which is more convenient, typically, than using the consumer directly +func main() { + ctx := context.Background() + client := zkafka.NewClient(zkafka.Config{ + BootstrapServers: []string{"localhost:9093"}, + }, + zkafka.LoggerOption(stdLogger{}), + ) + // It's important to close the client after consumption to gracefully leave the consumer group + // (this commits completed work, and informs the broker that this consumer is leaving the group which yields a faster rebalance) + defer client.Close() + + topicConfig := zkafka.ConsumerTopicConfig{ + // ClientID is used for caching inside zkafka, and observability within streamz dashboards. But it's not an important + // part of consumer group semantics. A typical convention is to use the service name executing the kafka worker + ClientID: "service-name", + // GroupID is the consumer group. If multiple instances of the same consumer group read messages for the same + // topic the topic's partitions will be split between the collection. The broker remembers + // what offset has been committed for a consumer group, and therefore work can be picked up where it was left off + // across releases + GroupID: "concierge/example/example-consumery", + Topic: "two-multi-partition", + // The formatter is registered internally to the `zkafka.Message` and used when calling `msg.Decode()` + // string fmt can be used for both binary and pure strings encoded in the value field of the kafka message. Other options include + // json, proto, avro, etc. + Formatter: zfmt.StringFmt, + AdditionalProps: map[string]any{ + // only important the first time a consumer group connects. Subsequent connections will start + // consuming messages + "auto.offset.reset": "earliest", + }, + } + // optionally set up a channel to signal when worker shutdown should occur. + // A nil channel is also acceptable, but this example demonstrates how to make utility of the signal. + // The channel should be closed, instead of simply written to, to properly broadcast to the separate worker threads. + stopCh := make(chan os.Signal, 1) + signal.Notify(stopCh, os.Interrupt, syscall.SIGTERM) + shutdown := make(chan struct{}) + + go func() { + <-stopCh + close(shutdown) + }() + + wf := zkafka.NewWorkFactory(client) + // Register a processor which is executed per message. + // Speedup is used to create multiple processor goroutines. Order is still maintained with this setup by way of `virtual partitions` + work := wf.Create(topicConfig, &Processor{}, zkafka.Speedup(5)) + work.Run(ctx, shutdown) +} + +type Processor struct{} + +func (p Processor) Process(_ context.Context, msg *zkafka.Message) error { + // sleep to simulate random amount of work + time.Sleep(100 * time.Millisecond) + var buf bytes.Buffer + err := msg.Decode(&buf) + if err != nil { + return err + } + // optionally, if you don't want to use the configured formatter at all, access the kafka message payload bytes directly. + // The commented out block shows accessing the byte array. In this case we're stringifying the bytes, but this could be json unmarshalled, + // proto unmarshalled etc., depending on the expected payload + //data := msg.Value() + //str := string(data) + + log.Printf("message: %s, offset: %d, partition: %d \n", buf.String(), msg.Offset, msg.Partition) + return nil +} + +type stdLogger struct{} + +func (l stdLogger) Debugw(_ context.Context, msg string, keysAndValues ...any) { + log.Printf("Debugw-"+msg, keysAndValues...) +} + +func (l stdLogger) Infow(_ context.Context, msg string, keysAndValues ...any) { + log.Printf("Infow-"+msg, keysAndValues...) +} + +func (l stdLogger) Errorw(_ context.Context, msg string, keysAndValues ...any) { + log.Printf("Errorw-"+msg, keysAndValues...) +} + +func (l stdLogger) Warnw(_ context.Context, msg string, keysAndValues ...any) { + log.Printf("Warnw-"+msg, keysAndValues...) +} diff --git a/formatter.go b/formatter.go new file mode 100644 index 0000000..f236367 --- /dev/null +++ b/formatter.go @@ -0,0 +1,34 @@ +package zkafka + +import ( + "errors" + + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +const ( + // CustomFmt indicates that the user would pass in their own Formatter later + CustomFmt zfmt.FormatterType = "custom" +) + +var errMissingFmtter = errors.New("custom formatter is missing, did you forget to call WithFormatter()") + +// Formatter allows the user to extend formatting capability to unsupported data types +type Formatter interface { + Marshall(v any) ([]byte, error) + Unmarshal(b []byte, v any) error +} + +// noopFormatter is a formatter that returns error when called. The error will remind the user +// to provide appropriate implementation +type noopFormatter struct{} + +// Marshall returns error with reminder +func (f noopFormatter) Marshall(_ any) ([]byte, error) { + return nil, errMissingFmtter +} + +// Unmarshal returns error with reminder +func (f noopFormatter) Unmarshal(_ []byte, _ any) error { + return errMissingFmtter +} diff --git a/formatter_test.go b/formatter_test.go new file mode 100644 index 0000000..d9e6aa8 --- /dev/null +++ b/formatter_test.go @@ -0,0 +1,18 @@ +package zkafka + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestNoopFormatter_Marshall_Unmarshal(t *testing.T) { + defer recoverThenFail(t) + fmtter := noopFormatter{} + _, err := fmtter.Marshall("anything") + require.ErrorIs(t, err, errMissingFmtter) + + var someInt int32 + err = fmtter.Unmarshal([]byte("test"), &someInt) + require.ErrorIs(t, err, errMissingFmtter) +} diff --git a/go.mod b/go.mod new file mode 100644 index 0000000..ef28870 --- /dev/null +++ b/go.mod @@ -0,0 +1,30 @@ +module github.com/zillow/zkafka + +go 1.22 + +require ( + github.com/confluentinc/confluent-kafka-go/v2 v2.4.0 + github.com/golang/mock v1.6.0 + github.com/google/go-cmp v0.6.0 + github.com/google/uuid v1.6.0 + github.com/pkg/errors v0.9.1 + github.com/sony/gobreaker v1.0.0 + github.com/stretchr/testify v1.9.0 + gitlab.zgtools.net/devex/archetypes/gomods/zfmt v1.0.68 + go.opentelemetry.io/otel v1.27.0 + go.opentelemetry.io/otel/trace v1.27.0 + golang.org/x/sync v0.7.0 +) + +require ( + github.com/actgardner/gogen-avro/v10 v10.2.1 // indirect + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/go-logr/logr v1.4.2 // indirect + github.com/go-logr/stdr v1.2.2 // indirect + github.com/golang/protobuf v1.5.4 // indirect + github.com/heetch/avro v0.4.5 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + go.opentelemetry.io/otel/metric v1.27.0 // indirect + google.golang.org/protobuf v1.34.1 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect +) diff --git a/go.sum b/go.sum new file mode 100644 index 0000000..3cf1ac1 --- /dev/null +++ b/go.sum @@ -0,0 +1,420 @@ +dario.cat/mergo v1.0.0 h1:AGCNq9Evsj31mOgNPcLyXc+4PNABt905YmuqPYYpBWk= +dario.cat/mergo v1.0.0/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk= +github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24 h1:bvDV9vkmnHYOMsOr4WLk+Vo07yKIzd94sVoIqshQ4bU= +github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24/go.mod h1:8o94RPi1/7XTJvwPpRSzSUedZrtlirdB3r9Z20bi2f8= +github.com/AlecAivazis/survey/v2 v2.3.7 h1:6I/u8FvytdGsgonrYsVn2t8t4QiRnh6QSTqkkhIiSjQ= +github.com/AlecAivazis/survey/v2 v2.3.7/go.mod h1:xUTIdE4KCOIjsBAE1JYsUPoCqYdZ1reCfTwbto0Fduo= +github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 h1:UQHMgLO+TxOElx5B5HZ4hJQsoJ/PvUvKRhJHDQXO8P8= +github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1/go.mod h1:xomTg63KZ2rFqZQzSB4Vz2SUXa1BpHTVz9L5PTmPC4E= +github.com/Masterminds/semver/v3 v3.2.1 h1:RN9w6+7QoMeJVGyfmbcgs28Br8cvmnucEXnY0rYXWg0= +github.com/Masterminds/semver/v3 v3.2.1/go.mod h1:qvl/7zhW3nngYb5+80sSMF+FG2BjYrf8m9wsX0PNOMQ= +github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= +github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= +github.com/Microsoft/hcsshim v0.11.4 h1:68vKo2VN8DE9AdN4tnkWnmdhqdbpUFM8OF3Airm7fz8= +github.com/Microsoft/hcsshim v0.11.4/go.mod h1:smjE4dvqPX9Zldna+t5FG3rnoHhaB7QYxPRqGcpAD9w= +github.com/actgardner/gogen-avro/v10 v10.2.1 h1:z3pOGblRjAJCYpkIJ8CmbMJdksi4rAhaygw0dyXZ930= +github.com/actgardner/gogen-avro/v10 v10.2.1/go.mod h1:QUhjeHPchheYmMDni/Nx7VB0RsT/ee8YIgGY/xpEQgQ= +github.com/aws/aws-sdk-go-v2 v1.17.6 h1:Y773UK7OBqhzi5VDXMi1zVGsoj+CVHs2eaC2bDsLwi0= +github.com/aws/aws-sdk-go-v2 v1.17.6/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= +github.com/aws/aws-sdk-go-v2/config v1.18.16 h1:4r7gsCu8Ekwl5iJGE/GmspA2UifqySCCkyyyPFeWs3w= +github.com/aws/aws-sdk-go-v2/config v1.18.16/go.mod h1:XjM6lVbq7UgELp9NjXBrb1DQY/ownlWsvDhEQksemJc= +github.com/aws/aws-sdk-go-v2/credentials v1.13.16 h1:GgToSxaENX/1zXIGNFfiVk4hxryYJ5Vt4Mh8XLAL7Lc= +github.com/aws/aws-sdk-go-v2/credentials v1.13.16/go.mod h1:KP7aFJhfwPFgx9aoVYL2nYHjya5WBD98CWaadpgmnpY= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.12.24 h1:5qyqXASrX2zy5cTnoHHa4N2c3Lc94GH7gjnBP3GwKdU= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.12.24/go.mod h1:neYVaeKr5eT7BzwULuG2YbLhzWZ22lpjKdCybR7AXrQ= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.30 h1:y+8n9AGDjikyXoMBTRaHHHSaFEB8267ykmvyPodJfys= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.30/go.mod h1:LUBAO3zNXQjoONBKn/kR1y0Q4cj/D02Ts0uHYjcCQLM= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.24 h1:r+Kv+SEJquhAZXaJ7G4u44cIwXV3f8K+N482NNAzJZA= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.24/go.mod h1:gAuCezX/gob6BSMbItsSlMb6WZGV7K2+fWOvk8xBSto= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.31 h1:hf+Vhp5WtTdcSdE+yEcUz8L73sAzN0R+0jQv+Z51/mI= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.31/go.mod h1:5zUjguZfG5qjhG9/wqmuyHRyUftl2B5Cp6NNxNC6kRA= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.24 h1:c5qGfdbCHav6viBwiyDns3OXqhqAbGjfIB4uVu2ayhk= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.24/go.mod h1:HMA4FZG6fyib+NDo5bpIxX1EhYjrAOveZJY2YR0xrNE= +github.com/aws/aws-sdk-go-v2/service/sso v1.12.5 h1:bdKIX6SVF3nc3xJFw6Nf0igzS6Ff/louGq8Z6VP/3Hs= +github.com/aws/aws-sdk-go-v2/service/sso v1.12.5/go.mod h1:vuWiaDB30M/QTC+lI3Wj6S/zb7tpUK2MSYgy3Guh2L0= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.5 h1:xLPZMyuZ4GuqRCIec/zWuIhRFPXh2UOJdLXBSi64ZWQ= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.5/go.mod h1:QjxpHmCwAg0ESGtPQnLIVp7SedTOBMYy+Slr3IfMKeI= +github.com/aws/aws-sdk-go-v2/service/sts v1.18.6 h1:rIFn5J3yDoeuKCE9sESXqM5POTAhOP1du3bv/qTL+tE= +github.com/aws/aws-sdk-go-v2/service/sts v1.18.6/go.mod h1:48WJ9l3dwP0GSHWGc5sFGGlCkuA82Mc2xnw+T6Q8aDw= +github.com/aws/smithy-go v1.13.5 h1:hgz0X/DX0dGqTYpGALqXJoRKRj5oQ7150i5FdTePzO8= +github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/buger/goterm v1.0.4 h1:Z9YvGmOih81P0FbVtEYTFF6YsSgxSUKEhf/f9bTMXbY= +github.com/buger/goterm v1.0.4/go.mod h1:HiFWV3xnkolgrBV3mY8m0X0Pumt4zg4QhbdOzQtB8tE= +github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= +github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/compose-spec/compose-go/v2 v2.0.0-rc.2 h1:eJ01FpliL/02KvsaPyH1bSLbM1S70yWQUojHVRbyvy4= +github.com/compose-spec/compose-go/v2 v2.0.0-rc.2/go.mod h1:IVsvFyGVhw4FASzUtlWNVaAOhYmakXAFY9IlZ7LAuD8= +github.com/confluentinc/confluent-kafka-go/v2 v2.4.0 h1:NbOku86JJlsRJPJKE0snNsz6D1Qr4j5VR/lticrLZrY= +github.com/confluentinc/confluent-kafka-go/v2 v2.4.0/go.mod h1:E1dEQy50ZLfqs7T9luxz0rLxaeFZJZE92XvApJOr/Rk= +github.com/containerd/console v1.0.3 h1:lIr7SlA5PxZyMV30bDW0MGbiOPXwc63yRuCP0ARubLw= +github.com/containerd/console v1.0.3/go.mod h1:7LqA/THxQ86k76b8c/EMSiaJ3h1eZkMkXar0TQ1gf3U= +github.com/containerd/containerd v1.7.12 h1:+KQsnv4VnzyxWcfO9mlxxELaoztsDEjOuCMPAuPqgU0= +github.com/containerd/containerd v1.7.12/go.mod h1:/5OMpE1p0ylxtEUGY8kuCYkDRzJm9NO1TFMWjUpdevk= +github.com/containerd/continuity v0.4.2 h1:v3y/4Yz5jwnvqPKJJ+7Wf93fyWoCB3F5EclWG023MDM= +github.com/containerd/continuity v0.4.2/go.mod h1:F6PTNCKepoxEaXLQp3wDAjygEnImnZ/7o4JzpodfroQ= +github.com/containerd/log v0.1.0 h1:TCJt7ioM2cr/tfR8GPbGf9/VRAX8D2B4PjzCpfX540I= +github.com/containerd/log v0.1.0/go.mod h1:VRRf09a7mHDIRezVKTRCrOq78v577GXq3bSa3EhrzVo= +github.com/containerd/typeurl/v2 v2.1.1 h1:3Q4Pt7i8nYwy2KmQWIw2+1hTvwTE/6w9FqcttATPO/4= +github.com/containerd/typeurl/v2 v2.1.1/go.mod h1:IDp2JFvbwZ31H8dQbEIY7sDl2L3o3HZj1hsSQlywkQ0= +github.com/cpuguy83/dockercfg v0.3.1 h1:/FpZ+JaygUR/lZP2NlFI2DVfrOEMAIKP5wWEJdoYe9E= +github.com/cpuguy83/dockercfg v0.3.1/go.mod h1:sugsbF4//dDlL/i+S+rtpIWp+5h0BHJHfjj5/jFyUJc= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= +github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= +github.com/docker/buildx v0.12.0-rc2.0.20231219140829-617f538cb315 h1:UZxx9xBADdf/9UmSdEUi+pdJoPKpgcf9QUAY5gEIYmY= +github.com/docker/buildx v0.12.0-rc2.0.20231219140829-617f538cb315/go.mod h1:X8ZHhuW6ncwtoJ36TlU+gyaROTcBkTE01VHYmTStQCE= +github.com/docker/cli v25.0.1+incompatible h1:mFpqnrS6Hsm3v1k7Wa/BO23oz0k121MTbTO1lpcGSkU= +github.com/docker/cli v25.0.1+incompatible/go.mod h1:JLrzqnKDaYBop7H2jaqPtU4hHvMKP+vjCwu2uszcLI8= +github.com/docker/compose/v2 v2.24.3 h1:BVc1oDV7aQgksH64pDKTvcI95G36uJ+Mz9DGGBBoZeQ= +github.com/docker/compose/v2 v2.24.3/go.mod h1:D8Nv9+juzD7xiMyyHJ7G2J/MOYiGBmb9SvdIW5+2zKo= +github.com/docker/distribution v2.8.3+incompatible h1:AtKxIZ36LoNK51+Z6RpzLpddBirtxJnzDrHLEKxTAYk= +github.com/docker/distribution v2.8.3+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= +github.com/docker/docker v25.0.3+incompatible h1:D5fy/lYmY7bvZa0XTZ5/UJPljor41F+vdyJG5luQLfQ= +github.com/docker/docker v25.0.3+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker-credential-helpers v0.8.0 h1:YQFtbBQb4VrpoPxhFuzEBPQ9E16qz5SpHLS+uswaCp8= +github.com/docker/docker-credential-helpers v0.8.0/go.mod h1:UGFXcuoQ5TxPiB54nHOZ32AWRqQdECoh/Mg0AlEYb40= +github.com/docker/go v1.5.1-1.0.20160303222718-d30aec9fd63c h1:lzqkGL9b3znc+ZUgi7FlLnqjQhcXxkNM/quxIjBVMD0= +github.com/docker/go v1.5.1-1.0.20160303222718-d30aec9fd63c/go.mod h1:CADgU4DSXK5QUlFslkQu2yW2TKzFZcXq/leZfM0UH5Q= +github.com/docker/go-connections v0.5.0 h1:USnMq7hx7gwdVZq1L49hLXaFtUdTADjXGp+uj1Br63c= +github.com/docker/go-connections v0.5.0/go.mod h1:ov60Kzw0kKElRwhNs9UlUHAE/F9Fe6GLaXnqyDdmEXc= +github.com/docker/go-metrics v0.0.1 h1:AgB/0SvBxihN0X8OR4SjsblXkbMvalQ8cjmtKQ2rQV8= +github.com/docker/go-metrics v0.0.1/go.mod h1:cG1hvH2utMXtqgqqYE9plW6lDxS3/5ayHzueweSI3Vw= +github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= +github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= +github.com/emicklei/go-restful/v3 v3.10.1 h1:rc42Y5YTp7Am7CS630D7JmhRjq4UlEUuEKfrDac4bSQ= +github.com/emicklei/go-restful/v3 v3.10.1/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRry+4bhvbpWn3mrbc= +github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= +github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= +github.com/frankban/quicktest v1.14.0 h1:+cqqvzZV87b4adx/5ayVOaYZ2CrvM4ejQvUdBzPPUss= +github.com/frankban/quicktest v1.14.0/go.mod h1:NeW+ay9A/U67EYXNFA1nPE8e/tnQv/09mUdL/ijj8og= +github.com/fsnotify/fsevents v0.1.1 h1:/125uxJvvoSDDBPen6yUZbil8J9ydKZnnl3TWWmvnkw= +github.com/fsnotify/fsevents v0.1.1/go.mod h1:+d+hS27T6k5J8CRaPLKFgwKYcpS7GwW3Ule9+SC2ZRc= +github.com/fvbommel/sortorder v1.0.2 h1:mV4o8B2hKboCdkJm+a7uX/SIpZob4JzUpc5GGnM45eo= +github.com/fvbommel/sortorder v1.0.2/go.mod h1:uk88iVf1ovNn1iLfgUVU2F9o5eO30ui720w+kxuqRs0= +github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= +github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= +github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= +github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-openapi/jsonpointer v0.19.5 h1:gZr+CIYByUqjcgeLXnQu2gHYQC9o73G2XUeOFYEICuY= +github.com/go-openapi/jsonpointer v0.19.5/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= +github.com/go-openapi/jsonreference v0.20.0 h1:MYlu0sBgChmCfJxxUKZ8g1cPWFOB37YSZqewK7OKeyA= +github.com/go-openapi/jsonreference v0.20.0/go.mod h1:Ag74Ico3lPc+zR+qjn4XBUmXymS4zJbYVCZmcgkasdo= +github.com/go-openapi/swag v0.19.14 h1:gm3vOOXfiuw5i9p5N9xJvfjvuofpyvLA9Wr6QfK5Fng= +github.com/go-openapi/swag v0.19.14/go.mod h1:QYRuS/SOXUCsnplDa677K7+DxSOj6IPNl/eQntq43wQ= +github.com/gofrs/flock v0.8.1 h1:+gYjHKf32LDeiEEFhQaotPbLuUXjY5ZqxKgXy7n59aw= +github.com/gofrs/flock v0.8.1/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU= +github.com/gogo/googleapis v1.4.1 h1:1Yx4Myt7BxzvUr5ldGSbwYiZG6t9wGBZ+8/fX3Wvtq0= +github.com/gogo/googleapis v1.4.1/go.mod h1:2lpHqI5OcWCtVElxXnPt+s8oJvMpySlOyM6xDCrzib4= +github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= +github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= +github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= +github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= +github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/gnostic v0.5.7-v3refs h1:FhTMOKj2VhjpouxvWJAV1TL304uMlb9zcDqkl6cEI54= +github.com/google/gnostic v0.5.7-v3refs/go.mod h1:73MKFl6jIHelAJNaBGFzt3SPtZULs9dYrGFt8OiIsHQ= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0= +github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 h1:El6M4kTTCOh6aBiKaUGG7oYTSPP8MxqL4YI3kZKwcP4= +github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510/go.mod h1:pupxD2MaaD3pAXIBCelhxNneeOaAeabZDe5s4K6zSpQ= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/gorilla/mux v1.8.1 h1:TuBL49tXwgrFYWhqrNgrUNEY92u81SPhu7sTdzQEiWY= +github.com/gorilla/mux v1.8.1/go.mod h1:AKf9I4AEqPTmMytcMc0KkNouC66V3BtZ4qD5fmWSiMQ= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0 h1:YBftPWNWd4WwGqtY2yeZL2ef8rHAxPBD8KFhJpmcqms= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0/go.mod h1:YN5jB8ie0yfIUg6VvR9Kz84aCaG7AsGZnLjhHbUqwPg= +github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= +github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-cleanhttp v0.5.2 h1:035FKYIWjmULyFRBKPs8TBQoi0x6d9G4xc9neXJWAZQ= +github.com/hashicorp/go-cleanhttp v0.5.2/go.mod h1:kO/YDlP8L1346E6Sodw+PrpBSV4/SoxCXGY6BqNFT48= +github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= +github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= +github.com/hashicorp/go-version v1.6.0 h1:feTTfFNnjP967rlCxM/I9g701jU+RN74YKx2mOkIeek= +github.com/hashicorp/go-version v1.6.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/heetch/avro v0.4.5 h1:BSnj4wEeUG1IjMTm9/tBwQnV3euuIVa1mRWHnm1t8VU= +github.com/heetch/avro v0.4.5/go.mod h1:gxf9GnbjTXmWmqxhdNbAMcZCjpye7RV5r9t3Q0dL6ws= +github.com/imdario/mergo v0.3.16 h1:wwQJbIsHYGMUyLSPrEq1CT16AhnhNJQ51+4fdHUnCl4= +github.com/imdario/mergo v0.3.16/go.mod h1:WBLT9ZmE3lPoWsEzCh9LPo3TiwVN+ZKEjmz+hD27ysY= +github.com/in-toto/in-toto-golang v0.5.0 h1:hb8bgwr0M2hGdDsLjkJ3ZqJ8JFLL/tgYdAxF/XEFBbY= +github.com/in-toto/in-toto-golang v0.5.0/go.mod h1:/Rq0IZHLV7Ku5gielPT4wPHJfH1GdHMCq8+WPxw8/BE= +github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= +github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= +github.com/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= +github.com/jonboulle/clockwork v0.4.0/go.mod h1:xgRqUGwRcjKCO1vbZUEtSLrqKoPSsUpK7fnezOII0kc= +github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY= +github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= +github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= +github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 h1:Z9n2FFNUXsshfwJMBgNA0RU6/i7WVaAegv3PtuIHPMs= +github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51/go.mod h1:CzGEWj7cYgsdH8dAjBGEr58BoE7ScuLd+fwFZ44+/x8= +github.com/klauspost/compress v1.17.4 h1:Ej5ixsIri7BrIjBkRZLTo6ghwrEtHFk7ijlczPW4fZ4= +github.com/klauspost/compress v1.17.4/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= +github.com/magiconair/properties v1.8.7 h1:IeQXZAiQcpL9mgcAe1Nu6cX9LLw6ExEHKjN0VQdvPDY= +github.com/magiconair/properties v1.8.7/go.mod h1:Dhd985XPs7jluiymwWYZ0G4Z61jb3vdS329zhj2hYo0= +github.com/mailru/easyjson v0.7.6 h1:8yTIVnZgCoiM1TgqoeTl+LfU5Jg6/xL3QhGQnimLYnA= +github.com/mailru/easyjson v0.7.6/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= +github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= +github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= +github.com/mattn/go-isatty v0.0.17 h1:BTarxUcIeDqL27Mc+vyvdWYSL28zpIhv3RoTdsLMPng= +github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= +github.com/mattn/go-runewidth v0.0.15 h1:UNAjwbU9l54TA3KzvqLGxwWjHmMgBUVhBiTjelZgg3U= +github.com/mattn/go-runewidth v0.0.15/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= +github.com/mattn/go-shellwords v1.0.12 h1:M2zGm7EW6UQJvDeQxo4T51eKPurbeFbe8WtebGE2xrk= +github.com/mattn/go-shellwords v1.0.12/go.mod h1:EZzvwXDESEeg03EKmM+RmDnNOPKG4lLtQsUlTZDWQ8Y= +github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= +github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= +github.com/mgutz/ansi v0.0.0-20170206155736-9520e82c474b h1:j7+1HpAFS1zy5+Q4qx1fWh90gTKwiN4QCGoY9TWyyO4= +github.com/mgutz/ansi v0.0.0-20170206155736-9520e82c474b/go.mod h1:01TrycV0kFyexm33Z7vhZRXopbI8J3TDReVlkTgMUxE= +github.com/miekg/pkcs11 v1.1.1 h1:Ugu9pdy6vAYku5DEpVWVFPYnzV+bxB+iRdbuFSu7TvU= +github.com/miekg/pkcs11 v1.1.1/go.mod h1:XsNlhZGX73bx86s2hdc/FuaLm2CPZJemRLMA+WTFxgs= +github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw= +github.com/mitchellh/copystructure v1.2.0/go.mod h1:qLl+cE2AmVv+CoeAwDPye/v+N2HKCj9FbZEVFJRxO9s= +github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= +github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mitchellh/reflectwalk v1.0.2 h1:G2LzWKi524PWgd3mLHV8Y5k7s6XUvT0Gef6zxSIeXaQ= +github.com/mitchellh/reflectwalk v1.0.2/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= +github.com/moby/buildkit v0.13.0-beta1.0.20231219135447-957cb50df991 h1:r80LLQ91uOLxU1ElAvrB1o8oBsph51lPzVnr7t2b200= +github.com/moby/buildkit v0.13.0-beta1.0.20231219135447-957cb50df991/go.mod h1:6MddWPSL5jxy+W8eMMHWDOfZzzRRKWXPZqajw72YHBc= +github.com/moby/locker v1.0.1 h1:fOXqR41zeveg4fFODix+1Ch4mj/gT0NE1XJbp/epuBg= +github.com/moby/locker v1.0.1/go.mod h1:S7SDdo5zpBK84bzzVlKr2V0hz+7x9hWbYC/kq7oQppc= +github.com/moby/patternmatcher v0.6.0 h1:GmP9lR19aU5GqSSFko+5pRqHi+Ohk1O69aFiKkVGiPk= +github.com/moby/patternmatcher v0.6.0/go.mod h1:hDPoyOpDY7OrrMDLaYoY3hf52gNCR/YOUYxkhApJIxc= +github.com/moby/spdystream v0.2.0 h1:cjW1zVyyoiM0T7b6UoySUFqzXMoqRckQtXwGPiBhOM8= +github.com/moby/spdystream v0.2.0/go.mod h1:f7i0iNDQJ059oMTcWxx8MA/zKFIuD/lY+0GqbN2Wy8c= +github.com/moby/sys/mountinfo v0.7.1 h1:/tTvQaSJRr2FshkhXiIpux6fQ2Zvc4j7tAhMTStAG2g= +github.com/moby/sys/mountinfo v0.7.1/go.mod h1:IJb6JQeOklcdMU9F5xQ8ZALD+CUr5VlGpwtX+VE0rpI= +github.com/moby/sys/sequential v0.5.0 h1:OPvI35Lzn9K04PBbCLW0g4LcFAJgHsvXsRyewg5lXtc= +github.com/moby/sys/sequential v0.5.0/go.mod h1:tH2cOOs5V9MlPiXcQzRC+eEyab644PWKGRYaaV5ZZlo= +github.com/moby/sys/signal v0.7.0 h1:25RW3d5TnQEoKvRbEKUGay6DCQ46IxAVTT9CUMgmsSI= +github.com/moby/sys/signal v0.7.0/go.mod h1:GQ6ObYZfqacOwTtlXvcmh9A26dVRul/hbOZn88Kg8Tg= +github.com/moby/sys/symlink v0.2.0 h1:tk1rOM+Ljp0nFmfOIBtlV3rTDlWOwFRhjEeAhZB0nZc= +github.com/moby/sys/symlink v0.2.0/go.mod h1:7uZVF2dqJjG/NsClqul95CqKOBRQyYSNnJ6BMgR/gFs= +github.com/moby/sys/user v0.1.0 h1:WmZ93f5Ux6het5iituh9x2zAG7NFY9Aqi49jjE1PaQg= +github.com/moby/sys/user v0.1.0/go.mod h1:fKJhFOnsCN6xZ5gSfbM6zaHGgDJMrqt9/reuj4T7MmU= +github.com/moby/term v0.5.0 h1:xt8Q1nalod/v7BqbG21f8mQPqH+xAaC9C3N3wfWbVP0= +github.com/moby/term v0.5.0/go.mod h1:8FzsFHVUBGZdbDsJw/ot+X+d5HLUbvklYLJ9uGfcI3Y= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= +github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= +github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U= +github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM= +github.com/opencontainers/image-spec v1.1.0 h1:8SG7/vwALn54lVB/0yZ/MMwhFrPYtpEHQb2IpWsCzug= +github.com/opencontainers/image-spec v1.1.0/go.mod h1:W4s4sFTMaBeK1BQLXbG4AdM2szdn85PY75RI83NrTrM= +github.com/pelletier/go-toml v1.9.5 h1:4yBQzkHv+7BHq2PQUZF3Mx0IYxG7LsP222s7Agd3ve8= +github.com/pelletier/go-toml v1.9.5/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= +github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8= +github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc= +github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= +github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= +github.com/prometheus/common v0.42.0 h1:EKsfXEYo4JpWMHH5cg+KOUWeuJSov1Id8zGR8eeI1YM= +github.com/prometheus/common v0.42.0/go.mod h1:xBwqVerjNdUDjgODMpudtOMwlOwf2SaTr1yjz4b7Zbc= +github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= +github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= +github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= +github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= +github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= +github.com/secure-systems-lab/go-securesystemslib v0.4.0 h1:b23VGrQhTA8cN2CbBw7/FulN9fTtqYUdS5+Oxzt+DUE= +github.com/secure-systems-lab/go-securesystemslib v0.4.0/go.mod h1:FGBZgq2tXWICsxWQW1msNf49F0Pf2Op5Htayx335Qbs= +github.com/serialx/hashring v0.0.0-20190422032157-8b2912629002 h1:ka9QPuQg2u4LGipiZGsgkg3rJCo4iIUCy75FddM0GRQ= +github.com/serialx/hashring v0.0.0-20190422032157-8b2912629002/go.mod h1:/yeG0My1xr/u+HZrFQ1tOQQQQrOawfyMUH13ai5brBc= +github.com/shibumi/go-pathspec v1.3.0 h1:QUyMZhFo0Md5B8zV8x2tesohbb5kfbpTi9rBnKh5dkI= +github.com/shibumi/go-pathspec v1.3.0/go.mod h1:Xutfslp817l2I1cZvgcfeMQJG5QnU2lh5tVaaMCl3jE= +github.com/shirou/gopsutil/v3 v3.23.12 h1:z90NtUkp3bMtmICZKpC4+WaknU1eXtp5vtbQ11DgpE4= +github.com/shirou/gopsutil/v3 v3.23.12/go.mod h1:1FrWgea594Jp7qmjHUUPlJDTPgcsb9mGnXDxavtikzM= +github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= +github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= +github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= +github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= +github.com/sony/gobreaker v1.0.0 h1:feX5fGGXSl3dYd4aHZItw+FpHLvvoaqkawKjVNiFMNQ= +github.com/sony/gobreaker v1.0.0/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= +github.com/spf13/cobra v1.8.0 h1:7aJaZx1B85qltLMc546zn58BxxfZdR/W22ej9CFoEf0= +github.com/spf13/cobra v1.8.0/go.mod h1:WXLWApfZ71AjXPya3WOlMsY9yMs7YeiHhFVlvLyhcho= +github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= +github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/testcontainers/testcontainers-go v0.29.1 h1:z8kxdFlovA2y97RWx98v/TQ+tR+SXZm6p35M+xB92zk= +github.com/testcontainers/testcontainers-go v0.29.1/go.mod h1:SnKnKQav8UcgtKqjp/AD8bE1MqZm+3TDb/B8crE3XnI= +github.com/testcontainers/testcontainers-go/modules/compose v0.29.1 h1:47ipPM+s+ltCDOP3Sa1j95AkNb+z+WGiHLDbLU8ixuc= +github.com/testcontainers/testcontainers-go/modules/compose v0.29.1/go.mod h1:Sqh+Ef2ESdbJQjTJl57UOkEHkOc7gXvQLg1b5xh6f1Y= +github.com/theupdateframework/notary v0.7.0 h1:QyagRZ7wlSpjT5N2qQAh/pN+DVqgekv4DzbAiAiEL3c= +github.com/theupdateframework/notary v0.7.0/go.mod h1:c9DRxcmhHmVLDay4/2fUYdISnHqbFDGRSlXPO0AhYWw= +github.com/tilt-dev/fsnotify v1.4.8-0.20220602155310-fff9c274a375 h1:QB54BJwA6x8QU9nHY3xJSZR2kX9bgpZekRKGkLTmEXA= +github.com/tilt-dev/fsnotify v1.4.8-0.20220602155310-fff9c274a375/go.mod h1:xRroudyp5iVtxKqZCrA6n2TLFRBf8bmnjr1UD4x+z7g= +github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= +github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= +github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= +github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= +github.com/tonistiigi/fsutil v0.0.0-20230825212630-f09800878302 h1:ZT8ibgassurSISJ1Pj26NsM3vY2jxFZn63Nd/TpHmRw= +github.com/tonistiigi/fsutil v0.0.0-20230825212630-f09800878302/go.mod h1:9kMVqMyQ/Sx2df5LtnGG+nbrmiZzCS7V6gjW3oGHsvI= +github.com/tonistiigi/units v0.0.0-20180711220420-6950e57a87ea h1:SXhTLE6pb6eld/v/cCndK0AMpt1wiVFb/YYmqB3/QG0= +github.com/tonistiigi/units v0.0.0-20180711220420-6950e57a87ea/go.mod h1:WPnis/6cRcDZSUvVmezrxJPkiO87ThFYsoUiMwWNDJk= +github.com/tonistiigi/vt100 v0.0.0-20230623042737-f9a4f7ef6531 h1:Y/M5lygoNPKwVNLMPXgVfsRT40CSFKXCxuU8LoHySjs= +github.com/tonistiigi/vt100 v0.0.0-20230623042737-f9a4f7ef6531/go.mod h1:ulncasL3N9uLrVann0m+CDlJKWsIAP34MPcOJF6VRvc= +github.com/xeipuuv/gojsonpointer v0.0.0-20190905194746-02993c407bfb h1:zGWFAtiMcyryUHoUjUJX0/lt1H2+i2Ka2n+D3DImSNo= +github.com/xeipuuv/gojsonpointer v0.0.0-20190905194746-02993c407bfb/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= +github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415 h1:EzJWgHovont7NscjpAxXsDA8S8BMYve8Y5+7cuRE7R0= +github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= +github.com/xeipuuv/gojsonschema v1.2.0 h1:LhYJRs+L4fBtjZUfuSZIKGeVu0QRy8e5Xi7D17UxZ74= +github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= +github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yusufpapurcu/wmi v1.2.3 h1:E1ctvB7uKFMOJw3fdOW32DwGE9I7t++CRUEMKvFoFiw= +github.com/yusufpapurcu/wmi v1.2.3/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= +gitlab.zgtools.net/devex/archetypes/gomods/zfmt v1.0.68 h1:EMTWPKIGT8Vh2JwqEbOkSFLFUsB2Cdc5gr0sn7YmUa4= +gitlab.zgtools.net/devex/archetypes/gomods/zfmt v1.0.68/go.mod h1:h8Wbct3spciNuM5Me/BccIu95vo2bpkhCIJvZx4v9sk= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.45.0 h1:RsQi0qJ2imFfCvZabqzM9cNXBG8k6gXMv1A0cXRmH6A= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.45.0/go.mod h1:vsh3ySueQCiKPxFLvjWC4Z135gIa34TQ/NSqkDTZYUM= +go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.45.0 h1:2ea0IkZBsWH+HA2GkD+7+hRw2u97jzdFyRtXuO14a1s= +go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.45.0/go.mod h1:4m3RnBBb+7dB9d21y510oO1pdB1V4J6smNf14WXcBFQ= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.45.0 h1:x8Z78aZx8cOF0+Kkazoc7lwUNMGy0LrzEMxTm4BbTxg= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.45.0/go.mod h1:62CPTSry9QZtOaSsE3tOzhx6LzDhHnXJ6xHeMNNiM6Q= +go.opentelemetry.io/otel v1.27.0 h1:9BZoF3yMK/O1AafMiQTVu0YDj5Ea4hPhxCs7sGva+cg= +go.opentelemetry.io/otel v1.27.0/go.mod h1:DMpAK8fzYRzs+bi3rS5REupisuqTheUlSZJ1WnZaPAQ= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.42.0 h1:ZtfnDL+tUrs1F0Pzfwbg2d59Gru9NCH3bgSHBM6LDwU= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.42.0/go.mod h1:hG4Fj/y8TR/tlEDREo8tWstl9fO9gcFkn4xrx0Io8xU= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v0.42.0 h1:NmnYCiR0qNufkldjVvyQfZTHSdzeHoZ41zggMsdMcLM= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v0.42.0/go.mod h1:UVAO61+umUsHLtYb8KXXRoHtxUkdOPkYidzW3gipRLQ= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.42.0 h1:wNMDy/LVGLj2h3p6zg4d0gypKfWKSWI14E1C4smOgl8= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.42.0/go.mod h1:YfbDdXAAkemWJK3H/DshvlrxqFB2rtW4rY6ky/3x/H0= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.19.0 h1:Mne5On7VWdx7omSrSSZvM4Kw7cS7NQkOOmLcgscI51U= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.19.0/go.mod h1:IPtUMKL4O3tH5y+iXVyAXqpAwMuzC1IrxVS81rummfE= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.19.0 h1:3d+S281UTjM+AbF31XSOYn1qXn3BgIdWl8HNEpx08Jk= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.19.0/go.mod h1:0+KuTDyKL4gjKCF75pHOX4wuzYDUZYfAQdSu43o+Z2I= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.19.0 h1:IeMeyr1aBvBiPVYihXIaeIZba6b8E1bYp7lbdxK8CQg= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.19.0/go.mod h1:oVdCUtjq9MK9BlS7TtucsQwUcXcymNiEDjgDD2jMtZU= +go.opentelemetry.io/otel/exporters/prometheus v0.42.0 h1:jwV9iQdvp38fxXi8ZC+lNpxjK16MRcZlpDYvbuO1FiA= +go.opentelemetry.io/otel/exporters/prometheus v0.42.0/go.mod h1:f3bYiqNqhoPxkvI2LrXqQVC546K7BuRDL/kKuxkujhA= +go.opentelemetry.io/otel/metric v1.27.0 h1:hvj3vdEKyeCi4YaYfNjv2NUje8FqKqUY8IlF0FxV/ik= +go.opentelemetry.io/otel/metric v1.27.0/go.mod h1:mVFgmRlhljgBiuk/MP/oKylr4hs85GZAylncepAX/ak= +go.opentelemetry.io/otel/sdk v1.19.0 h1:6USY6zH+L8uMH8L3t1enZPR3WFEmSTADlqldyHtJi3o= +go.opentelemetry.io/otel/sdk v1.19.0/go.mod h1:NedEbbS4w3C6zElbLdPJKOpJQOrGUJ+GfzpjUvI0v1A= +go.opentelemetry.io/otel/sdk/metric v1.19.0 h1:EJoTO5qysMsYCa+w4UghwFV/ptQgqSL/8Ni+hx+8i1k= +go.opentelemetry.io/otel/sdk/metric v1.19.0/go.mod h1:XjG0jQyFJrv2PbMvwND7LwCEhsJzCzV5210euduKcKY= +go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= +go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= +go.opentelemetry.io/proto/otlp v1.0.0 h1:T0TX0tmXU8a3CbNXzEKGeU5mIVOdf0oykP+u2lIVU/I= +go.opentelemetry.io/proto/otlp v1.0.0/go.mod h1:Sy6pihPLfYHkr3NkUbEhGHFhINUSI/v80hjKIs5JXpM= +go.uber.org/mock v0.4.0 h1:VcM4ZOtdbR4f6VXfiOpwpVJDL6lCReaZ6mw31wqh7KU= +go.uber.org/mock v0.4.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= +golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= +golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 h1:MGwJjxBy0HJshjDNfLsYO8xppfqWlA5ZT9OhtUUhTNw= +golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= +golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.16.0 h1:QX4fJ0Rr5cPQCF7O9lh9Se4pmwfwskqZfq5moyldzic= +golang.org/x/mod v0.16.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.22.0 h1:9sGLhx7iRIHEiX0oAJ3MRZMUCElJgy7Br1nO+AMN3Tc= +golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= +golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= +golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= +golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= +golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= +golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= +golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.13.0 h1:Iey4qkscZuv0VvIt8E0neZjtPVQFSc870HQ448QgEmQ= +golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= +google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= +google.golang.org/genproto v0.0.0-20240325203815-454cdb8f5daa h1:ePqxpG3LVx+feAUOx8YmR5T7rc0rdzK8DyxM8cQ9zq0= +google.golang.org/genproto v0.0.0-20240325203815-454cdb8f5daa/go.mod h1:CnZenrTdRJb7jc+jOm0Rkywq+9wh0QC4U8tyiRbEPPM= +google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237 h1:RFiFrvy37/mpSpdySBDrUdipW/dHwsRwh3J3+A9VgT4= +google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237/go.mod h1:Z5Iiy3jtmioajWHDGFk7CeugTyHtPvMHA4UTmUkyalE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240318140521-94a12d6c2237 h1:NnYq6UN9ReLM9/Y01KWNOWyI5xQ9kbIms5GGJVwS/Yc= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240318140521-94a12d6c2237/go.mod h1:WtryC6hu0hhx87FDGxWCDptyssuo68sk10vYjF+T9fY= +google.golang.org/grpc v1.62.1 h1:B4n+nfKzOICUXMgyrNd19h/I9oH0L1pizfk1d4zSgTk= +google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= +gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= +gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +k8s.io/api v0.26.7 h1:Lf4iEBEJb5OFNmawtBfSZV/UNi9riSJ0t1qdhyZqI40= +k8s.io/api v0.26.7/go.mod h1:Vk9bMadzA49UHPmHB//lX7VRCQSXGoVwfLd3Sc1SSXI= +k8s.io/apimachinery v0.26.7 h1:590jSBwaSHCAFCqltaEogY/zybFlhGsnLteLpuF2wig= +k8s.io/apimachinery v0.26.7/go.mod h1:qYzLkrQ9lhrZRh0jNKo2cfvf/R1/kQONnSiyB7NUJU0= +k8s.io/apiserver v0.26.7 h1:NX/zBZZn4R+Cq6shwyn8Pn8REd0yJJ16dbtv9WkEVEU= +k8s.io/apiserver v0.26.7/go.mod h1:r0wDRWHI7VL/KlQLTkJJBVGZ3KeNfv+VetlyRtr86xs= +k8s.io/client-go v0.26.7 h1:hyU9aKHlwVOykgyxzGYkrDSLCc4+mimZVyUJjPyUn1E= +k8s.io/client-go v0.26.7/go.mod h1:okYjy0jtq6sdeztALDvCh24tg4opOQS1XNvsJlERDAo= +k8s.io/klog/v2 v2.90.1 h1:m4bYOKall2MmOiRaR1J+We67Do7vm9KiQVlT96lnHUw= +k8s.io/klog/v2 v2.90.1/go.mod h1:y1WjHnz7Dj687irZUWR/WLkLc5N1YHtjLdmgWjndZn0= +k8s.io/kube-openapi v0.0.0-20221012153701-172d655c2280 h1:+70TFaan3hfJzs+7VK2o+OGxg8HsuBr/5f6tVAjDu6E= +k8s.io/kube-openapi v0.0.0-20221012153701-172d655c2280/go.mod h1:+Axhij7bCpeqhklhUTe3xmOn6bWxolyZEeyaFpjGtl4= +k8s.io/utils v0.0.0-20230220204549-a5ecb0141aa5 h1:kmDqav+P+/5e1i9tFfHq1qcF3sOrDp+YEkVDAHu7Jwk= +k8s.io/utils v0.0.0-20230220204549-a5ecb0141aa5/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= +sigs.k8s.io/json v0.0.0-20220713155537-f223a00ba0e2 h1:iXTIw73aPyC+oRdyqqvVJuloN1p0AC/kzH07hu3NE+k= +sigs.k8s.io/json v0.0.0-20220713155537-f223a00ba0e2/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0= +sigs.k8s.io/structured-merge-diff/v4 v4.2.3 h1:PRbqxJClWWYMNV1dhaG4NsibJbArud9kFxnAMREiWFE= +sigs.k8s.io/structured-merge-diff/v4 v4.2.3/go.mod h1:qjx8mGObPmV2aSZepjQjbmb2ihdVs8cGKBraizNC69E= +sigs.k8s.io/yaml v1.3.0 h1:a2VclLzOGrwOHDiV8EfBGhvjHvP46CtW5j6POvhYGGo= +sigs.k8s.io/yaml v1.3.0/go.mod h1:GeOyir5tyXNByN85N/dRIT9es5UQNerPYEKK56eTBm8= +tags.cncf.io/container-device-interface v0.6.2 h1:dThE6dtp/93ZDGhqaED2Pu374SOeUkBfuvkLuiTdwzg= +tags.cncf.io/container-device-interface v0.6.2/go.mod h1:Shusyhjs1A5Na/kqPVLL0KqnHQHuunol9LFeUNkuGVE= diff --git a/heap.go b/heap.go new file mode 100644 index 0000000..7743348 --- /dev/null +++ b/heap.go @@ -0,0 +1,68 @@ +package zkafka + +import ( + "container/heap" + "errors" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" +) + +type offsetHeap struct { + data _offsetHeap +} + +// Push adds an offset to the heap +func (h *offsetHeap) Push(offset kafka.TopicPartition) { + heap.Push(&h.data, offset) +} + +// Pop returns the minimum offset from the heap and rearranges the heap to put the new minimum at the root +func (h *offsetHeap) Pop() kafka.TopicPartition { + if len(h.data) == 0 { + panic("popped empty heap") + } + return heap.Pop(&h.data).(kafka.TopicPartition) +} + +// Peek returns the minimum offset from the heap without any side effects. +func (h *offsetHeap) Peek() (kafka.TopicPartition, error) { + if len(h.data) == 0 { + return kafka.TopicPartition{}, errors.New("peeked empty heap") + } + return (h.data)[0], nil +} + +// SeekPop linearly searches the heap looking for a match, and removes and returns it. +// If nothing is found, nil is returned and the heap isn't mutated. +// It is an O(n) and therefore is not as efficient as Peek or Pop, but is necessary +// for removing arbitrary items from the data structure +func (h *offsetHeap) SeekPop(partition kafka.TopicPartition) *kafka.TopicPartition { + for i, d := range h.data { + if d == partition { + h.data = append(h.data[:i], h.data[i+1:]...) + return &d + } + } + return nil +} + +// An _offsetHeap is a min-heap of topicPartitions where offset is used to determine order +type _offsetHeap []kafka.TopicPartition + +var _ heap.Interface = (*_offsetHeap)(nil) + +func (h _offsetHeap) Len() int { return len(h) } +func (h _offsetHeap) Less(i, j int) bool { return h[i].Offset < h[j].Offset } +func (h _offsetHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } + +func (h *_offsetHeap) Push(x any) { + *h = append(*h, x.(kafka.TopicPartition)) +} + +func (h *_offsetHeap) Pop() any { + old := *h + n := len(old) + x := old[n-1] + *h = old[0 : n-1] + return x +} diff --git a/heap_test.go b/heap_test.go new file mode 100644 index 0000000..2149110 --- /dev/null +++ b/heap_test.go @@ -0,0 +1,112 @@ +package zkafka + +import ( + "slices" + "testing" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/stretchr/testify/require" +) + +func Test_offsetHeap_PushPopPeek_WhenInsertSmaller(t *testing.T) { + defer recoverThenFail(t) + heap := offsetHeap{} + input1 := kafka.TopicPartition{Partition: 1, Offset: 1} + heap.Push(input1) + got, err := heap.Peek() + require.NoError(t, err) + require.Equal(t, input1, got, "expected the minimum offset, which is the only offset") + + input2 := kafka.TopicPartition{Partition: 1, Offset: 0} + // insert smaller item into heap + heap.Push(input2) + got, err = heap.Peek() + require.NoError(t, err) + require.Equal(t, input2, got) + + got = heap.Pop() + require.Equal(t, input2, got) + + got = heap.Pop() + require.Equal(t, input1, got) +} + +func Test_offsetHeap_PushPopPeek_WhenInsertBigger(t *testing.T) { + defer recoverThenFail(t) + heap := offsetHeap{} + input1 := kafka.TopicPartition{Partition: 1, Offset: 1} + heap.Push(input1) + got, err := heap.Peek() + require.NoError(t, err) + require.Equal(t, input1, got, "expected the minimum offset, which is the only offset") + + input2 := kafka.TopicPartition{Partition: 1, Offset: 100} + // insert smaller item into heap + heap.Push(input2) + got, err = heap.Peek() + require.NoError(t, err) + require.Equal(t, input1, got) + + got = heap.Pop() + require.Equal(t, input1, got) +} + +func Test_offsetHeap_PopWhenEmptyResultsInPanic(t *testing.T) { + defer func() { + if r := recover(); r == nil { + require.Fail(t, "expected panic on pop when empty") + } + }() + heap := offsetHeap{} + _ = heap.Pop() +} + +func Test_offsetHeap_PeekWhenEmpty(t *testing.T) { + defer recoverThenFail(t) + + heap := offsetHeap{} + _, err := heap.Peek() + require.Error(t, err, "expected error on peek when empty") +} + +// Test_offsetHeap_SeekPop_DoesntImpactHeapOrdering +// given 100 items in the heap. +// when n are seek popped (taken out of the middle of the heap) +// then when we heap.Pop we still get the minimum offsets +func Test_offsetHeap_SeekPop_DoesntImpactHeapOrdering(t *testing.T) { + defer recoverThenFail(t) + heap := offsetHeap{} + var offsets []kafka.TopicPartition + + // build up a heap of size N + count := 100 + for i := 0; i < count; i++ { + offset := kafka.TopicPartition{Partition: 1, Offset: kafka.Offset(i)} + offsets = append(offsets, offset) + heap.Push(offset) + } + require.Len(t, heap.data, count) + + // remove M items from heap + offsetsToRemove := []int{95, 34, 12, 2, 44, 45} + for _, index := range offsetsToRemove { + heap.SeekPop(offsets[index]) + } + + remainingCount := count - len(offsetsToRemove) + require.Len(t, heap.data, remainingCount) + + // Loop through the N items that were in the heap + // skip the ones known to be seekPopped out + i := 0 + for i < count { + if slices.Contains(offsetsToRemove, i) { + i++ + continue + } + got := heap.Pop() + want := offsets[i] + require.Equal(t, want, got, "Expect pop to still pop minumums even after seek pops") + i++ + } +} diff --git a/lifecycle.go b/lifecycle.go new file mode 100644 index 0000000..ce8f12a --- /dev/null +++ b/lifecycle.go @@ -0,0 +1,132 @@ +package zkafka + +//go:generate mockgen -package=mock_zkafka -destination=./mocks/mock_lifecycle.go -source=./lifecycle.go + +import ( + "context" + "errors" + "time" +) + +type LifecyclePostReadMeta struct { + Topic string + GroupID string + // Message that was read + Message *Message +} +type LifecyclePreProcessingMeta struct { + Topic string + GroupID string + VirtualPartitionIndex int + // Time since the message was sent to the topic + TopicLag time.Duration + // Message containing being processed + Message *Message +} + +type LifecyclePostProcessingMeta struct { + Topic string + GroupID string + VirtualPartitionIndex int + // Time taken to process the message + ProcessingTime time.Duration + // Message processed + Msg *Message + // Response code returned by the processor + ResponseErr error +} + +type LifecyclePostAckMeta struct { + Topic string + // Time when the message was published to the queue + ProduceTime time.Time +} + +type LifecyclePreWriteMeta struct{} + +type LifecyclePreWriteResp struct { + Headers map[string][]byte +} + +type LifecycleHooks struct { + // Called by work after reading a message, offers the ability to customize the context object (resulting context object passed to work processor) + PostRead func(ctx context.Context, meta LifecyclePostReadMeta) (context.Context, error) + + // Called after receiving a message and before processing it. + PreProcessing func(ctx context.Context, meta LifecyclePreProcessingMeta) (context.Context, error) + + // Called after processing a message + PostProcessing func(ctx context.Context, meta LifecyclePostProcessingMeta) error + + // Called after sending a message to the queue + PostAck func(ctx context.Context, meta LifecyclePostAckMeta) error + + // Called prior to executing write operation + PreWrite func(ctx context.Context, meta LifecyclePreWriteMeta) (LifecyclePreWriteResp, error) + + // Call after the reader attempts a fanout call. + PostFanout func(ctx context.Context) +} + +// ChainLifecycleHooks chains multiple lifecycle hooks into one. The hooks are +// called in the order they are passed. All hooks are called, even when +// errors occur. Errors are accumulated in a wrapper error and returned to the +// caller. +func ChainLifecycleHooks(hooks ...LifecycleHooks) LifecycleHooks { + if len(hooks) == 0 { + return LifecycleHooks{} + } + if len(hooks) == 1 { + return hooks[0] + } + return LifecycleHooks{ + PreProcessing: func(ctx context.Context, meta LifecyclePreProcessingMeta) (context.Context, error) { + var allErrs error + + hookCtx := ctx + + for _, h := range hooks { + if h.PreProcessing != nil { + var err error + + hookCtx, err = h.PreProcessing(hookCtx, meta) + if err != nil { + allErrs = errors.Join(allErrs, err) + } + } + } + + return hookCtx, allErrs + }, + + PostProcessing: func(ctx context.Context, meta LifecyclePostProcessingMeta) error { + var allErrs error + + for _, h := range hooks { + if h.PostProcessing != nil { + err := h.PostProcessing(ctx, meta) + if err != nil { + allErrs = errors.Join(allErrs, err) + } + } + } + + return allErrs + }, + + PostAck: func(ctx context.Context, meta LifecyclePostAckMeta) error { + var allErrs error + + for _, h := range hooks { + if h.PostAck != nil { + err := h.PostAck(ctx, meta) + if err != nil { + allErrs = errors.Join(allErrs, err) + } + } + } + + return allErrs + }, + } +} diff --git a/lifecycle_test.go b/lifecycle_test.go new file mode 100644 index 0000000..2ba609e --- /dev/null +++ b/lifecycle_test.go @@ -0,0 +1,68 @@ +package zkafka + +import ( + "context" + "testing" + + "github.com/stretchr/testify/require" +) + +func Test_LifecycleChainedHooksAreCalled(t *testing.T) { + lhState := make(map[string]int) // Map from state to number of times called + + hooks1 := LifecycleHooks{ + PreProcessing: func(ctx context.Context, meta LifecyclePreProcessingMeta) (context.Context, error) { + lhState["hooks1-pre-processing"] += 1 + return ctx, nil + }, + PostProcessing: func(ctx context.Context, meta LifecyclePostProcessingMeta) error { + lhState["hooks1-post-processing"] += 1 + return nil + }, + PostAck: func(ctx context.Context, meta LifecyclePostAckMeta) error { + lhState["hooks1-post-ack"] += 1 + return nil + }, + } + + hooks2 := LifecycleHooks{ + PreProcessing: func(ctx context.Context, meta LifecyclePreProcessingMeta) (context.Context, error) { + lhState["hooks2-pre-processing"] += 1 + return ctx, nil + }, + PostProcessing: func(ctx context.Context, meta LifecyclePostProcessingMeta) error { + lhState["hooks2-post-processing"] += 1 + return nil + }, + PostAck: func(ctx context.Context, meta LifecyclePostAckMeta) error { + lhState["hooks2-post-ack"] += 1 + return nil + }, + } + + lh := ChainLifecycleHooks(hooks1, hooks2) + + lh.PreProcessing(context.Background(), LifecyclePreProcessingMeta{}) + require.Equal(t, 1, lhState["hooks1-pre-processing"], "hooks1-pre-processing not called") + require.Equal(t, 1, lhState["hooks2-pre-processing"], "hooks2-pre-processing not called") + require.Equal(t, 0, lhState["hooks1-post-processing"], "hooks1-post-processing called") + require.Equal(t, 0, lhState["hooks2-post-processing"], "hooks2-post-processing called") + require.Equal(t, 0, lhState["hooks1-post-ack"], "hooks1-post-ack called") + require.Equal(t, 0, lhState["hooks2-post-ack"], "hooks2-post-ack called") + + lh.PostProcessing(context.Background(), LifecyclePostProcessingMeta{}) + require.Equal(t, 1, lhState["hooks1-pre-processing"], "hooks1-pre-processing not called") + require.Equal(t, 1, lhState["hooks2-pre-processing"], "hooks2-pre-processing not called") + require.Equal(t, 1, lhState["hooks1-post-processing"], "hooks1-post-processing not called") + require.Equal(t, 1, lhState["hooks2-post-processing"], "hooks2-post-processing not called") + require.Equal(t, 0, lhState["hooks1-post-ack"], "hooks1-post-ack called") + require.Equal(t, 0, lhState["hooks2-post-ack"], "hooks2-post-ack called") + + lh.PostAck(context.Background(), LifecyclePostAckMeta{}) + require.Equal(t, 1, lhState["hooks1-pre-processing"], "hooks1-pre-processing not called") + require.Equal(t, 1, lhState["hooks2-pre-processing"], "hooks2-pre-processing not called") + require.Equal(t, 1, lhState["hooks1-post-processing"], "hooks1-post-processing not called") + require.Equal(t, 1, lhState["hooks2-post-processing"], "hooks2-post-processing not called") + require.Equal(t, 1, lhState["hooks1-post-ack"], "hooks1-post-ack not called") + require.Equal(t, 1, lhState["hooks2-post-ack"], "hooks2-post-ack not called") +} diff --git a/logger.go b/logger.go new file mode 100644 index 0000000..4f47c5b --- /dev/null +++ b/logger.go @@ -0,0 +1,26 @@ +package zkafka + +//go:generate mockgen -destination=./mocks/mock_logger.go -source=logger.go + +import ( + "context" +) + +// Logger is the interface that wraps basic logging functions +type Logger interface { + Debugw(ctx context.Context, msg string, keysAndValues ...any) + Infow(ctx context.Context, msg string, keysAndValues ...any) + Warnw(ctx context.Context, msg string, keysAndValues ...any) + Errorw(ctx context.Context, msg string, keysAndValues ...any) +} + +var _ Logger = (*NoopLogger)(nil) + +type NoopLogger struct{} + +func (l NoopLogger) Debugw(_ context.Context, _ string, _ ...any) {} + +func (l NoopLogger) Infow(_ context.Context, _ string, _ ...any) {} + +func (l NoopLogger) Warnw(_ context.Context, _ string, _ ...any) {} +func (l NoopLogger) Errorw(_ context.Context, _ string, _ ...any) {} diff --git a/message.go b/message.go new file mode 100644 index 0000000..e97758b --- /dev/null +++ b/message.go @@ -0,0 +1,151 @@ +package zkafka + +import ( + "context" + "fmt" + "os" + "sync" + "time" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/google/uuid" + "github.com/pkg/errors" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +// Message is a container for kafka message +type Message struct { + Key string + // There's a difference between a nil key and an empty key. A nil key gets assigned a topic partition by kafka via round-robin. + // An empty key is treated as a key with a value of "" and is assigned to a topic partition via the hash of the key (so will consistently go to the same key) + isKeyNil bool + Headers map[string][]byte + Offset int64 + Partition int32 + Topic string + GroupID string + TimeStamp time.Time + value []byte + topicPartition kafka.TopicPartition + fmt zfmt.Formatter + doneFunc func(ctx context.Context) + doneOnce sync.Once +} + +// A set of observability headers for ZG Kafka +const ( + obsKeyMessageID = "GUID" + obsKeyEventTime = "eventTime" + obsKeyOriginService = "originService" + obsKeyOriginHost = "originHost" +) + +func makeProducerMessageRaw(_ context.Context, serviceName, topic string, key *string, value []byte) kafka.Message { + kafkaMessage := kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Topic: &topic, + Partition: kafka.PartitionAny, + }, + Value: value, + } + + if key != nil { + kafkaMessage.Key = []byte(*key) + } + // Observability + kafkaMessage.Headers = append(kafkaMessage.Headers, kafka.Header{ + Key: obsKeyMessageID, + Value: []byte(uuid.New().String()), + }) + kafkaMessage.Headers = append(kafkaMessage.Headers, kafka.Header{ + Key: obsKeyEventTime, + Value: []byte(fmt.Sprintf("%d", time.Now().Unix())), + }) + kafkaMessage.Headers = append(kafkaMessage.Headers, kafka.Header{ + Key: obsKeyOriginService, + Value: []byte(serviceName), + }) + hostname, _ := os.Hostname() + // hn is empty string if there's an error + kafkaMessage.Headers = append(kafkaMessage.Headers, kafka.Header{ + Key: obsKeyOriginHost, + Value: []byte(hostname), + }) + return kafkaMessage +} + +func addHeaders(kafkaMessage kafka.Message, headers map[string][]byte) kafka.Message { + for k, v := range headers { + addStringAttribute(&kafkaMessage, k, v) + } + return kafkaMessage +} + +// addStringAttribute updates a kafka message header in place if the key exists already. +// If the key does not exist, it appends a new header. +func addStringAttribute(msg *kafka.Message, k string, v []byte) { + for i, h := range msg.Headers { + if h.Key == k { + msg.Headers[i].Value = v + return + } + } + msg.Headers = append(msg.Headers, kafka.Header{Key: k, Value: v}) +} + +// Headers extracts metadata from kafka message and stores it in a basic map +func headers(msg kafka.Message) map[string][]byte { + res := make(map[string][]byte) + for _, h := range msg.Headers { + res[h.Key] = h.Value + } + return res +} + +// DoneWithContext is used to alert that message processing has completed. +// This marks the message offset to be committed +func (m *Message) DoneWithContext(ctx context.Context) { + m.doneOnce.Do(func() { + m.doneFunc(ctx) + }) +} + +// Done is used to alert that message processing has completed. +// This marks the message offset to be committed +func (m *Message) Done() { + if m == nil { + return + } + m.doneOnce.Do(func() { + m.doneFunc(context.Background()) + }) +} + +// Decode reads message data and stores it in the value pointed to by v. +func (m *Message) Decode(v any) error { + if m.value == nil { + return errors.New("message is empty") + } + if m.fmt == nil { + // is error is most likely due to user calling KReader/KWriter + // with custom Formatter which can sometimes be nil + return errors.New("formatter is not set") + } + return m.fmt.Unmarshal(m.value, v) +} + +// Value returns a copy of the current value byte array. Useful for debugging +func (m *Message) Value() []byte { + if m == nil || m.value == nil { + return nil + } + out := make([]byte, len(m.value)) + copy(out, m.value) + return out +} + +// Response is a kafka response with the Partition where message was sent to along with its assigned Offset +type Response struct { + Partition int32 + Offset int64 +} diff --git a/message_test.go b/message_test.go new file mode 100644 index 0000000..d5d1f2b --- /dev/null +++ b/message_test.go @@ -0,0 +1,338 @@ +package zkafka + +import ( + "bytes" + "context" + "sync" + "testing" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/stretchr/testify/require" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +func Test_makeProducerMessageRaw(t *testing.T) { + type args struct { + ctx context.Context + serviceName string + topic string + key *string + value []byte + } + tests := []struct { + name string + args args + want kafka.Message + hasHeaders bool + }{ + { + name: "has fmtter with valid input, no key, no partition", + args: args{ + serviceName: "concierge/test/test_group", + topic: "test_topic", + value: []byte("string"), + }, + want: kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Topic: ptr("test_topic"), + // this indicates any partition to confluent-kafka-go + Partition: -1, + }, + Opaque: nil, + Headers: nil, + }, + hasHeaders: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + got := makeProducerMessageRaw(tt.args.ctx, tt.args.serviceName, tt.args.topic, tt.args.key, tt.args.value) + require.Equal(t, tt.want.TopicPartition, got.TopicPartition) + require.Equal(t, tt.want.Key, got.Key) + require.Equal(t, tt.want.Key, got.Key) + if tt.hasHeaders { + require.NotEmpty(t, got.Headers) + } + }) + } +} + +func TestMessage_Headers(t *testing.T) { + type fields struct { + msg kafka.Message + } + tests := []struct { + name string + fields fields + want map[string][]byte + }{ + { + name: "empty message", + fields: fields{}, + want: make(map[string][]byte), + }, + { + name: "msgs with headers", + fields: fields{msg: kafka.Message{ + Headers: []kafka.Header{ + { + Key: "key1", + Value: []byte("value1"), + }, + { + Key: "key2", + Value: []byte("value2"), + }, + }, + }}, + want: map[string][]byte{ + "key1": []byte("value1"), + "key2": []byte("value2"), + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + got := headers(tt.fields.msg) + require.Equal(t, tt.want, got) + }) + } +} + +func TestMessage_Decode(t *testing.T) { + type fields struct { + value []byte + fmt zfmt.Formatter + } + type args struct { + v any + } + tests := []struct { + name string + fields fields + args args + wantErr bool + }{ + { + name: "empty message, empty input => error", + fields: fields{}, + args: args{}, + wantErr: true, + }, + { + name: "valid message, no formatter, empty input => error", + fields: fields{ + value: []byte("test"), + }, + args: args{}, + wantErr: true, + }, + { + name: "valid message, formatter, empty input => error", + fields: fields{ + value: []byte("test"), + fmt: &zfmt.StringFormatter{}, + }, + args: args{}, + wantErr: true, + }, + { + name: "valid message, formatter, valid input => no error", + fields: fields{ + value: []byte("test"), + fmt: &zfmt.StringFormatter{}, + }, + args: args{ + v: &bytes.Buffer{}, + }, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + m := Message{ + value: tt.fields.value, + fmt: tt.fields.fmt, + } + err := m.Decode(tt.args.v) + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + }) + } +} + +func TestMessage_Done(t *testing.T) { + type fields struct { + Key string + Headers map[string][]byte + value []byte + fmt zfmt.Formatter + doneSig chan bool + } + tests := []struct { + name string + fields fields + }{ + { + name: "multiple calls to done", + fields: fields{ + doneSig: make(chan bool, 1), + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + isCalled := false + m := &Message{ + Key: tt.fields.Key, + Headers: tt.fields.Headers, + value: tt.fields.value, + fmt: tt.fields.fmt, + doneFunc: func(ctx context.Context) { + isCalled = true + }, + } + // call done multiple times and function should still return + var wg sync.WaitGroup + wg.Add(3) + for i := 0; i < 3; i++ { + go func() { + defer wg.Done() + m.DoneWithContext(context.Background()) + }() + } + wg.Wait() + require.True(t, isCalled, "doneFunc should have been called at least once") + }) + } +} + +func Test_addHeaders(t *testing.T) { + type args struct { + kafkaMessage kafka.Message + headers map[string][]byte + } + tests := []struct { + name string + args args + want kafka.Message + }{ + { + name: "populated RequestContext", + args: args{ + kafkaMessage: kafka.Message{}, + headers: map[string][]byte{ + "x-b3-traceid": []byte("2"), + "x-request-id": []byte("1"), + "x-user-id": []byte("userID1"), + "x-application-trail": []byte("trail"), + }, + }, + want: kafka.Message{ + Headers: []kafka.Header{ + { + Key: "x-b3-traceid", + Value: []byte("2"), + }, + { + Key: "x-request-id", + Value: []byte("1"), + }, + { + Key: "x-user-id", + Value: []byte("userID1"), + }, + { + Key: "x-application-trail", + Value: []byte("trail"), + }, + }, + }, + }, + { + name: "conflicting-fields-are-overwritten", + args: args{ + kafkaMessage: kafka.Message{ + Headers: []kafka.Header{ + { + Key: "x-request-id", + Value: []byte("999"), + }, + { + Key: "extra-header", + Value: []byte("77"), + }, + }, + }, + headers: map[string][]byte{ + "x-b3-traceid": []byte("2"), + "x-request-id": []byte("1"), + "x-user-id": []byte("userID1"), + "x-application-trail": []byte("trail"), + }, + }, + want: kafka.Message{ + Headers: []kafka.Header{ + { + Key: "x-b3-traceid", + Value: []byte("2"), + }, + { + Key: "x-request-id", + Value: []byte("1"), + }, + { + Key: "x-user-id", + Value: []byte("userID1"), + }, + { + Key: "x-application-trail", + Value: []byte("trail"), + }, + { + Key: "extra-header", + Value: []byte("77"), + }, + }, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + + got := addHeaders(tt.args.kafkaMessage, tt.args.headers) + + require.ElementsMatch(t, tt.want.Headers, got.Headers, "RecordHeaders do not match") + }) + } +} + +func TestMessage_Value(t *testing.T) { + defer recoverThenFail(t) + + valueStr := "here is some string" + m := Message{ + value: []byte(valueStr), + } + got := m.Value() + require.NotSame(t, m.value, got, "should not return the same reference") + require.Equal(t, valueStr, string(got), "should return the same string representation") +} + +func TestMessage_Value_HandleNil(t *testing.T) { + defer recoverThenFail(t) + + m := Message{ + value: nil, + } + got := m.Value() + require.Nil(t, got, "should have returned nil since underlying data is nil") +} diff --git a/mocks/confluent/kafka_consumer.go b/mocks/confluent/kafka_consumer.go new file mode 100644 index 0000000..4c71479 --- /dev/null +++ b/mocks/confluent/kafka_consumer.go @@ -0,0 +1,138 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: github.com/zillow/zkafka (interfaces: KafkaConsumer) + +// Package mock_confluent is a generated GoMock package. +package mock_confluent + +import ( + reflect "reflect" + time "time" + + kafka "github.com/confluentinc/confluent-kafka-go/v2/kafka" + gomock "github.com/golang/mock/gomock" +) + +// MockKafkaConsumer is a mock of KafkaConsumer interface. +type MockKafkaConsumer struct { + ctrl *gomock.Controller + recorder *MockKafkaConsumerMockRecorder +} + +// MockKafkaConsumerMockRecorder is the mock recorder for MockKafkaConsumer. +type MockKafkaConsumerMockRecorder struct { + mock *MockKafkaConsumer +} + +// NewMockKafkaConsumer creates a new mock instance. +func NewMockKafkaConsumer(ctrl *gomock.Controller) *MockKafkaConsumer { + mock := &MockKafkaConsumer{ctrl: ctrl} + mock.recorder = &MockKafkaConsumerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockKafkaConsumer) EXPECT() *MockKafkaConsumerMockRecorder { + return m.recorder +} + +// Assignment mocks base method. +func (m *MockKafkaConsumer) Assignment() ([]kafka.TopicPartition, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Assignment") + ret0, _ := ret[0].([]kafka.TopicPartition) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Assignment indicates an expected call of Assignment. +func (mr *MockKafkaConsumerMockRecorder) Assignment() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Assignment", reflect.TypeOf((*MockKafkaConsumer)(nil).Assignment)) +} + +// AssignmentLost mocks base method. +func (m *MockKafkaConsumer) AssignmentLost() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "AssignmentLost") + ret0, _ := ret[0].(bool) + return ret0 +} + +// AssignmentLost indicates an expected call of AssignmentLost. +func (mr *MockKafkaConsumerMockRecorder) AssignmentLost() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AssignmentLost", reflect.TypeOf((*MockKafkaConsumer)(nil).AssignmentLost)) +} + +// Close mocks base method. +func (m *MockKafkaConsumer) Close() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Close") + ret0, _ := ret[0].(error) + return ret0 +} + +// Close indicates an expected call of Close. +func (mr *MockKafkaConsumerMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockKafkaConsumer)(nil).Close)) +} + +// Commit mocks base method. +func (m *MockKafkaConsumer) Commit() ([]kafka.TopicPartition, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Commit") + ret0, _ := ret[0].([]kafka.TopicPartition) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Commit indicates an expected call of Commit. +func (mr *MockKafkaConsumerMockRecorder) Commit() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Commit", reflect.TypeOf((*MockKafkaConsumer)(nil).Commit)) +} + +// ReadMessage mocks base method. +func (m *MockKafkaConsumer) ReadMessage(arg0 time.Duration) (*kafka.Message, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ReadMessage", arg0) + ret0, _ := ret[0].(*kafka.Message) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ReadMessage indicates an expected call of ReadMessage. +func (mr *MockKafkaConsumerMockRecorder) ReadMessage(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReadMessage", reflect.TypeOf((*MockKafkaConsumer)(nil).ReadMessage), arg0) +} + +// StoreOffsets mocks base method. +func (m *MockKafkaConsumer) StoreOffsets(arg0 []kafka.TopicPartition) ([]kafka.TopicPartition, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "StoreOffsets", arg0) + ret0, _ := ret[0].([]kafka.TopicPartition) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// StoreOffsets indicates an expected call of StoreOffsets. +func (mr *MockKafkaConsumerMockRecorder) StoreOffsets(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StoreOffsets", reflect.TypeOf((*MockKafkaConsumer)(nil).StoreOffsets), arg0) +} + +// SubscribeTopics mocks base method. +func (m *MockKafkaConsumer) SubscribeTopics(arg0 []string, arg1 kafka.RebalanceCb) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SubscribeTopics", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// SubscribeTopics indicates an expected call of SubscribeTopics. +func (mr *MockKafkaConsumerMockRecorder) SubscribeTopics(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubscribeTopics", reflect.TypeOf((*MockKafkaConsumer)(nil).SubscribeTopics), arg0, arg1) +} diff --git a/mocks/confluent/kafka_producer.go b/mocks/confluent/kafka_producer.go new file mode 100644 index 0000000..795b5f9 --- /dev/null +++ b/mocks/confluent/kafka_producer.go @@ -0,0 +1,61 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: github.com/zillow/zkafka (interfaces: KafkaProducer) + +// Package mock_confluent is a generated GoMock package. +package mock_confluent + +import ( + reflect "reflect" + + kafka "github.com/confluentinc/confluent-kafka-go/v2/kafka" + gomock "github.com/golang/mock/gomock" +) + +// MockKafkaProducer is a mock of KafkaProducer interface. +type MockKafkaProducer struct { + ctrl *gomock.Controller + recorder *MockKafkaProducerMockRecorder +} + +// MockKafkaProducerMockRecorder is the mock recorder for MockKafkaProducer. +type MockKafkaProducerMockRecorder struct { + mock *MockKafkaProducer +} + +// NewMockKafkaProducer creates a new mock instance. +func NewMockKafkaProducer(ctrl *gomock.Controller) *MockKafkaProducer { + mock := &MockKafkaProducer{ctrl: ctrl} + mock.recorder = &MockKafkaProducerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockKafkaProducer) EXPECT() *MockKafkaProducerMockRecorder { + return m.recorder +} + +// Close mocks base method. +func (m *MockKafkaProducer) Close() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Close") +} + +// Close indicates an expected call of Close. +func (mr *MockKafkaProducerMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockKafkaProducer)(nil).Close)) +} + +// Produce mocks base method. +func (m *MockKafkaProducer) Produce(arg0 *kafka.Message, arg1 chan kafka.Event) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Produce", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// Produce indicates an expected call of Produce. +func (mr *MockKafkaProducerMockRecorder) Produce(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Produce", reflect.TypeOf((*MockKafkaProducer)(nil).Produce), arg0, arg1) +} diff --git a/mocks/kreader.go b/mocks/kreader.go new file mode 100644 index 0000000..70d7cc4 --- /dev/null +++ b/mocks/kreader.go @@ -0,0 +1,20 @@ +package mock_zkafka + +//go:generate mockgen -source=kreader.go -package mock_zkafka -mock_names reader=MockReader -destination=./mock_reader.go + +import ( + "context" + + "github.com/zillow/zkafka" +) + +var ( + _ reader = (*zkafka.KReader)(nil) + _ zkafka.Reader = (reader)(nil) +) + +type reader interface { + Read(ctx context.Context) (*zkafka.Message, error) + Close() error + Assignments(_ context.Context) ([]zkafka.Assignment, error) +} diff --git a/mocks/kwriter.go b/mocks/kwriter.go new file mode 100644 index 0000000..6dc2dc1 --- /dev/null +++ b/mocks/kwriter.go @@ -0,0 +1,21 @@ +package mock_zkafka + +//go:generate mockgen -source=kwriter.go -package mock_zkafka -mock_names writer=MockWriter -destination=./mock_writer.go + +import ( + "context" + + "github.com/zillow/zkafka" +) + +var ( + _ writer = (*zkafka.KWriter)(nil) + _ zkafka.Writer = (writer)(nil) +) + +type writer interface { + Write(ctx context.Context, value any, opts ...zkafka.WriteOption) (zkafka.Response, error) + WriteKey(ctx context.Context, key string, value any, opts ...zkafka.WriteOption) (zkafka.Response, error) + WriteRaw(ctx context.Context, key *string, value []byte, opts ...zkafka.WriteOption) (zkafka.Response, error) + Close() +} diff --git a/mocks/mock_client.go b/mocks/mock_client.go new file mode 100644 index 0000000..ef89c83 --- /dev/null +++ b/mocks/mock_client.go @@ -0,0 +1,90 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: ./client.go + +// Package mock_zkafka is a generated GoMock package. +package mock_zkafka + +import ( + context "context" + reflect "reflect" + + gomock "github.com/golang/mock/gomock" + zkafka "github.com/zillow/zkafka" +) + +// MockClientProvider is a mock of ClientProvider interface. +type MockClientProvider struct { + ctrl *gomock.Controller + recorder *MockClientProviderMockRecorder +} + +// MockClientProviderMockRecorder is the mock recorder for MockClientProvider. +type MockClientProviderMockRecorder struct { + mock *MockClientProvider +} + +// NewMockClientProvider creates a new mock instance. +func NewMockClientProvider(ctrl *gomock.Controller) *MockClientProvider { + mock := &MockClientProvider{ctrl: ctrl} + mock.recorder = &MockClientProviderMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockClientProvider) EXPECT() *MockClientProviderMockRecorder { + return m.recorder +} + +// Close mocks base method. +func (m *MockClientProvider) Close() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Close") + ret0, _ := ret[0].(error) + return ret0 +} + +// Close indicates an expected call of Close. +func (mr *MockClientProviderMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockClientProvider)(nil).Close)) +} + +// Reader mocks base method. +func (m *MockClientProvider) Reader(ctx context.Context, topicConfig zkafka.ConsumerTopicConfig, opts ...zkafka.ReaderOption) (zkafka.Reader, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, topicConfig} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "Reader", varargs...) + ret0, _ := ret[0].(zkafka.Reader) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Reader indicates an expected call of Reader. +func (mr *MockClientProviderMockRecorder) Reader(ctx, topicConfig interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, topicConfig}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reader", reflect.TypeOf((*MockClientProvider)(nil).Reader), varargs...) +} + +// Writer mocks base method. +func (m *MockClientProvider) Writer(ctx context.Context, topicConfig zkafka.ProducerTopicConfig, opts ...zkafka.WriterOption) (zkafka.Writer, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, topicConfig} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "Writer", varargs...) + ret0, _ := ret[0].(zkafka.Writer) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Writer indicates an expected call of Writer. +func (mr *MockClientProviderMockRecorder) Writer(ctx, topicConfig interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, topicConfig}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Writer", reflect.TypeOf((*MockClientProvider)(nil).Writer), varargs...) +} diff --git a/mocks/mock_lifecycle.go b/mocks/mock_lifecycle.go new file mode 100644 index 0000000..d7d006e --- /dev/null +++ b/mocks/mock_lifecycle.go @@ -0,0 +1,5 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: ./lifecycle.go + +// Package mock_zkafka is a generated GoMock package. +package mock_zkafka diff --git a/mocks/mock_logger.go b/mocks/mock_logger.go new file mode 100644 index 0000000..b888a6b --- /dev/null +++ b/mocks/mock_logger.go @@ -0,0 +1,103 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: logger.go + +// Package mock_zkafka is a generated GoMock package. +package mock_zkafka + +import ( + context "context" + reflect "reflect" + + gomock "github.com/golang/mock/gomock" +) + +// MockLogger is a mock of Logger interface. +type MockLogger struct { + ctrl *gomock.Controller + recorder *MockLoggerMockRecorder +} + +// MockLoggerMockRecorder is the mock recorder for MockLogger. +type MockLoggerMockRecorder struct { + mock *MockLogger +} + +// NewMockLogger creates a new mock instance. +func NewMockLogger(ctrl *gomock.Controller) *MockLogger { + mock := &MockLogger{ctrl: ctrl} + mock.recorder = &MockLoggerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockLogger) EXPECT() *MockLoggerMockRecorder { + return m.recorder +} + +// Debugw mocks base method. +func (m *MockLogger) Debugw(ctx context.Context, msg string, keysAndValues ...interface{}) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, msg} + for _, a := range keysAndValues { + varargs = append(varargs, a) + } + m.ctrl.Call(m, "Debugw", varargs...) +} + +// Debugw indicates an expected call of Debugw. +func (mr *MockLoggerMockRecorder) Debugw(ctx, msg interface{}, keysAndValues ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, msg}, keysAndValues...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Debugw", reflect.TypeOf((*MockLogger)(nil).Debugw), varargs...) +} + +// Errorw mocks base method. +func (m *MockLogger) Errorw(ctx context.Context, msg string, keysAndValues ...interface{}) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, msg} + for _, a := range keysAndValues { + varargs = append(varargs, a) + } + m.ctrl.Call(m, "Errorw", varargs...) +} + +// Errorw indicates an expected call of Errorw. +func (mr *MockLoggerMockRecorder) Errorw(ctx, msg interface{}, keysAndValues ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, msg}, keysAndValues...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Errorw", reflect.TypeOf((*MockLogger)(nil).Errorw), varargs...) +} + +// Infow mocks base method. +func (m *MockLogger) Infow(ctx context.Context, msg string, keysAndValues ...interface{}) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, msg} + for _, a := range keysAndValues { + varargs = append(varargs, a) + } + m.ctrl.Call(m, "Infow", varargs...) +} + +// Infow indicates an expected call of Infow. +func (mr *MockLoggerMockRecorder) Infow(ctx, msg interface{}, keysAndValues ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, msg}, keysAndValues...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Infow", reflect.TypeOf((*MockLogger)(nil).Infow), varargs...) +} + +// Warnw mocks base method. +func (m *MockLogger) Warnw(ctx context.Context, msg string, keysAndValues ...interface{}) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, msg} + for _, a := range keysAndValues { + varargs = append(varargs, a) + } + m.ctrl.Call(m, "Warnw", varargs...) +} + +// Warnw indicates an expected call of Warnw. +func (mr *MockLoggerMockRecorder) Warnw(ctx, msg interface{}, keysAndValues ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, msg}, keysAndValues...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Warnw", reflect.TypeOf((*MockLogger)(nil).Warnw), varargs...) +} diff --git a/mocks/mock_reader.go b/mocks/mock_reader.go new file mode 100644 index 0000000..7431991 --- /dev/null +++ b/mocks/mock_reader.go @@ -0,0 +1,80 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: kreader.go + +// Package mock_zkafka is a generated GoMock package. +package mock_zkafka + +import ( + context "context" + reflect "reflect" + + gomock "github.com/golang/mock/gomock" + zkafka "github.com/zillow/zkafka" +) + +// MockReader is a mock of reader interface. +type MockReader struct { + ctrl *gomock.Controller + recorder *MockReaderMockRecorder +} + +// MockReaderMockRecorder is the mock recorder for MockReader. +type MockReaderMockRecorder struct { + mock *MockReader +} + +// NewMockReader creates a new mock instance. +func NewMockReader(ctrl *gomock.Controller) *MockReader { + mock := &MockReader{ctrl: ctrl} + mock.recorder = &MockReaderMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockReader) EXPECT() *MockReaderMockRecorder { + return m.recorder +} + +// Assignments mocks base method. +func (m *MockReader) Assignments(arg0 context.Context) ([]zkafka.Assignment, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Assignments", arg0) + ret0, _ := ret[0].([]zkafka.Assignment) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Assignments indicates an expected call of Assignments. +func (mr *MockReaderMockRecorder) Assignments(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Assignments", reflect.TypeOf((*MockReader)(nil).Assignments), arg0) +} + +// Close mocks base method. +func (m *MockReader) Close() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Close") + ret0, _ := ret[0].(error) + return ret0 +} + +// Close indicates an expected call of Close. +func (mr *MockReaderMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockReader)(nil).Close)) +} + +// Read mocks base method. +func (m *MockReader) Read(ctx context.Context) (*zkafka.Message, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Read", ctx) + ret0, _ := ret[0].(*zkafka.Message) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Read indicates an expected call of Read. +func (mr *MockReaderMockRecorder) Read(ctx interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Read", reflect.TypeOf((*MockReader)(nil).Read), ctx) +} diff --git a/mocks/mock_writer.go b/mocks/mock_writer.go new file mode 100644 index 0000000..a304214 --- /dev/null +++ b/mocks/mock_writer.go @@ -0,0 +1,108 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: kwriter.go + +// Package mock_zkafka is a generated GoMock package. +package mock_zkafka + +import ( + context "context" + reflect "reflect" + + gomock "github.com/golang/mock/gomock" + zkafka "github.com/zillow/zkafka" +) + +// MockWriter is a mock of writer interface. +type MockWriter struct { + ctrl *gomock.Controller + recorder *MockWriterMockRecorder +} + +// MockWriterMockRecorder is the mock recorder for MockWriter. +type MockWriterMockRecorder struct { + mock *MockWriter +} + +// NewMockWriter creates a new mock instance. +func NewMockWriter(ctrl *gomock.Controller) *MockWriter { + mock := &MockWriter{ctrl: ctrl} + mock.recorder = &MockWriterMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockWriter) EXPECT() *MockWriterMockRecorder { + return m.recorder +} + +// Close mocks base method. +func (m *MockWriter) Close() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Close") +} + +// Close indicates an expected call of Close. +func (mr *MockWriterMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockWriter)(nil).Close)) +} + +// Write mocks base method. +func (m *MockWriter) Write(ctx context.Context, value interface{}, opts ...zkafka.WriteOption) (zkafka.Response, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, value} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "Write", varargs...) + ret0, _ := ret[0].(zkafka.Response) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Write indicates an expected call of Write. +func (mr *MockWriterMockRecorder) Write(ctx, value interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, value}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Write", reflect.TypeOf((*MockWriter)(nil).Write), varargs...) +} + +// WriteKey mocks base method. +func (m *MockWriter) WriteKey(ctx context.Context, key string, value interface{}, opts ...zkafka.WriteOption) (zkafka.Response, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, key, value} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "WriteKey", varargs...) + ret0, _ := ret[0].(zkafka.Response) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// WriteKey indicates an expected call of WriteKey. +func (mr *MockWriterMockRecorder) WriteKey(ctx, key, value interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, key, value}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteKey", reflect.TypeOf((*MockWriter)(nil).WriteKey), varargs...) +} + +// WriteRaw mocks base method. +func (m *MockWriter) WriteRaw(ctx context.Context, key *string, value []byte, opts ...zkafka.WriteOption) (zkafka.Response, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, key, value} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "WriteRaw", varargs...) + ret0, _ := ret[0].(zkafka.Response) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// WriteRaw indicates an expected call of WriteRaw. +func (mr *MockWriterMockRecorder) WriteRaw(ctx, key, value interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, key, value}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteRaw", reflect.TypeOf((*MockWriter)(nil).WriteRaw), varargs...) +} diff --git a/reader.go b/reader.go new file mode 100644 index 0000000..6a24c08 --- /dev/null +++ b/reader.go @@ -0,0 +1,306 @@ +package zkafka + +//go:generate mockgen -package mock_confluent --destination=./mocks/confluent/kafka_consumer.go . KafkaConsumer +//go:generate mockgen --package=mock_zkafka --destination=./mocks/mock_reader.go . Reader + +import ( + "context" + "sync" + "time" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/pkg/errors" +) + +//// go:generate mockgen -destination=./mocks/mock_metrics.go -source=reader.go + +// Reader is the convenient interface for kafka KReader +type Reader interface { + Read(ctx context.Context) (*Message, error) + Close() error +} + +// static type checking for the convenient Reader interface +var _ Reader = (*KReader)(nil) + +type KafkaConsumer interface { + SubscribeTopics(topics []string, rebalanceCb kafka.RebalanceCb) error + ReadMessage(timeout time.Duration) (*kafka.Message, error) + Commit() ([]kafka.TopicPartition, error) + StoreOffsets(offsets []kafka.TopicPartition) (storedOffsets []kafka.TopicPartition, err error) + Close() error + Assignment() (partitions []kafka.TopicPartition, err error) + AssignmentLost() bool +} + +var _ KafkaConsumer = (*kafka.Consumer)(nil) + +// KReader is a Reader implementation which allows for the subscription to multiple topics. It provides methods +// for consuming messages from its subscribed topics and assigned partitions. +type KReader struct { + consumer KafkaConsumer + topicConfig ConsumerTopicConfig + isClosed bool + + fmtter Formatter + logger Logger + lifecycle LifecycleHooks + once sync.Once + tCommitMgr *topicCommitMgr +} + +// newReader makes a new reader based on the configurations +func newReader(conf Config, topicConfig ConsumerTopicConfig, provider confluentConsumerProvider, logger Logger, prefix string) (*KReader, error) { + confluentConfig := makeConsumerConfig(conf, topicConfig, prefix) + consumer, err := provider(confluentConfig) + if err != nil { + return nil, err + } + + fmtter, err := getFormatter(topicConfig) + if err != nil { + return nil, err + } + return &KReader{ + consumer: consumer, + fmtter: fmtter, + topicConfig: topicConfig, + logger: logger, + tCommitMgr: newTopicCommitMgr(), + }, nil +} + +// Read consumes a single message at a time. Blocks until a message is returned or some +// non-fatal error occurs in which case a nil message is returned +func (r *KReader) Read(ctx context.Context) (*Message, error) { + r.once.Do(func() { + rebalanceCb := r.getRebalanceCb() + err := r.consumer.SubscribeTopics(r.topicConfig.topics(), rebalanceCb) + if err != nil { + r.logger.Errorw(ctx, "Failed to subscribe to topic", "topics", r.topicConfig.topics(), "error", err) + r.isClosed = true + } + }) + if r.isClosed { + return nil, errors.New("reader closed") + } + kmsg, err := r.consumer.ReadMessage(time.Duration(*r.topicConfig.ReadTimeoutMillis) * time.Millisecond) + if err != nil { + switch v := err.(type) { + case kafka.Error: + // timeouts occur (because the assigned partitions aren't being written to, lack of activity, etc.). We'll + // log them for debugging purposes + if v.Code() == kafka.ErrTimedOut { + r.logger.Debugw(ctx, "timed out on read", "topics", r.topicConfig.topics()) + return nil, nil + } + if v.IsRetriable() { + r.logger.Debugw(ctx, "Retryable error occurred", "topics", r.topicConfig.topics(), "error", v) + return nil, nil + } + return nil, errors.Wrap(err, "failed to read kafka message") + } + return nil, errors.Wrap(err, "failed to read kafka message") + } + if kmsg == nil { + return nil, nil + } + return r.mapMessage(ctx, *kmsg), nil +} + +// Close terminates the consumer. This will gracefully unsubscribe +// the consumer from the kafka topic (which includes properly +// revoking the assigned partitions) +func (r *KReader) Close() error { + if r.isClosed { + return nil + } + r.isClosed = true + err := r.consumer.Close() + if err != nil { + return errors.Wrap(err, "failed to close kafka reader") + } + return nil +} + +// Assignments returns the current partition assignments for the kafka consumer +func (r *KReader) Assignments(_ context.Context) ([]Assignment, error) { + assignments, err := r.consumer.Assignment() + if err != nil { + return nil, errors.Wrap(err, "failed to get assignments") + } + topicPartitions := make([]Assignment, 0, len(assignments)) + for _, tp := range assignments { + if tp.Topic == nil { + continue + } + topicPartitions = append(topicPartitions, Assignment{ + Partition: tp.Partition, + Topic: *tp.Topic, + }) + } + return topicPartitions, nil +} + +func (r *KReader) removeInWork(offset kafka.TopicPartition) { + topicName := getTopicName(offset.Topic) + c := r.tCommitMgr.get(topicName) + c.RemoveInWork(offset) +} + +// mapMessage is responsible for mapping the confluent kafka.Message to a zkafka.Message. +func (r *KReader) mapMessage(_ context.Context, msg kafka.Message) *Message { + headerMap := headers(msg) + + topicName := getTopicName(msg.TopicPartition.Topic) + c := r.tCommitMgr.get(topicName) + c.PushInWork(msg.TopicPartition) + + partition := msg.TopicPartition.Partition + offset := int64(msg.TopicPartition.Offset) + return &Message{ + Key: string(msg.Key), + isKeyNil: msg.Key == nil, + Headers: headerMap, + Partition: partition, + Topic: topicName, + GroupID: r.topicConfig.GroupID, + Offset: offset, + topicPartition: msg.TopicPartition, + TimeStamp: msg.Timestamp, + doneFunc: func(ctx context.Context) { + c.PushCompleted(msg.TopicPartition) + commitOffset := c.TryPop(ctx, partition) + if commitOffset == nil { + r.logger.Debugw(ctx, "Message complete, but can't commit yet", "topicName", topicName, "groupID", r.topicConfig.GroupID, "partition", partition, "offset", offset) + return + } + + if commitOffset.Error != nil { + r.logger.Errorw(ctx, "Message complete, but can't commit because of error", "commitOffset", commitOffset) + return + } + + if commitOffset.Offset < 0 { + r.logger.Errorw(ctx, "Message complete, but can't commit because offset < 0", "commitOffset", commitOffset) + return + } + + // https://github.com/confluentinc/confluent-kafka-go/v2/blob/master/kafka/consumer.go#L297 + // https://github.com/confluentinc/confluent-kafka-go/v2/issues/656 + commitOffset.Offset++ + + _, err := r.consumer.StoreOffsets([]kafka.TopicPartition{*commitOffset}) + r.logger.Debugw(ctx, "Stored offsets", "offset", commitOffset, "groupID", r.topicConfig.GroupID) + if err != nil { + r.logger.Errorw(ctx, "Error storing offsets", "topicName", topicName, "groupID", r.topicConfig.GroupID, "partition", partition, "offset", offset, "error", err) + } + }, + value: msg.Value, + fmt: r.fmtter, + } +} + +// getRebalanceCb returns a callback which can be used during rebalances. +// It previously attempted to do one final, explict commit of stored offsets. +// This was unncessary per the mantainer of librdkafka (https://github.com/confluentinc/librdkafka/issues/1829#issuecomment-393427324) +// since when using auto.offset.commit=true (which this library does) the offsets are commit at configured intervals, during close and finally during rebalance. +// +// We do however, want to attempt to let current work complete before allowing a rebalance (so we check the in progress heap) for up to 10 seconds. +// +// This is part of the commit management strategy per guidance here https://docs.confluent.io/platform/current/clients/consumer.html#offset-management +// commit when partitions are revoked +func (r *KReader) getRebalanceCb() kafka.RebalanceCb { + ctx := context.Background() + rebalanceCb := func(_ *kafka.Consumer, event kafka.Event) error { + switch e := event.(type) { + case kafka.AssignedPartitions: + r.logger.Infow(ctx, "Assigned partitions event received", "event", e, "topics", r.topicConfig.topics(), "groupID", r.topicConfig.GroupID) + case kafka.RevokedPartitions: + r.logger.Infow(ctx, "Revoked partitions event received", "event", e, "topics", r.topicConfig.topics(), "groupID", r.topicConfig.GroupID) + + // Usually, the rebalance callback for `RevokedPartitions` is called + // just before the partitions are revoked. We can be certain that a + // partition being revoked is not yet owned by any other consumer. + // This way, logic like storing any pending offsets or committing + // offsets can be handled. + // However, there can be cases where the assignment is lost + // involuntarily. In this case, the partition might already be owned + // by another consumer, and operations including committing + // offsets may not work. (this part of the check comes from this confluent-kafka-go example)[https://github.com/confluentinc/confluent-kafka-go/blob/master/examples/consumer_rebalance_example/consumer_rebalance_example.go] + if r.consumer.AssignmentLost() { + r.logger.Infow(ctx, "Assignment lost prior to revoke (possibly because client was closed)", "event", e, "topics", r.topicConfig.topics(), "groupID", r.topicConfig.GroupID) + return nil + } + + // we're going to try and finish processing the inwork messages. + // We'll do this by checking the commit manager. When the RevokedPartitions event is emitted, + // subsequent ReadMessage() calls from a consumer will return nil, allowing the commit manager to get widdled down. + ctxNew, cancel := context.WithTimeout(ctx, time.Second*10) + defer cancel() + for { + var inWorkCount int64 = 0 + for _, t := range getTopics(e.Partitions) { + cmtMgr := r.tCommitMgr.get(t) + inWorkCount += cmtMgr.InWorkCount() + } + if inWorkCount == 0 { + break + } + if ctxNew.Err() != nil { + r.logger.Warnw(ctx, "Incomplete inwork drain during revokedPartitions event", "event", e, "inWorkCount", inWorkCount, "error", ctxNew.Err(), "topics", r.topicConfig.topics(), "groupID", r.topicConfig.GroupID) + break + } + // we're polling the commit manager, we'll do a small pause to avoid a busy loop + time.Sleep(time.Microsecond * 1) + } + } + return nil + } + return rebalanceCb +} + +func getTopics(partitions []kafka.TopicPartition) []string { + uniqueTopics := map[string]struct{}{} + for _, p := range partitions { + if p.Topic == nil { + continue + } + uniqueTopics[*p.Topic] = struct{}{} + } + topics := make([]string, 0, len(uniqueTopics)) + for t := range uniqueTopics { + topics = append(topics, t) + } + return topics +} + +func getTopicName(topicName *string) string { + topic := "" + if topicName != nil { + topic = *topicName + } + return topic +} + +// ReaderOption is a function that modify the KReader configurations +type ReaderOption func(*KReader) + +// RFormatterOption sets the formatter for this reader +func RFormatterOption(fmtter Formatter) ReaderOption { + return func(r *KReader) { + if fmtter != nil { + r.fmtter = fmtter + } + } +} + +type TopicPartition struct { + Partition int32 + Offset int64 +} + +type Assignment struct { + Partition int32 + Topic string +} diff --git a/reader_test.go b/reader_test.go new file mode 100644 index 0000000..02beaf3 --- /dev/null +++ b/reader_test.go @@ -0,0 +1,588 @@ +package zkafka + +import ( + "context" + "errors" + "fmt" + "sync" + "testing" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/stretchr/testify/require" + mock_confluent "github.com/zillow/zkafka/mocks/confluent" + + "github.com/golang/mock/gomock" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +func TestReader_Read_NilReturn(t *testing.T) { + defer recoverThenFail(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).Times(1) + mockConsumer.EXPECT().ReadMessage(gomock.Any()).Times(1) + + topicConfig := ConsumerTopicConfig{ + AutoCommitIntervalMs: ptr(10), + ReadTimeoutMillis: ptr(1000), + Formatter: zfmt.StringFmt, + } + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &NoopLogger{}, "") + + got, err := r.Read(context.TODO()) + require.NoError(t, err) + require.Nil(t, got) +} + +func TestReader_Read(t *testing.T) { + defer recoverThenFail(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).Times(1) + mockConsumer.EXPECT().ReadMessage(gomock.Any()).Return(&kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Topic: ptr("topic"), + }, + Value: []byte("test"), + }, nil).Times(1) + + topicConfig := ConsumerTopicConfig{ + AutoCommitIntervalMs: ptr(10), + ReadTimeoutMillis: ptr(1000), + Formatter: zfmt.StringFmt, + } + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &NoopLogger{}, "") + + got, err := r.Read(context.TODO()) + require.NoError(t, err) + require.NotNil(t, got.fmt, "message should have formatter") + require.NotEmpty(t, string(got.value), "expect a non-empty value on call to read") +} + +func TestReader_Read_Error(t *testing.T) { + readMessageErrors := []error{ + errors.New("error"), + kafka.NewError(kafka.ErrAllBrokersDown, "error", true), + } + for _, readMessageError := range readMessageErrors { + t.Run(readMessageError.Error(), func(t *testing.T) { + defer recoverThenFail(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).Times(1) + mockConsumer.EXPECT().ReadMessage(gomock.Any()).Return(&kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Topic: ptr("topic"), + }, + Value: []byte("test"), + }, nil).Return(nil, readMessageError).Times(1) + + topicConfig := ConsumerTopicConfig{ + Topic: "topic", + AutoCommitIntervalMs: ptr(10), + ReadTimeoutMillis: ptr(10), + Formatter: zfmt.StringFmt, + } + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &NoopLogger{}, "") + + got, err := r.Read(context.TODO()) + require.Error(t, err) + require.Nil(t, got) + }) + } +} + +func TestReader_Read_TimeoutError(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).Times(1) + mockConsumer.EXPECT().ReadMessage(gomock.Any()).Return(&kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Topic: ptr("topic"), + }, + Value: []byte("test"), + }, nil).Return(nil, kafka.NewError(kafka.ErrTimedOut, "error", false)).Times(1) + + topicConfig := ConsumerTopicConfig{ + Topic: "topic", + AutoCommitIntervalMs: ptr(10), + Formatter: zfmt.StringFmt, + ReadTimeoutMillis: ptr(1000), + } + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &NoopLogger{}, "") + + got, err := r.Read(context.TODO()) + require.NoError(t, err, "expect no error to be returned on timeout") + require.Nil(t, got, "expect a timeout to result in no read message") +} + +func TestReader_Read_SubscriberError(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).Return(errors.New("subscriber error")).Times(1) + topicConfig := ConsumerTopicConfig{ + AutoCommitIntervalMs: ptr(10), + Formatter: zfmt.ProtoRawFmt, + } + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &NoopLogger{}, "") + + _, err := r.Read(context.TODO()) + require.Error(t, err, "expect an error to bubble up on Read because of subscribe error") +} + +func TestReader_Read_CloseError(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().Close().Return(errors.New("close error")).Times(1) + l := mockLogger{} + topicConfig := ConsumerTopicConfig{ + AutoCommitIntervalMs: ptr(10), + Formatter: zfmt.StringFmt, + } + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &l, "") + + err := r.Close() + + require.Error(t, err) +} + +func TestReader_ReadWhenConnectionIsClosed(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).Times(1) + mockConsumer.EXPECT().Close().Times(1) + + topicConfig := ConsumerTopicConfig{ + AutoCommitIntervalMs: ptr(10), + Formatter: zfmt.StringFmt, + } + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &NoopLogger{}, "") + + err := r.Close() + require.NoError(t, err) + _, err = r.Read(context.TODO()) + require.Error(t, err, "KReader.Read() message should return error due to connection lost") +} + +func Test_newReader(t *testing.T) { + type args struct { + conf Config + topicConfig ConsumerTopicConfig + consumeProvider confluentConsumerProvider + } + tests := []struct { + name string + args args + wantErr bool + }{ + { + name: "custom formatter, no error. It is implied that user will supply formatter later", + args: args{ + topicConfig: ConsumerTopicConfig{ + Formatter: zfmt.FormatterType("custom"), + }, + consumeProvider: defaultConfluentConsumerProvider{}.NewConsumer, + }, + wantErr: false, + }, + { + name: "invalid formatter", + args: args{ + consumeProvider: defaultConfluentConsumerProvider{}.NewConsumer, + topicConfig: ConsumerTopicConfig{ + Formatter: zfmt.FormatterType("invalid_fmt"), + }, + }, + wantErr: true, + }, + { + name: "valid formatter but has error when creating NewConsumer", + args: args{ + consumeProvider: mockConfluentConsumerProvider{err: true}.NewConsumer, + }, + wantErr: true, + }, + { + name: "minimum config with formatter", + args: args{ + consumeProvider: defaultConfluentConsumerProvider{}.NewConsumer, + topicConfig: ConsumerTopicConfig{ + Formatter: zfmt.StringFmt, + }, + }, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + _, err := newReader(tt.args.conf, tt.args.topicConfig, tt.args.consumeProvider, &NoopLogger{}, "") + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + }) + } +} + +func Test_ProcessMessage(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + dupMessage := kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Topic: ptr("test-topic"), + Partition: 10, + Offset: 99, + }, + } + + l := mockLogger{} + + topicConfig := ConsumerTopicConfig{ + GroupID: "test-group", + AutoCommitIntervalMs: ptr(10), + Formatter: zfmt.StringFmt, + } + m := mockConfluentConsumerProvider{ + c: mock_confluent.NewMockKafkaConsumer(ctrl), + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &l, "") + got := r.mapMessage(context.Background(), dupMessage) + + require.Equal(t, got.Partition, dupMessage.TopicPartition.Partition) + require.Equal(t, got.Offset, int64(dupMessage.TopicPartition.Offset)) + require.Empty(t, got.Key) + require.Empty(t, l.InfoStr, "no info calls should have been called") +} + +func Test_ProcessMultipleMessagesFromDifferentTopics_UpdatesInternalStateProperly(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + topic1 := "test-topic1" + topic2 := "test-topic2" + msgs := []kafka.Message{ + {TopicPartition: kafka.TopicPartition{Topic: &topic1, Partition: 10, Offset: 99}}, + {TopicPartition: kafka.TopicPartition{Topic: &topic2, Partition: 5, Offset: 99}}, + {TopicPartition: kafka.TopicPartition{Topic: &topic2, Partition: 5, Offset: 100}}, + } + + l := mockLogger{} + + topicConfig := ConsumerTopicConfig{ + GroupID: "test-group", + AutoCommitIntervalMs: ptr(10), + Formatter: zfmt.StringFmt, + } + m := mockConfluentConsumerProvider{ + c: mock_confluent.NewMockKafkaConsumer(ctrl), + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &l, "") + for _, msg := range msgs { + got := r.mapMessage(context.Background(), msg) + require.Equal(t, got.Partition, msg.TopicPartition.Partition) + require.Equal(t, got.Offset, int64(msg.TopicPartition.Offset)) + require.Empty(t, got.Key) + } + + expectedManagedTopicsCount := 2 + require.Len(t, r.tCommitMgr.topicToCommitMgr, expectedManagedTopicsCount, "expected to have 2 managed topics") +} + +func Test_ProcessMessage_StoreOffsetError(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + topicConfig := ConsumerTopicConfig{ + GroupID: "test-group", + Topic: "test-topic", + AutoCommitIntervalMs: ptr(10), + Formatter: zfmt.StringFmt, + } + + dupMessage := kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Topic: ptr("test-topic"), + Partition: 10, + Offset: 99, + }, + } + + l := mockLogger{} + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().StoreOffsets(gomock.Any()).DoAndReturn(func(m []kafka.TopicPartition) ([]kafka.TopicPartition, error) { + return nil, errors.New("error occurred on store") + }).Times(1) + + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &l, "") + + mgr := newTopicCommitMgr() + cmgr := mgr.get(*dupMessage.TopicPartition.Topic) + cmgr.PushInWork(dupMessage.TopicPartition) + r.tCommitMgr = mgr + // assert that we won't block if Done on message is called + msg := r.mapMessage(context.Background(), dupMessage) + msg.Done() + + // if we can't store the offset, we expect a log to be set with no other errors raised + require.Len(t, l.ErrorStr, 1, "expected an error log on fail to store offsets") +} + +func Test_ProcessMessage_SetError(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + topicConfig := ConsumerTopicConfig{ + GroupID: "test-group", + Topic: "topic", + AutoCommitIntervalMs: ptr(10), + Formatter: zfmt.StringFmt, + } + + dupMessage := kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Topic: ptr(topicConfig.Topic), + Partition: 10, + Offset: 99, + }, + } + + mockConsumer := mock_confluent.NewMockKafkaConsumer(ctrl) + mockConsumer.EXPECT().StoreOffsets(gomock.Any()).DoAndReturn(func(m []kafka.TopicPartition) ([]kafka.TopicPartition, error) { + require.Len(t, m, 1, "expect one topicPartition to be passed") + tp := m[0] + require.Equal(t, dupMessage.TopicPartition.Partition, tp.Partition) + require.Equal(t, dupMessage.TopicPartition.Offset+1, tp.Offset) + return m, nil + }).Times(1) + + l := mockLogger{} + + m := mockConfluentConsumerProvider{ + c: mockConsumer, + }.NewConsumer + r, _ := newReader(Config{}, topicConfig, m, &l, "") + mgr := newTopicCommitMgr() + cmgr := mgr.get(*dupMessage.TopicPartition.Topic) + cmgr.PushInWork(dupMessage.TopicPartition) + r.tCommitMgr = mgr + msg := r.mapMessage(context.Background(), dupMessage) + msg.Done() +} + +func TestReader_CloseCalledMultipleTimesDoesntOnlyTriesToCloseKafkaConsumerOnce(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + // setup mockKafkaConsumer to mimic behavior of real consumer (only being able to be closed once) + mockKafkaConsumer1 := mock_confluent.NewMockKafkaConsumer(ctrl) + gomock.InOrder( + mockKafkaConsumer1.EXPECT().Close().Return(nil), + mockKafkaConsumer1.EXPECT().Close().Return(errors.New("cant call twice")).AnyTimes(), + ) + + w := &KReader{ + consumer: mockKafkaConsumer1, + logger: NoopLogger{}, + } + err := w.Close() + require.NoError(t, err) + + err = w.Close() + require.NoError(t, err) +} + +func TestReader_Close(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockKafkaConsumer1 := mock_confluent.NewMockKafkaConsumer(ctrl) + mockKafkaConsumer1.EXPECT().Close().Return(nil) + mockKafkaConsumer2 := mock_confluent.NewMockKafkaConsumer(ctrl) + mockKafkaConsumer2.EXPECT().Close().Return(errors.New("close error")) + + type fields struct { + Mutex *sync.Mutex + kafkaConsumer KafkaConsumer + } + tests := []struct { + name string + fields fields + wantErr bool + }{ + { + name: "basic closure", + fields: fields{ + kafkaConsumer: mockKafkaConsumer1, + }, + wantErr: false, + }, + { + name: "basic closure with fake close error", + fields: fields{ + kafkaConsumer: mockKafkaConsumer2, + }, + wantErr: true, + }, + } + + for _, tt := range tests { + w := &KReader{ + consumer: tt.fields.kafkaConsumer, + logger: NoopLogger{}, + } + err := w.Close() + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + require.True(t, w.isClosed, "KReader.Close() should have been closed") + } +} + +func TestKReader_getRebalanceCb_RevokedPartitionsWhenAssignedLostDoesntErrorDuringStandardExecution(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockKafkaConsumer1 := mock_confluent.NewMockKafkaConsumer(ctrl) + mockKafkaConsumer1.EXPECT().AssignmentLost().Return(true) + + w := &KReader{ + consumer: mockKafkaConsumer1, + logger: NoopLogger{}, + } + + callback := w.getRebalanceCb() + err := callback(nil, kafka.RevokedPartitions{}) + require.NoError(t, err) +} + +func TestKReader_getRebalanceCb_RevokedPartitionsWhenAssignedNotLostDoesntErrorDuringStandardExecution(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockKafkaConsumer1 := mock_confluent.NewMockKafkaConsumer(ctrl) + mockKafkaConsumer1.EXPECT().AssignmentLost().Return(false) + + w := &KReader{ + consumer: mockKafkaConsumer1, + logger: NoopLogger{}, + } + + callback := w.getRebalanceCb() + err := callback(nil, kafka.RevokedPartitions{}) + require.NoError(t, err) +} + +func TestKReader_getRebalanceCb_RecognizedKafkaErrorOccursDuringNonRebalanceCbEvent(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + mockKafkaConsumer1 := mock_confluent.NewMockKafkaConsumer(ctrl) + mockKafkaConsumer1.EXPECT().AssignmentLost().Return(false).Times(0) + + w := &KReader{ + consumer: mockKafkaConsumer1, + logger: NoopLogger{}, + } + + callback := w.getRebalanceCb() + err := callback(nil, kafka.PartitionEOF{}) + require.NoError(t, err) +} + +func Test_getTopics(t *testing.T) { + type args struct { + partitions []kafka.TopicPartition + } + tests := []struct { + name string + args args + want []string + }{ + { + name: "ignore_nil_topic_names", + args: args{ + []kafka.TopicPartition{ + { + Topic: nil, + }, + { + Topic: ptr("a"), + }, + { + Topic: ptr("b"), + }, + }, + }, + want: []string{"a", "b"}, + }, + { + name: "handle_nil_slice", + args: args{ + partitions: nil, + }, + want: []string{}, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got := getTopics(tt.args.partitions) + require.ElementsMatch(t, tt.want, got) + }) + } +} + +type mockLogger struct { + ErrorStr []string + InfoStr []string + WarnStr []string + DPanicStr []string +} + +func (m *mockLogger) Debugw(_ context.Context, _ string, _ ...any) { +} + +func (m *mockLogger) Infow(_ context.Context, msg string, keysAndValues ...any) { + m.InfoStr = append(m.InfoStr, fmt.Sprint(msg, keysAndValues)) +} + +func (m *mockLogger) Warnw(_ context.Context, msg string, keysAndValues ...any) { + m.WarnStr = append(m.WarnStr, fmt.Sprint(msg, keysAndValues)) +} + +func (m *mockLogger) Errorw(_ context.Context, msg string, keysAndValues ...any) { + m.ErrorStr = append(m.ErrorStr, fmt.Sprint(msg, keysAndValues)) +} diff --git a/test/integration_test.go b/test/integration_test.go new file mode 100644 index 0000000..acad4e6 --- /dev/null +++ b/test/integration_test.go @@ -0,0 +1,1265 @@ +//go:build integration +// +build integration + +package test + +import ( + "context" + "errors" + "fmt" + "math/rand" + "os" + "slices" + "sync" + "testing" + "time" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/google/go-cmp/cmp" + "github.com/google/uuid" + "github.com/stretchr/testify/require" + "github.com/zillow/zkafka" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" + "golang.org/x/sync/errgroup" +) + +// TestKafkaClientsCanReadOwnWritesAndBehaveProperlyAfterRestart will test that a kafka consumer can properly read messages +// written by the kafka producer. It will additionally, confirm that when a group is restarted that it starts off where +// it left off (addressing an off by 1 bug seen with an earlier version) +// +// The following steps are followed +// 1. Create a new consumer group that is reading from the topic +// 1. Write two messages to the topic +// 1. At this point the first message should be returned to the consumer. Assert based on offsets and message payload +// 1. Close the reader +// 1. Restart a consumer (being sure to reuse the same consumer group from before) +// 1. Read another message. Assert its the second written message (first was already read and committed) +func TestKafkaClientsCanReadOwnWritesAndBehaveProperlyAfterRestart(t *testing.T) { + ctx := context.Background() + topic := "integration-test-topic-2" + uuid.NewString() + bootstrapServer := getBootstrap() + + createTopic(t, bootstrapServer, topic, 1) + + groupID := uuid.NewString() + + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, zkafka.LoggerOption(stdLogger{})) + defer func() { require.NoError(t, client.Close()) }() + + writer, err := client.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + }) + consumerTopicConfig := zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("reader-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + GroupID: groupID, + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + } + reader, err := client.Reader(ctx, consumerTopicConfig) + require.NoError(t, err) + + readResponses := make(chan struct { + msg *zkafka.Message + err error + }) + + // helper method for reading from topic in loop until all messages specified in map have been read + // will signal on channel the messages read from topic + funcReadSpecifiedMessages := func(reader zkafka.Reader, msgsToRead map[Msg]struct{}, responses chan struct { + msg *zkafka.Message + err error + }, + ) { + for { + func() { + rmsg, errRead := reader.Read(ctx) + defer func() { + if rmsg == nil { + return + } + rmsg.DoneWithContext(ctx) + }() + if errRead != nil || rmsg == nil { + return + } + gotMsg := Msg{} + err = rmsg.Decode(&gotMsg) + if _, ok := msgsToRead[gotMsg]; ok { + delete(msgsToRead, gotMsg) + } + responses <- struct { + msg *zkafka.Message + err error + }{msg: rmsg, err: err} + if len(msgsToRead) == 0 { + return + } + }() + } + } + + // start the reader before we write messages (otherwise, since its a new consumer group, auto.offset.reset=latest will be started at an offset later than the just written messages). + // Loop in the reader until msg1 appears + msg1 := Msg{Val: "1"} + go funcReadSpecifiedMessages(reader, map[Msg]struct{}{ + msg1: {}, + }, readResponses) + + // write msg1, and msg2 + resWrite1, err := writer.Write(ctx, msg1) + require.NoError(t, err) + + msg2 := Msg{Val: "2"} + resWrite2, err := writer.Write(ctx, msg2) + require.NoError(t, err) + + // reader will send on channel the messages it has read (should just be msg1) + resp := <-readResponses + rmsg1 := resp.msg + + require.NoError(t, resp.err) + require.NotNil(t, rmsg1, "expected written message to be read") + require.Equal(t, int(rmsg1.Offset), int(resWrite1.Offset), "expected read offset to match written") + + gotMsg1 := Msg{} + err = resp.msg.Decode(&gotMsg1) + + require.NoError(t, err) + assertEqual(t, gotMsg1, msg1) + + // close consumer so we can test a new consumer (same group) on restart + err = reader.Close() + require.NoError(t, err) + + // create another reader (consumer) with same consumer group from before. + reader2, err := client.Reader(ctx, consumerTopicConfig) + require.NoError(t, err) + + go funcReadSpecifiedMessages(reader2, map[Msg]struct{}{ + msg2: {}, + }, readResponses) + resp2 := <-readResponses + rmsg2 := resp2.msg + + require.NoError(t, resp2.err) + require.NotNil(t, rmsg2, "expected written message to be read") + + // assert offset is for second message written (no replay of old message) + require.Equal(t, int(rmsg2.Offset), int(resWrite2.Offset), "expected read offset to match written") + + gotMsg2 := Msg{} + err = rmsg2.Decode(&gotMsg2) + + require.NoError(t, err) + assertEqual(t, gotMsg2, msg2) +} + +// Test_RebalanceDoesntCauseDuplicateMessages given a n=messageCount messages written to a topic +// when a consumer joins and starts consuming messages and later when another consumer joins +// then there are no duplicate messages processed. +// +// This is in response to a noted issue where rebalance was prone to replayed messages. +// There are multiple versions of the tests which vary the processing duration +func Test_RebalanceDoesntCauseDuplicateMessages(t *testing.T) { + type testCase struct { + name string + processingDuration time.Duration + messageCount int + } + testCases := []testCase{ + { + name: "processtime-10ms", + processingDuration: time.Millisecond * 10, + messageCount: 1000, + }, + { + name: "processtime-100ms", + processingDuration: time.Millisecond * 100, + messageCount: 200, + }, + { + name: "processtime-1000ms", + processingDuration: time.Millisecond * 1000, + messageCount: 50, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + + ctx := context.Background() + + groupID := uuid.NewString() + bootstrapServer := getBootstrap() + + // create a randomly named topic so we don't interfere with other tests + topic := "topic-" + uuid.NewString() + createTopic(t, bootstrapServer, topic, 2) + + l := stdLogger{} + wclient := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}) + defer func() { require.NoError(t, wclient.Close()) }() + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, zkafka.LoggerOption(l)) + defer func() { require.NoError(t, client.Close()) }() + + writer, err := wclient.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + }) + require.NoError(t, err) + + msg := Msg{ + Val: "sdfds", + } + + // write N messages to topic + msgCount := tc.messageCount + writtenMsgs := map[int32][]int{} + for i := 0; i < msgCount; i++ { + msgResp, err := writer.WriteKey(ctx, uuid.NewString(), msg) + writtenMsgs[msgResp.Partition] = append(writtenMsgs[msgResp.Partition], int(msgResp.Offset)) + require.NoError(t, err) + } + t.Log("Completed writing messages") + + // create work1 which has its own processor + cTopicCfg1 := zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("reader-%s-%s", t.Name(), tc.name), + Topic: topic, + Formatter: zfmt.JSONFmt, + GroupID: groupID, + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + } + wf := zkafka.NewWorkFactory(client, zkafka.WithLogger(l)) + + maxDurationMillis := int(tc.processingDuration / time.Millisecond) + minDurationMillis := int(tc.processingDuration / time.Millisecond) + ctx1, cancel1 := context.WithCancel(ctx) + defer cancel1() + processor1 := &Processor{maxDurationMillis: maxDurationMillis, minDurationMillis: minDurationMillis, l: zkafka.NoopLogger{}} + processor2 := &Processor{maxDurationMillis: maxDurationMillis, minDurationMillis: minDurationMillis, l: zkafka.NoopLogger{}} + + work1 := wf.Create(cTopicCfg1, processor1, + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + if len(processor1.ProcessedMessages()) > 3 && len(processor2.ProcessedMessages()) > 3 { + cancel1() + } + }})) + + // create work2 which has its own processor + ctx2, cancel2 := context.WithCancel(ctx) + defer cancel2() + cTopicCfg2 := cTopicCfg1 + cTopicCfg2.ClientID += "-2" + work2 := wf.Create(cTopicCfg2, processor2, + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + totalProcessed := len(processor1.ProcessedMessages()) + len(processor2.ProcessedMessages()) + if totalProcessed == tc.messageCount { + cancel2() + } + }})) + + t.Log("starting work1") + grp := errgroup.Group{} + grp.Go(func() error { + if err := work1.Run(context.Background(), ctx1.Done()); err != nil { + return err + } + // close reader client so work1 gracefully leaves consumergroup and work2 can finish where it left off. + r1, err := client.Reader(context.Background(), cTopicCfg1) + if err != nil { + return err + } + return r1.Close() + }) + + // wait until processor1 has begun to process messages + for { + if len(processor1.ProcessedMessages()) > 5 { + break + } + time.Sleep(time.Millisecond) + } + + t.Log("starting work2") + grp.Go(func() error { + return work2.Run(context.Background(), ctx2.Done()) + }) + + t.Log("starting exit polling") + grp.Go(func() error { + for { + select { + case <-time.After(3 * time.Minute): + return errors.New("test could not complete before timeout") + case <-ctx2.Done(): + if err1 := ctx1.Err(); err1 != nil { + return nil + } + } + } + }) + err = grp.Wait() + require.NoError(t, err, "Dont expect error returned from workers or poller") + t.Log("work1 and work2 complete") + + // keep track of how many times each message (identified by the topic/partition/offset) is processed + messageProcessCounter := make(map[partition]int) + updateProcessCounter := func(msgs []*zkafka.Message) { + for _, m := range msgs { + key := partition{ + partition: m.Partition, + offset: m.Offset, + topic: m.Topic, + } + if _, ok := messageProcessCounter[key]; !ok { + messageProcessCounter[key] = 0 + } + messageProcessCounter[key] += 1 + } + } + updateProcessCounter(processor1.ProcessedMessages()) + updateProcessCounter(processor2.ProcessedMessages()) + + for key, val := range messageProcessCounter { + require.GreaterOrEqual(t, val, 1, fmt.Sprintf("Message Processed More than Once: partition %d, offset %d, topic %s", key.partition, key.offset, key.topic)) + } + // organized processor1 and processor2 messages into a single map of partition to sorted offset. + // Convenient for comparison with the written messages (which are already sorted) + processedMessages := append(processor1.ProcessedMessages(), processor2.ProcessedMessages()...) + perPartitionMessages := map[int32][]int{} + for _, m := range processedMessages { + perPartitionMessages[m.Partition] = append(perPartitionMessages[m.Partition], int(m.Offset)) + } + for p := range perPartitionMessages { + slices.Sort(perPartitionMessages[p]) + } + require.Equal(t, writtenMsgs, perPartitionMessages, "Expected every message written to topic be processed (no misses)") + }) + } +} + +// Test_WithMultipleTopics_RebalanceDoesntCauseDuplicateMessages given a n=messageCount messages written to two topics +// when a consumer joins and starts consuming messages and later when another consumer joins +// then there are no duplicate messages processed. +func Test_WithMultipleTopics_RebalanceDoesntCauseDuplicateMessages(t *testing.T) { + type testCase struct { + name string + processingDuration time.Duration + messageCount int + } + testCases := []testCase{ + { + name: "processtime-10ms", + processingDuration: time.Millisecond * 10, + messageCount: 1000, + }, + { + name: "processtime-200ms", + processingDuration: time.Millisecond * 200, + messageCount: 300, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + ctx := context.Background() + + groupID := uuid.NewString() + bootstrapServer := getBootstrap() + + // create a randomly named topic1 so we don't interfere with other tests + topic1 := "topic1-" + uuid.NewString() + topic2 := "topic2-" + uuid.NewString() + createTopic(t, bootstrapServer, topic1, 2) + createTopic(t, bootstrapServer, topic2, 2) + + l := stdLogger{} + wclient := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}) + defer func() { require.NoError(t, wclient.Close()) }() + + writer1, err := wclient.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer1-%s-%s", t.Name(), tc.name), + Topic: topic1, + Formatter: zfmt.JSONFmt, + RequestRequiredAcks: ptr("0"), + EnableIdempotence: ptr(false), + }) + require.NoError(t, err) + + writer2, err := wclient.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer2-%s-%s", t.Name(), tc.name), + Topic: topic2, + Formatter: zfmt.JSONFmt, + RequestRequiredAcks: ptr("0"), + EnableIdempotence: ptr(false), + }) + require.NoError(t, err) + + msg := Msg{ + Val: "sdfds", + } + + t.Log("Begin writing to Test Topic") + // write N messages to topic1 + msgCount := tc.messageCount + for i := 0; i < msgCount; i++ { + _, err = writer1.WriteKey(ctx, uuid.NewString(), msg) + require.NoError(t, err) + + _, err = writer2.WriteKey(ctx, uuid.NewString(), msg) + require.NoError(t, err) + } + + cTopicCfg1 := zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("dltReader-%s-%s", t.Name(), tc.name), + Topics: []string{topic1, topic2}, + Formatter: zfmt.JSONFmt, + GroupID: groupID, + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + } + + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, zkafka.LoggerOption(l)) + defer func() { require.NoError(t, client.Close()) }() + + wf := zkafka.NewWorkFactory(client, zkafka.WithLogger(l)) + + maxDurationMillis := int(tc.processingDuration / time.Millisecond) + minDurationMillis := int(tc.processingDuration / time.Millisecond) + processor1 := &Processor{maxDurationMillis: maxDurationMillis, minDurationMillis: minDurationMillis, l: zkafka.NoopLogger{}} + processor2 := &Processor{maxDurationMillis: maxDurationMillis, minDurationMillis: minDurationMillis, l: zkafka.NoopLogger{}} + breakProcessingCondition := func() bool { + uniqueTopics1 := map[string]struct{}{} + for _, m := range processor1.ProcessedMessages() { + uniqueTopics1[m.Topic] = struct{}{} + } + uniqueTopics2 := map[string]struct{}{} + for _, m := range processor2.ProcessedMessages() { + uniqueTopics2[m.Topic] = struct{}{} + } + return len(uniqueTopics1) > 1 && len(uniqueTopics2) > 1 + } + ctx1, cancel1 := context.WithCancel(ctx) + work1 := wf.Create(cTopicCfg1, processor1, + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + if breakProcessingCondition() { + cancel1() + } + }})) + + cTopicCfg2 := cTopicCfg1 + cTopicCfg2.ClientID = fmt.Sprintf("dltReader2-%s-%s", t.Name(), tc.name) + //cTopicCfg2.ClientID += "-2" + ctx2, cancel2 := context.WithCancel(ctx) + work2 := wf.Create(cTopicCfg2, processor2, + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + if breakProcessingCondition() { + cancel2() + } + }})) + + // break out of processing loop when a message has been processed from each subscribed topic + + t.Log("starting work1") + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + defer wg.Done() + work1.Run(ctx1, nil) + }() + + // wait until processor1 has begun to process messages + for { + if len(processor1.ProcessedMessages()) > 5 { + break + } + time.Sleep(time.Millisecond) + } + + t.Log("starting work2") + wg.Add(1) + go func() { + defer wg.Done() + work2.Run(ctx2, nil) + }() + wg.Wait() + + messageProcessCounter := make(map[partition]int) + updateProcessCounter := func(msgs []*zkafka.Message) { + for _, m := range msgs { + key := partition{ + partition: m.Partition, + offset: m.Offset, + topic: m.Topic, + } + if _, ok := messageProcessCounter[key]; !ok { + messageProcessCounter[key] = 0 + } + messageProcessCounter[key] += 1 + } + } + updateProcessCounter(processor1.ProcessedMessages()) + updateProcessCounter(processor2.ProcessedMessages()) + + for key, val := range messageProcessCounter { + require.GreaterOrEqual(t, val, 1, fmt.Sprintf("Message Processed More than Once: partition %d, offset %d, topic %s", key.partition, key.offset, key.topic)) + } + }) + } +} + +// Test_WithConcurrentProcessing_RebalanceDoesntCauseDuplicateMessages given n=messageCount messages written to a topic +// when a consumer joins and starts consuming messages (concurrently) and later when another consumer joins (concurrently) +// then there are no duplicate messages processed. +// +// The consumer's processing times are set to a range as opposed to a specific duration. This allows lookahead processing (where messages +// of higher offsets are processed and completed, potentially, before lower offsets +func Test_WithConcurrentProcessing_RebalanceDoesntCauseDuplicateMessages(t *testing.T) { + type testCase struct { + name string + processingDurationMinMillis int + processingDurationMaxMillis int + messageCount int + } + testCases := []testCase{ + { + name: "processtime-200ms", + processingDurationMinMillis: 0, + processingDurationMaxMillis: 400, + messageCount: 300, + }, + { + name: "processtime-1000ms", + processingDurationMinMillis: 800, + processingDurationMaxMillis: 1200, + messageCount: 100, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + ctx := context.Background() + + groupID := uuid.NewString() + bootstrapServer := getBootstrap() + + // create a randomly named topic so we don't interfere with other tests + topic := "topic-" + uuid.NewString() + t.Logf("Creating topic %s", topic) + createTopic(t, bootstrapServer, topic, 2) + t.Log("Completed creating topic") + + l := zkafka.NoopLogger{} + wclient := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}) + defer func() { require.NoError(t, wclient.Close()) }() + + writer, err := wclient.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer-%s-%s", t.Name(), tc.name), + Topic: topic, + Formatter: zfmt.JSONFmt, + RequestRequiredAcks: ptr("0"), + EnableIdempotence: ptr(false), + }) + require.NoError(t, err) + + msg := Msg{ + Val: "sdfds", + } + + // write N messages to topic + t.Logf("Writing n = %d", tc.messageCount) + msgCount := tc.messageCount + for i := 0; i < msgCount; i++ { + _, err = writer.WriteKey(ctx, uuid.NewString(), msg) + require.NoError(t, err) + } + t.Logf("Completed writing n message") + + cTopicCfg1 := zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("dltReader-%s-%s", t.Name(), tc.name), + Topic: topic, + Formatter: zfmt.JSONFmt, + GroupID: groupID, + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + } + + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, zkafka.LoggerOption(l)) + defer func() { require.NoError(t, client.Close()) }() + + wf := zkafka.NewWorkFactory(client, zkafka.WithLogger(l)) + + processor1 := &Processor{minDurationMillis: tc.processingDurationMinMillis, maxDurationMillis: tc.processingDurationMaxMillis, l: l} + processor2 := &Processor{minDurationMillis: tc.processingDurationMinMillis, maxDurationMillis: tc.processingDurationMaxMillis, l: l} + breakProcessingCondition := func() bool { + return len(processor1.ProcessedMessages()) > 3 && len(processor2.ProcessedMessages()) > 3 + } + ctx1, cancel1 := context.WithCancel(ctx) + work1 := wf.Create(cTopicCfg1, processor1, zkafka.Speedup(5), + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + if breakProcessingCondition() { + cancel1() + } + }})) + + cTopicCfg2 := cTopicCfg1 + cTopicCfg2.ClientID += "-2" + ctx2, cancel2 := context.WithCancel(ctx) + work2 := wf.Create(cTopicCfg2, processor2, zkafka.Speedup(5), + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + if breakProcessingCondition() { + cancel2() + } + }})) + + t.Log("starting work1") + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + defer wg.Done() + work1.Run(ctx1, nil) + }() + + // wait until processor1 has begun to process messages + for { + if len(processor1.ProcessedMessages()) > 5 { + break + } + time.Sleep(time.Millisecond) + } + + t.Log("starting work2") + wg.Add(1) + go func() { + defer wg.Done() + work2.Run(ctx2, nil) + }() + wg.Wait() + // keep track of how many messages + messageProcessCounter := make(map[partition]int) + updateProcessCounter := func(msgs []*zkafka.Message) { + for _, m := range msgs { + key := partition{ + partition: m.Partition, + offset: m.Offset, + topic: m.Topic, + } + if _, ok := messageProcessCounter[key]; !ok { + messageProcessCounter[key] = 0 + } + messageProcessCounter[key] += 1 + } + } + updateProcessCounter(processor1.ProcessedMessages()) + updateProcessCounter(processor2.ProcessedMessages()) + + for key, val := range messageProcessCounter { + if val > 1 { + t.Errorf("Message Processed More than Once: partition %d, offset %d, topic %s", key.partition, key.offset, key.topic) + } + } + }) + } +} + +// Test_AssignmentsReflectsConsumerAssignments given a single consumer in a group all the partitions (2) are assigned to them +// when another consumer joins the group +// then a rebalance occurs and the partitions are split between the two and this is reflected in the Assignments call +// +// The rebalances are handled during the Poll call under the hood (which is only called while a KReader is in the attempt of Reading. +// So as we simulate two members of a group we'll need to keep calling from both consumers so the rebalance eventually occurs +func Test_AssignmentsReflectsConsumerAssignments(t *testing.T) { + ctx := context.Background() + + groupID := uuid.NewString() + bootstrapServer := getBootstrap() + + // create a randomly named topic so we don't interfere with other tests + topic := "topic-" + uuid.NewString() + partitionCount := 2 + createTopic(t, bootstrapServer, topic, partitionCount) + + l := zkafka.NoopLogger{} + wclient := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}) + defer func() { require.NoError(t, wclient.Close()) }() + + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, zkafka.LoggerOption(l)) + defer func() { require.NoError(t, client.Close()) }() + + writer, err := wclient.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + }) + + require.NoError(t, err) + + msg := Msg{ + Val: "sdfds", + } + + t.Log("Begin writing messages") + // write N messages to topic + msgCount := 40 + for i := 0; i < msgCount; i++ { + _, err = writer.WriteKey(ctx, uuid.NewString(), msg) + require.NoError(t, err) + } + + t.Log("Completed writing messages") + + // create consumer 1 which has its own processor + cTopicCfg1 := zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("reader-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + GroupID: groupID, + // use increase readtimeout so less likely for reader1 to finish processing before r2 joins. + ReadTimeoutMillis: ptr(5000), + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + } + + reader1, err := client.Reader(ctx, cTopicCfg1) + require.NoError(t, err) + r1, ok := reader1.(*zkafka.KReader) + require.True(t, ok, "expected reader to be KReader") + + // create consumer 2 which has its own processor + cTopicCfg2 := cTopicCfg1 + cTopicCfg2.ClientID += "-2" + reader2, err := client.Reader(ctx, cTopicCfg2) + require.NoError(t, err) + r2, ok := reader2.(*zkafka.KReader) + require.True(t, ok, "expected reader to be KReader") + + // a helper method for reading a message and committing if its available (similar to what the work loop would do) + readAndCommit := func(t *testing.T, r *zkafka.KReader) *zkafka.Message { + t.Helper() + msg, err := r.Read(context.Background()) + require.NoError(t, err) + if msg != nil { + msg.Done() + } + return msg + } + + // wait until a message is consumed (meaning the consumer has joined the group + for { + msg1 := readAndCommit(t, r1) + if msg1 != nil { + break + } + } + + assignments, err := r1.Assignments(ctx) + require.NoError(t, err) + require.Equal(t, partitionCount, len(assignments), "expected all partitions to be assigned to consumer 1") + + t.Log("Begin attempted consumption by consumer 2. Should result in rebalance") + // now start trying to consume from both consumer 1 and 2. Eventually both will join group (Break when second consumer has joined group, signaled by processing a message) + for { + msg2 := readAndCommit(t, r2) + if msg2 != nil { + _ = readAndCommit(t, r1) + break + } + msg3 := readAndCommit(t, r1) + require.NotNil(t, msg3, "test can deadlock if reader1 finishes processing all messages before reader2 joins group. We'll fail if reader1 exhausts all its messages") + } + + assignments1, err := r1.Assignments(ctx) + require.NoError(t, err) + + assignments2, err := r2.Assignments(ctx) + require.NoError(t, err) + + // expect assignments to be split amongst the two consumers + require.Len(t, assignments1, 1, "unexpected number of assignments") + require.Len(t, assignments2, 1, "unexpected number of assignments") +} + +// Test_UnfinishableWorkDoesntBlockWorkIndefinitely given two consumers 1 with unfinished work +// consuming from a topic that has multiple partitions (so both can simultaneously consume) +// when the second consumer joins and causes a rebalance +// then the first isn't infinitely blocked in its rebalance +func Test_UnfinishableWorkDoesntBlockWorkIndefinitely(t *testing.T) { + ctx := context.Background() + + groupID := uuid.NewString() + bootstrapServer := getBootstrap() + + // create a randomly named topic so we don't interfere with other tests + topic := "topic-" + uuid.NewString() + partitionCount := 4 + createTopic(t, bootstrapServer, topic, partitionCount) + + l := stdLogger{} + wclient := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}) + defer func() { require.NoError(t, wclient.Close()) }() + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, zkafka.LoggerOption(l)) + defer func() { require.NoError(t, client.Close()) }() + + writer, err := wclient.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + RequestRequiredAcks: ptr("0"), + EnableIdempotence: ptr(false), + }) + require.NoError(t, err) + + msg := Msg{ + Val: "sdfds", + } + + // write N messages to topic + msgCount := 40 + for i := 0; i < msgCount; i++ { + _, err = writer.WriteKey(ctx, uuid.NewString(), msg) + require.NoError(t, err) + } + + // create consumer 1 which has its own processor + cTopicCfg1 := zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("reader-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + GroupID: groupID, + ReadTimeoutMillis: ptr(5000), + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + } + + reader1, err := client.Reader(ctx, cTopicCfg1) + require.NoError(t, err) + + r1, ok := reader1.(*zkafka.KReader) + require.True(t, ok, "expected reader to be KReader") + + // create consumer 2 which has its own processor + cTopicCfg2 := cTopicCfg1 + cTopicCfg2.ClientID += "-2" + reader2, err := client.Reader(ctx, cTopicCfg2) + require.NoError(t, err) + + r2, ok := reader2.(*zkafka.KReader) + require.True(t, ok, "expected reader to be KReader") + defer client.Close() + + // wait until a message is consumed (meaning the consumer has joined the group) but dont commit it + // this will be marked as unfinished work. This will cause issues on rebalance but shouldn't block it forever + for { + msg1, err := r1.Read(ctx) + require.NoError(t, err) + if msg1 != nil { + break + } + } + t.Log("Successfully read 1 message with reader 1") + + // now start trying to consume from both consumer 1 and 2. + // Eventually both will join group. If rebalance isn't properly implemented with timeout, reader1 could block forever on account of + // unfinished work + for { + msg2, err := r2.Read(ctx) + require.NoError(t, err) + if msg2 != nil { + break + } + t.Log("Haven't read message with reader 2 yet") + _, err = r1.Read(ctx) + require.NoError(t, err) + } +} + +// TestKafkaClientsCanWriteToTheirDeadLetterTopic given a message in a source topic +// when processing that message errors and a deadletter is configured +// then the errored message will be written to the dlt +func Test_KafkaClientsCanWriteToTheirDeadLetterTopic(t *testing.T) { + bootstrapServer := getBootstrap() + topic := "topic1" + uuid.NewString() + dlt := "deadlettertopic1" + uuid.NewString() + + createTopic(t, bootstrapServer, topic, 2) + createTopic(t, bootstrapServer, dlt, 2) + + groupID := uuid.NewString() + + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, zkafka.LoggerOption(stdLogger{})) + defer func() { require.NoError(t, client.Close()) }() + + ctx := context.Background() + + writer, err := client.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + }) + + consumerTopicConfig := zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("worker-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + GroupID: groupID, + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + DeadLetterTopicConfig: &zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("dltWriter-%s-%s", t.Name(), uuid.NewString()), + Topic: dlt, + Formatter: zfmt.JSONFmt, + }, + } + + msg := Msg{ + Val: "sdfds", + } + + key := "original-key" + _, err = writer.WriteKey(ctx, key, msg) + require.NoError(t, err) + + wf := zkafka.NewWorkFactory(client) + + processor := &Processor{ + reterr: errors.New("processing error"), + } + work := wf.Create(consumerTopicConfig, processor) + + ctx2, cancel := context.WithCancel(ctx) + defer cancel() + go func() { + work.Run(ctx2, nil) + }() + for { + if len(processor.ProcessedMessages()) == 1 { + cancel() + break + } + time.Sleep(time.Millisecond) + } + + dltReader, err := client.Reader(ctx, zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("reader-for-test-%s-%s", t.Name(), uuid.NewString()), + GroupID: uuid.NewString(), + Topic: dlt, + Formatter: zfmt.JSONFmt, + ReadTimeoutMillis: ptr(15000), + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + }) + + require.NoError(t, err) + + t.Log("reading from dlt") + // read message from dlt to confirm it was forwarded + dltMsgWrapper, err := dltReader.Read(ctx) + require.NoError(t, err) + require.NotNil(t, dltMsgWrapper) + + got := Msg{} + err = dltMsgWrapper.Decode(&got) + require.NoError(t, err) + + t.Log("assert message from dlt") + assertEqual(t, got, msg) + assertEqual(t, dltMsgWrapper.Key, key) +} + +func Test_WorkDelay_GuaranteesProcessingDelayedAtLeastSpecifiedDelayDurationFromWhenMessageWritten(t *testing.T) { + ctx := context.Background() + + groupID := uuid.NewString() + bootstrapServer := getBootstrap() + + // create a randomly named topic so we don't interfere with other tests + topic := "topic-" + uuid.NewString() + createTopic(t, bootstrapServer, topic, 2) + + l := stdLogger{} + wclient := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}) + defer func() { require.NoError(t, wclient.Close()) }() + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, zkafka.LoggerOption(l)) + defer func() { require.NoError(t, client.Close()) }() + + processDelayMillis := 2000 + // create work which has its own processor + cTopicCfg1 := zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("reader-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + GroupID: groupID, + ProcessDelayMillis: &processDelayMillis, + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + } + wf := zkafka.NewWorkFactory(client, zkafka.WithLogger(l)) + + ctx1, cancel1 := context.WithCancel(ctx) + defer cancel1() + + type result struct { + message *zkafka.Message + processingInstant time.Time + } + var results []result + processor1 := &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + results = append(results, result{ + message: message, + processingInstant: time.Now(), + }) + return nil + }, + } + + work := wf.Create(cTopicCfg1, processor1) + + t.Log("starting work") + grp := errgroup.Group{} + grp.Go(func() error { + return work.Run(context.Background(), ctx1.Done()) + }) + + writer, err := wclient.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + }) + require.NoError(t, err) + + msg := Msg{ + Val: "sdfds", + } + + t.Log("Started writing messages") + // write N messages to topic + msgCount := 1000 + for i := 0; i < msgCount; i++ { + _, err = writer.Write(ctx, msg) + require.NoError(t, err) + } + t.Log("Completed writing messages") + + t.Log("starting exit polling") + pollWait(func() bool { + return len(results) >= msgCount + }, + pollOpts{ + exit: cancel1, + timeoutExit: func() { + require.Failf(t, "Polling condition not met prior to test timeout", "Processing count %s", len(results)) + }, + }, + ) + err = grp.Wait() + require.NoError(t, err) + + t.Log("work complete") + for _, r := range results { + require.NotEmpty(t, r.processingInstant) + require.NotEmpty(t, r.message.TimeStamp) + require.GreaterOrEqual(t, r.processingInstant, r.message.TimeStamp.Add(time.Duration(processDelayMillis)*time.Millisecond), "Expect processing time to be equal to the write timestamp plus the work delay duration") + } +} + +// Test_WorkDelay_DoesntHaveDurationStackEffect confirms that a work doesn't unnecessarily add delay (only pausing a partition when a message was written within the last `duration` moments). +// A bad implementation of work delay would call that pause for each message, and in a built up topic, this would lead to increased perceived latency. +// +// This test creates N messages on the topic and then starts processing +// 1. It asserts that the time since the message was written is at least that of the delay. +// This is a weak assertion sicne the messages are written before the work consumer group is started. Other tests do a better job confirming this behavior +// 2. It also asserts that the time between the first and last message is very short. +// This is expected in a backlog situation, since the worker will delay once, and with monotonically increasing timestamps won't have to dely again +func Test_WorkDelay_DoesntHaveDurationStackEffect(t *testing.T) { + ctx := context.Background() + + groupID := uuid.NewString() + bootstrapServer := getBootstrap() + + // create a randomly named topic so we don't interfere with other tests + topic := "topic-" + uuid.NewString() + createTopic(t, bootstrapServer, topic, 2) + + l := stdLogger{} + wclient := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}) + defer func() { require.NoError(t, wclient.Close()) }() + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, zkafka.LoggerOption(l)) + defer func() { require.NoError(t, client.Close()) }() + + writer, err := wclient.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: fmt.Sprintf("writer-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + }) + require.NoError(t, err) + + msg := Msg{ + Val: "sdfds", + } + + t.Log("Started writing messages") + // write N messages to topic + msgCount := 500 + for i := 0; i < msgCount; i++ { + _, err = writer.Write(ctx, msg) + require.NoError(t, err) + } + t.Log("Completed writing messages") + + processDelayMillis := 2000 + // create work which has its own processor + cTopicCfg1 := zkafka.ConsumerTopicConfig{ + ClientID: fmt.Sprintf("reader-%s-%s", t.Name(), uuid.NewString()), + Topic: topic, + Formatter: zfmt.JSONFmt, + GroupID: groupID, + ProcessDelayMillis: &processDelayMillis, + AdditionalProps: map[string]any{ + "auto.offset.reset": "earliest", + }, + } + wf := zkafka.NewWorkFactory(client, zkafka.WithLogger(l)) + + ctx1, cancel1 := context.WithCancel(ctx) + defer cancel1() + + type result struct { + message *zkafka.Message + processingInstant time.Time + } + var results []result + processor1 := &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + results = append(results, result{ + message: message, + processingInstant: time.Now(), + }) + return nil + }, + } + + work := wf.Create(cTopicCfg1, processor1) + + t.Log("starting work") + grp := errgroup.Group{} + grp.Go(func() error { + return work.Run(context.Background(), ctx1.Done()) + }) + + t.Log("starting exit polling") + pollWait(func() bool { + return len(results) >= msgCount + }, + pollOpts{ + exit: cancel1, + timeoutExit: func() { + require.Failf(t, "Polling condition not met prior to test timeout", "Processing count %s", len(results)) + }, + }, + ) + err = grp.Wait() + require.NoError(t, err) + + t.Log("work complete") + for _, r := range results { + require.NotEmpty(t, r.processingInstant) + require.NotEmpty(t, r.message.TimeStamp) + require.GreaterOrEqual(t, r.processingInstant, r.message.TimeStamp.Add(time.Duration(processDelayMillis)*time.Millisecond), "Expect processing time to be equal to the write timestamp plus the work delay duration") + } + first := results[0] + last := results[len(results)-1] + require.WithinDuration(t, last.processingInstant, first.processingInstant, time.Duration(processDelayMillis/2)*time.Millisecond, "Time since first and last processed message should be very short, since processing just updates an in memory slice. This should take on the order of microseconds, but to account for scheduling drift the assertion is half the delay") +} + +func createTopic(t *testing.T, bootstrapServer, topic string, partitions int) { + t.Helper() + aclient, err := kafka.NewAdminClient(&kafka.ConfigMap{"bootstrap.servers": bootstrapServer}) + require.NoError(t, err) + _, err = aclient.CreateTopics(context.Background(), []kafka.TopicSpecification{ + { + Topic: topic, + NumPartitions: partitions, + ReplicationFactor: 1, + }, + }) + require.NoError(t, err) +} + +// getBootstrap returns the kafka broker to be used for integration tests. It allows the overwrite of default via +// envvar +func getBootstrap() string { + bootstrapServer, ok := os.LookupEnv("KAFKA_BOOTSTRAP_SERVER") + if !ok { + bootstrapServer = "localhost:9093" // local development + } + return bootstrapServer +} + +type Msg struct { + Val string +} + +type Processor struct { + m sync.Mutex + processedMessages []*zkafka.Message + reterr error + minDurationMillis int + maxDurationMillis int + l zkafka.Logger +} + +func (p *Processor) Process(ctx context.Context, msg *zkafka.Message) error { + p.m.Lock() + p.processedMessages = append(p.processedMessages, msg) + p.m.Unlock() + if p.l != nil { + p.l.Infow(ctx, "Process", "partition", msg.Partition, "offset", msg.Offset) + } + durationRange := p.maxDurationMillis - p.minDurationMillis + delayMillis := p.minDurationMillis + if durationRange != 0 { + delayMillis += rand.Intn(durationRange) + } + delay := time.Duration(delayMillis) * time.Millisecond + <-time.After(delay) + return p.reterr +} + +func (p *Processor) ProcessedMessages() []*zkafka.Message { + p.m.Lock() + defer p.m.Unlock() + + var msgs []*zkafka.Message + for _, m := range p.processedMessages { + msgs = append(msgs, m) + } + return msgs +} + +func assertEqual(t *testing.T, got, want any, opts ...cmp.Option) { + t.Helper() + if diff := cmp.Diff(got, want, opts...); diff != "" { + diff = fmt.Sprintf("\ngot: -\nwant: +\n%s", diff) + t.Fatal(diff) + } +} + +type partition struct { + partition int32 + offset int64 + topic string +} diff --git a/test/work_test.go b/test/work_test.go new file mode 100644 index 0000000..da3f54e --- /dev/null +++ b/test/work_test.go @@ -0,0 +1,1989 @@ +package test + +import ( + "context" + "errors" + "fmt" + "log" + "math/rand" + "runtime/debug" + "strconv" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/google/uuid" + "github.com/stretchr/testify/require" + "github.com/zillow/zkafka" + zkafka_mocks "github.com/zillow/zkafka/mocks" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" + + "github.com/golang/mock/gomock" +) + +var ( + topicName = "orange" + NoopOnDone = func() {} +) + +func TestWork_Run_FailsWithLogsWhenFailedToGetReader(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka_mocks.NewMockLogger(ctrl) + l.EXPECT().Warnw(gomock.Any(), "Kafka worker read message failed", "error", gomock.Any(), "topics", gomock.Any()).MinTimes(1) + l.EXPECT().Warnw(gomock.Any(), "Kafka topic processing circuit open", "topics", gomock.Any()).AnyTimes() + + cp := zkafka_mocks.NewMockClientProvider(ctrl) + cp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(nil, errors.New("no kafka client reader created")).MinTimes(1) + + kwf := zkafka.NewWorkFactory(cp, zkafka.WithLogger(l)) + fanoutCount := atomic.Int64{} + w := kwf.Create(zkafka.ConsumerTopicConfig{Topic: topicName}, + &fakeProcessor{}, + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + fanoutCount.Add(1) + }})) + + ctx, cancel := context.WithCancel(ctx) + defer cancel() + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return fanoutCount.Load() >= 1 + }, pollOpts{ + exit: cancel, + timeoutExit: func() { + require.Fail(t, "Polling condition not met prior to test timeout") + }, + pollPause: time.Millisecond, + maxWait: 10 * time.Second, + }) +} + +func TestWork_Run_FailsWithLogsWhenGotNilReader(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka_mocks.NewMockLogger(ctrl) + l.EXPECT().Warnw(gomock.Any(), "Kafka worker read message failed", "error", gomock.Any(), "topics", gomock.Any()).Times(1) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(nil, nil) + + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + w := kwf.Create(zkafka.ConsumerTopicConfig{Topic: topicName}, &fakeProcessor{}, + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + cancel() + }})) + + w.Run(ctx, nil) +} + +func TestWork_Run_FailsWithLogsForReadError(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka_mocks.NewMockLogger(ctrl) + + l.EXPECT().Warnw(gomock.Any(), "Kafka worker read message failed", "error", gomock.Any(), "topics", gomock.Any()).MinTimes(1) + + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).Times(1).Return(nil, errors.New("error occurred during read")) + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(r, nil) + + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + w := kwf.Create(zkafka.ConsumerTopicConfig{Topic: topicName}, &fakeProcessor{}, + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + cancel() + }})) + + w.Run(ctx, nil) +} + +func TestWork_Run_CircuitBreakerOpensOnReadError(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).AnyTimes().Return(nil, errors.New("error occurred during read")) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(r, nil) + + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + + cnt := atomic.Int64{} + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &fakeProcessor{}, + zkafka.CircuitBreakAfter(1), // Circuit breaks after 1 error. + zkafka.CircuitBreakFor(50*time.Millisecond), + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + l.Warnw(ctx, "Fanout callback called") + cnt.Add(1) + }})) + + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + start := time.Now() + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return cnt.Load() >= 10 + }, pollOpts{ + exit: cancel, + timeoutExit: func() { + require.Failf(t, "Polling condition not met prior to test timeout", "Processing count %s", 10) + }, + }) + require.GreaterOrEqual(t, time.Since(start), 200*time.Millisecond, "Every circuit breaker stoppage is 50ms, and we expect it to be in open state (stoppage) for half the messages (and half open for the other half, 1 message through).") +} + +func TestWork_Run_CircuitBreaksOnProcessError(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + msg := zkafka.GetFakeMessage("1", nil, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).AnyTimes().Return(msg, nil) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).AnyTimes().Return(r, nil) + + kproc := &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return errors.New("KafkaError.Process error") + }, + } + + cnt := atomic.Int64{} + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + kproc, + zkafka.CircuitBreakAfter(1), // Circuit breaks after 1 error. + zkafka.CircuitBreakFor(50*time.Millisecond), + zkafka.WithOnDone(func(ctx context.Context, message *zkafka.Message, err error) { + cnt.Add(1) + }), + ) + + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + start := time.Now() + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return cnt.Load() >= 10 + }, pollOpts{ + exit: cancel, + timeoutExit: func() { + require.Failf(t, "Polling condition not met prior to test timeout", "Processing count %s", 10) + }, + }) + + require.GreaterOrEqual(t, time.Since(start), 400*time.Millisecond, "Every circuit breaker stoppage is 50ms, and we expect it to be executed for each of the n -2 failed messages (first one results in error and trips the circuit breaker. Second message read prior to trip") +} + +func TestWork_Run_DoNotSkipCircuitBreak(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + failureMessage := zkafka.GetFakeMessage("1", nil, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + + r.EXPECT().Read(gomock.Any()).Return(failureMessage, nil).AnyTimes() + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).AnyTimes().Return(r, nil) + + kproc := &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return zkafka.ProcessError{ + Err: errors.New("kafka.ProcessError"), + DisableCircuitBreak: false, + } + }, + } + + cnt := atomic.Int64{} + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + kproc, + zkafka.CircuitBreakAfter(1), // Circuit breaks after 1 error. + zkafka.CircuitBreakFor(50*time.Millisecond), + zkafka.WithOnDone(func(ctx context.Context, _ *zkafka.Message, _ error) { + cnt.Add(1) + }), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + start := time.Now() + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return cnt.Load() > 10 + }, pollOpts{ + exit: cancel, + pollPause: time.Microsecond * 100, + timeoutExit: func() { + require.Failf(t, "Polling condition not met prior to test timeout", "Processing count %s", 10) + }, + }) + require.GreaterOrEqual(t, time.Since(start), 450*time.Millisecond, "Every circuit breaker stoppage is 50ms, and we expect it to be executed for each of the n -1 failed messages (first one results in error and trips the circuit breaker") +} + +func TestWork_Run_DoSkipCircuitBreak(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + failureMessage := zkafka.GetFakeMessage("1", nil, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + + r.EXPECT().Read(gomock.Any()).Return(failureMessage, nil).AnyTimes() + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).AnyTimes().Return(r, nil) + + kproc := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return zkafka.ProcessError{ + Err: errors.New("kafka.ProcessError"), + DisableCircuitBreak: true, + } + }, + } + + cnt := atomic.Int64{} + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &kproc, + zkafka.CircuitBreakAfter(1), // Circuit breaks after 1 error. + zkafka.CircuitBreakFor(50*time.Millisecond), + zkafka.WithOnDone(func(ctx context.Context, _ *zkafka.Message, _ error) { + cnt.Add(1) + }), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + start := time.Now() + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return cnt.Load() >= 10 + }, pollOpts{ + exit: cancel, + timeoutExit: func() { + require.Failf(t, "Polling condition not met prior to test timeout", "Processing count %s", 10) + }, + }) + + require.LessOrEqual(t, time.Since(start), 50*time.Millisecond, "Every circuit breaker stoppage is 50ms, and we expect it to be skipped for each of the 10 failed messages. The expected time to process 10 messages is on the order of micro/nanoseconds, but we'll conservatievely be happy with being less than a single circuit break cycle") +} + +func TestWork_Run_CircuitBreaksOnProcessPanicInsideProcessorGoRoutine(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + msg := zkafka.GetFakeMessage("1", nil, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).AnyTimes().Return(msg, nil) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).AnyTimes().Return(r, nil) + + kproc := &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + panic("fake a panic occurring on process") + }, + } + + cnt := atomic.Int64{} + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + kproc, + zkafka.CircuitBreakAfter(1), // Circuit breaks after 1 error. + zkafka.CircuitBreakFor(50*time.Millisecond), + zkafka.WithOnDone(func(ctx context.Context, _ *zkafka.Message, _ error) { + cnt.Add(1) + }), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + start := time.Now() + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + ok := cnt.Load() >= 10 + if ok { + cancel() + } + return ok + }, pollOpts{ + timeoutExit: func() { + require.Failf(t, "Polling condition not met prior to test timeout", "Processing count %s", 10) + }, + }) + + require.GreaterOrEqual(t, time.Since(start), 400*time.Millisecond, "Every circuit breaker stoppage is 50ms, and we expect it to be executed for each of the n failed messages with the exception of the first and second message (first trips, and second is read before the trip)") +} + +func TestWork_Run_DisabledCircuitBreakerContinueReadError(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka_mocks.NewMockLogger(ctrl) + + processingCount := 4 + l.EXPECT().Errorw(gomock.Any(), "Kafka topic single message processing failed", "error", gomock.Any(), "kmsg", gomock.Any()).AnyTimes() + l.EXPECT().Warnw(gomock.Any(), "Kafka worker read message failed", "error", gomock.Any(), "topics", gomock.Any()).MinTimes(processingCount) + l.EXPECT().Warnw(gomock.Any(), "Outside context canceled", "error", gomock.Any(), "kmsg", gomock.Any()).AnyTimes() + l.EXPECT().Warnw(gomock.Any(), "Kafka topic processing circuit open", "topics", gomock.Any()).Times(0) + l.EXPECT().Debugw(gomock.Any(), "Kafka topic message received", "offset", gomock.Any(), "partition", gomock.Any(), "topic", gomock.Any(), "groupID", gomock.Any()).AnyTimes() + + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).MinTimes(4).Return(nil, errors.New("error occurred on read")) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(r, nil) + + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + + cnt := atomic.Int64{} + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &fakeProcessor{}, + zkafka.DisableCircuitBreaker(), + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + cnt.Add(1) + }}), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + ok := cnt.Load() >= int64(processingCount) + if ok { + cancel() + } + return ok + }, pollOpts{ + timeoutExit: func() { + require.Failf(t, "Polling condition not met prior to test timeout", "Processing count %s", processingCount) + }, + }) +} + +func TestWork_Run_SpedUpIsFaster(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockReader := zkafka_mocks.NewMockReader(ctrl) + + mockReader.EXPECT().Read(gomock.Any()).DoAndReturn(func(ctx context.Context) (*zkafka.Message, error) { + return zkafka.GetFakeMessage(uuid.NewString(), nil, &zfmt.JSONFormatter{}, NoopOnDone), nil + }).AnyTimes() + mockReader.EXPECT().Close().Return(nil).AnyTimes() + + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(2).Return(mockReader, nil) + + kwf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(zkafka.NoopLogger{})) + slow := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + time.Sleep(time.Millisecond * 10) + return nil + }, + } + fast := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + time.Sleep(time.Millisecond * 10) + return nil + }, + } + + func() { + workerSlow := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &slow, + ) + + ctx, cancel := context.WithTimeout(ctx, 50*time.Millisecond) + defer cancel() + + workerSlow.Run(ctx, nil) + }() + + // use te speedup option so more go routines process the read messages. + // We'll let it process over the same amount of time (defined by timeout in context) + func() { + workerSpedUp := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &fast, + zkafka.Speedup(10), + ) + + ctx, cancel := context.WithTimeout(ctx, 50*time.Millisecond) + defer cancel() + + // wait for the cancel to occur via timeout + workerSpedUp.Run(ctx, nil) + }() + + // by putting a delay in the work.do method we minimize the comparative overhead in creating additional goroutines + // and our speedup should begin to approach the KafkaSpeedup option of 10. + // Because of hardware variance and context variance we'll only softly assert this speed up factor by asserting a range + lowRangeSpeedup := 3 + highRangeSpeedup := 15 + slowCount := len(slow.ProcessedMessages()) + fastCount := len(fast.ProcessedMessages()) + lowerRange := slowCount * lowRangeSpeedup + higherRange := slowCount * highRangeSpeedup + + if fastCount < lowerRange { + t.Errorf("fast count should be at least %d times faster. fast count %d, slow count %d", lowRangeSpeedup, fastCount, slowCount) + } + if fastCount > higherRange { + t.Errorf("fast count should have an upper limit on how much faster it is (no more than approximately %d faster). fast count %d, slow count %d", highRangeSpeedup, fastCount, slowCount) + } +} + +func TestKafkaWork_ProcessorReturnsErrorIsLoggedAsWarning(t *testing.T) { + defer recoverThenFail(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + // + l := zkafka_mocks.NewMockLogger(ctrl) + l.EXPECT().Warnw(gomock.Any(), "Kafka topic processing circuit open", "topics", gomock.Any()).AnyTimes() + l.EXPECT().Warnw(gomock.Any(), "Kafka topic single message processing failed", "error", gomock.Any(), "kmsg", gomock.Any()).MinTimes(1) + l.EXPECT().Warnw(gomock.Any(), "Outside context canceled", "kmsg", gomock.Any(), "error", gomock.Any()).AnyTimes() + l.EXPECT().Debugw(gomock.Any(), "Kafka topic message received", "offset", gomock.Any(), "partition", gomock.Any(), "topic", gomock.Any(), "groupID", gomock.Any()).AnyTimes() + + msg := zkafka.GetFakeMessage("key", "val", &zfmt.JSONFormatter{}, NoopOnDone) + mockReader := zkafka_mocks.NewMockReader(ctrl) + mockReader.EXPECT().Read(gomock.Any()).AnyTimes().Return(msg, nil) + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(mockReader, nil) + + processor := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return errors.New("error for testcase TestKafkaWork_ProcessorReturnsErrorIsLoggedAsWarning") + }, + } + wf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(l)) + count := atomic.Int64{} + work := wf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &processor, + zkafka.WithOnDone(func(ctx context.Context, message *zkafka.Message, err error) { + count.Add(1) + })) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + work.Run(ctx, nil) + }() + for { + if count.Load() >= 1 { + cancel() + break + } + time.Sleep(time.Microsecond * 100) + } +} + +// TestKafkaWork_ProcessorTimeoutCausesContextCancellation demonstrates that ProcessTimeoutMillis will +// cancel the context passed to the processor callback. +// The processor callback blocks until this context is cancelled, and then returns the error. +func TestKafkaWork_ProcessorTimeoutCausesContextCancellation(t *testing.T) { + defer recoverThenFail(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + msg := zkafka.GetFakeMessage("key", "val", &zfmt.JSONFormatter{}, NoopOnDone) + mockReader := zkafka_mocks.NewMockReader(ctrl) + mockReader.EXPECT().Read(gomock.Any()).AnyTimes().Return(msg, nil) + + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(mockReader, nil) + + wf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(l)) + + processor := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + <-ctx.Done() + return ctx.Err() + }, + } + count := atomic.Int64{} + work := wf.Create( + zkafka.ConsumerTopicConfig{ + Topic: topicName, + ProcessTimeoutMillis: ptr(1)}, + &processor, + zkafka.WithOnDone(func(ctx context.Context, message *zkafka.Message, err error) { + count.Add(1) + }), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + work.Run(ctx, nil) + }() + for { + if count.Load() >= 1 { + cancel() + break + } + time.Sleep(time.Microsecond * 100) + } +} + +func TestWork_WithDeadLetterTopic_NoMessagesWrittenToDLTSinceNoErrorsOccurred(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + mockReader := zkafka_mocks.NewMockReader(ctrl) + gomock.InOrder( + mockReader.EXPECT().Read(gomock.Any()).Return(getRandomMessage(), nil), + mockReader.EXPECT().Read(gomock.Any()).Return(getRandomMessage(), nil), + mockReader.EXPECT().Read(gomock.Any()).Return(nil, nil).AnyTimes(), + ) + mockReader.EXPECT().Close().Return(nil).AnyTimes() + + mockWriter := zkafka_mocks.NewMockWriter(ctrl) + // no messages written into dlt because there weren't errors + mockWriter.EXPECT().Write(gomock.Any(), gomock.Any()).Times(0) + mockWriter.EXPECT().Close().AnyTimes() + + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(mockReader, nil) + mockClientProvider.EXPECT().Writer(gomock.Any(), gomock.Any()).Times(2).Return(mockWriter, nil) + + kwf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(l)) + + processor := fakeProcessor{} + + var cnt atomic.Int64 + w1 := kwf.Create( + zkafka.ConsumerTopicConfig{ + Topic: topicName, + DeadLetterTopicConfig: &zkafka.ProducerTopicConfig{ + ClientID: uuid.NewString(), + Topic: "topic2", + }, + }, + &processor, + zkafka.WithOnDone(func(ctx context.Context, message *zkafka.Message, err error) { + cnt.Add(1) + }), + ) + + workCompleted := atomic.Bool{} + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + w1.Run(ctx, nil) + workCompleted.Store(true) + }() + + pollWait(func() bool { + stop := cnt.Load() == 2 + if stop { + cancel() + } + return stop + }, pollOpts{ + timeoutExit: func() { + require.Fail(t, "Timed out during poll") + }, + maxWait: 10 * time.Second, + }) + + pollWait(func() bool { + return workCompleted.Load() + }, pollOpts{ + timeoutExit: func() { + require.Fail(t, "Timed out during poll waiting for work exit") + }, + maxWait: 10 * time.Second, + }) +} + +func TestWork_WithDeadLetterTopic_MessagesWrittenToDLTSinceErrorOccurred(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + mockReader := zkafka_mocks.NewMockReader(ctrl) + msg1 := getRandomMessage() + msg2 := getRandomMessage() + gomock.InOrder( + mockReader.EXPECT().Read(gomock.Any()).Return(msg1, nil), + mockReader.EXPECT().Read(gomock.Any()).Return(msg2, nil), + mockReader.EXPECT().Read(gomock.Any()).Return(nil, nil).AnyTimes(), + ) + mockReader.EXPECT().Close().Return(nil).AnyTimes() + + mockWriter := zkafka_mocks.NewMockWriter(ctrl) + // each errored message gets forwarded + mockWriter.EXPECT().WriteRaw(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Times(2) + mockWriter.EXPECT().Close().AnyTimes() + + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(mockReader, nil) + mockClientProvider.EXPECT().Writer(gomock.Any(), gomock.Any()).Times(2).Return(mockWriter, nil) + + kwf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(l)) + + processor := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return errors.New("processor error") + }, + } + + w1 := kwf.Create( + zkafka.ConsumerTopicConfig{ + Topic: topicName, + DeadLetterTopicConfig: &zkafka.ProducerTopicConfig{ + ClientID: uuid.NewString(), + Topic: "topic2", + }, + }, + &processor, + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + w1.Run(ctx, nil) + }() + + pollWait(func() bool { + return len(processor.ProcessedMessages()) == 2 + }, pollOpts{ + timeoutExit: func() { + require.Fail(t, "Timed out during poll") + }, + pollPause: time.Millisecond, + maxWait: 10 * time.Second, + }) +} + +// TestWork_WithDeadLetterTopic_FailedToGetWriterDoesntPauseProcessing even if get topic writer (for DLT) returns error processing still continues. +// This test configures a single virtual partition to process the reader. If processing halted on account of DLT write error, +// the test wouldn't get through all 10 messages +func TestWork_WithDeadLetterTopic_FailedToGetWriterDoesntPauseProcessing(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + mockReader := zkafka_mocks.NewMockReader(ctrl) + msg1 := getRandomMessage() + mockReader.EXPECT().Read(gomock.Any()).Times(10).Return(msg1, nil) + mockReader.EXPECT().Read(gomock.Any()).Return(nil, nil).AnyTimes() + mockReader.EXPECT().Close().Return(nil).AnyTimes() + + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(mockReader, nil) + mockClientProvider.EXPECT().Writer(gomock.Any(), gomock.Any()).AnyTimes().Return(nil, errors.New("failed to get dlt writer")) + + kwf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(l)) + + processor := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return errors.New("processor error") + }, + } + + dltTopic1 := "dlt-topic2" + w1 := kwf.Create( + zkafka.ConsumerTopicConfig{ + Topic: topicName, + ClientID: uuid.NewString(), + GroupID: uuid.NewString(), + DeadLetterTopicConfig: &zkafka.ProducerTopicConfig{ + ClientID: uuid.NewString(), + Topic: dltTopic1, + }, + }, + &processor, + zkafka.DisableCircuitBreaker(), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + w1.Run(ctx, nil) + }() + + // the previous poll doesn't fully guarantee that the piece of code that + pollWait(func() bool { + return len(processor.ProcessedMessages()) == 10 + }, pollOpts{ + timeoutExit: func() { + require.Failf(t, "Timed out during poll", "Processed Messages %d", len(processor.ProcessedMessages())) + }, + pollPause: time.Millisecond, + maxWait: 10 * time.Second, + }) +} + +// TestWork_WithDeadLetterTopic_FailedToWriteToDLTDoesntPauseProcessing even if callback can't write to DLT, processing still continues. +// This test configures a single virtual partition to process the reader. If processing halted on account of DLT write error, +// the test wouldn't get through all 10 messages +func TestWork_WithDeadLetterTopic_FailedToWriteToDLTDoesntPauseProcessing(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + mockReader := zkafka_mocks.NewMockReader(ctrl) + msg1 := getRandomMessage() + mockReader.EXPECT().Read(gomock.Any()).Times(10).Return(msg1, nil) + mockReader.EXPECT().Read(gomock.Any()).Return(nil, nil).AnyTimes() + mockReader.EXPECT().Close().Return(nil).AnyTimes() + + mockWriter := zkafka_mocks.NewMockWriter(ctrl) + mockWriter.EXPECT().WriteRaw(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(zkafka.Response{}, errors.New("error writing to dlt")).AnyTimes() + mockWriter.EXPECT().Close().AnyTimes() + + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(mockReader, nil) + mockClientProvider.EXPECT().Writer(gomock.Any(), gomock.Any()).AnyTimes().Return(mockWriter, nil) + + kwf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(l)) + + processor := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return errors.New("processor error") + }, + } + + dltTopic1 := "dlt-topic2" + w1 := kwf.Create( + zkafka.ConsumerTopicConfig{ + Topic: topicName, + ClientID: uuid.NewString(), + GroupID: uuid.NewString(), + DeadLetterTopicConfig: &zkafka.ProducerTopicConfig{ + ClientID: uuid.NewString(), + Topic: dltTopic1, + }, + }, + &processor, + zkafka.DisableCircuitBreaker(), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + w1.Run(ctx, nil) + }() + + // the previous poll doesn't fully guarantee that the piece of code that + pollWait(func() bool { + return len(processor.ProcessedMessages()) == 10 + }, pollOpts{ + timeoutExit: func() { + require.Failf(t, "Timed out during poll", "Processed Messages %d", len(processor.ProcessedMessages())) + }, + pollPause: time.Millisecond, + maxWait: 10 * time.Second, + }) +} + +func TestWork_DisableDLTWrite(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + mockReader := zkafka_mocks.NewMockReader(ctrl) + msg1 := getRandomMessage() + msg2 := getRandomMessage() + gomock.InOrder( + mockReader.EXPECT().Read(gomock.Any()).Return(msg1, nil), + mockReader.EXPECT().Read(gomock.Any()).Return(msg2, nil), + mockReader.EXPECT().Read(gomock.Any()).Return(nil, nil).AnyTimes(), + ) + mockReader.EXPECT().Close().Return(nil).AnyTimes() + + mockWriter := zkafka_mocks.NewMockWriter(ctrl) + // as we disabled the forwarding, we expect write to be called zero times + mockWriter.EXPECT().WriteRaw(gomock.Any(), gomock.Any(), gomock.Any()).Times(0) + mockWriter.EXPECT().Close().AnyTimes() + + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(mockReader, nil) + mockClientProvider.EXPECT().Writer(gomock.Any(), gomock.Any()).Times(2).Return(mockWriter, nil) + + kwf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(l)) + + processor := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return zkafka.ProcessError{ + Err: errors.New("processor error"), + DisableDLTWrite: true, + } + }, + } + + w1 := kwf.Create( + zkafka.ConsumerTopicConfig{ + Topic: topicName, + DeadLetterTopicConfig: &zkafka.ProducerTopicConfig{ + ClientID: uuid.NewString(), + Topic: "topic2", + }, + }, + &processor, + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + w1.Run(ctx, nil) + }() + + pollWait(func() bool { + return len(processor.ProcessedMessages()) == 2 + }, pollOpts{ + timeoutExit: func() { + require.Fail(t, "Timed out during poll") + }, + pollPause: time.Millisecond, + maxWait: 10 * time.Second, + }) + +} + +// TestWork_Run_OnDoneCallbackCalledOnProcessorError asserts that our callback +// is called on processing error. It does this by registering a callback that will signal a channel when it's called. +// If there's a coding error +func TestWork_Run_OnDoneCallbackCalledOnProcessorError(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + msg := zkafka.GetFakeMessage("key", "val", &zfmt.StringFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).AnyTimes().Return(msg, nil) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(r, nil) + + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + + sig := make(chan struct{}, 1) + + processingError := errors.New("failed processing") + p := &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return processingError + }, + } + var errReceived error + errorCallback := func(ctx context.Context, _ *zkafka.Message, e error) { + errReceived = e + sig <- struct{}{} + } + + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + p, + zkafka.WithOnDone(errorCallback), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go func() { w.Run(ctx, nil) }() + // wait until channel from error callback is written to + <-sig + + require.ErrorIs(t, errReceived, processingError, "Expected processing error to be passed to callback") +} + +func TestWork_Run_WritesMetrics(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + msg := zkafka.GetFakeMessage("key", "val", &zfmt.StringFormatter{}, NoopOnDone) + msg.Topic = topicName + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).MinTimes(1).Return(msg, nil) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(r, nil) + + lhMtx := sync.Mutex{} + lhState := FakeLifecycleState{ + numCalls: map[string]int{}, + } + lh := NewFakeLifecycleHooks(&lhMtx, &lhState) + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithWorkLifecycleHooks(lh)) + + sig := make(chan struct{}, 1) + + p := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return nil + }, + } + + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName, GroupID: "xxx"}, + &p, + zkafka.WithOnDone(func(ctx context.Context, _ *zkafka.Message, e error) { sig <- struct{}{} }), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go func() { w.Run(ctx, nil) }() + // wait until channel from error callback is written to + <-sig +} + +func TestWork_LifecycleHooksCalledForEachItem_Reader(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + numMsgs := 5 + msgs := zkafka.GetFakeMessages(topicName, numMsgs, struct{ name string }{name: "arish"}, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + + gomock.InOrder( + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[0], nil), + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[1], nil), + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[2], nil), + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[3], nil), + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[4], nil), + r.EXPECT().Read(gomock.Any()).AnyTimes().Return(nil, nil), + ) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(r, nil) + + lhMtx := sync.Mutex{} + lhState := FakeLifecycleState{ + numCalls: map[string]int{}, + } + lm := NewFakeLifecycleHooks(&lhMtx, &lhState) + wf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l), zkafka.WithWorkLifecycleHooks(lm)) + p := fakeProcessor{} + + var numProcessedItems int32 + w := wf.Create(zkafka.ConsumerTopicConfig{Topic: topicName, GroupID: "xxx"}, + &p, + zkafka.WithOnDone(func(ctx context.Context, msg *zkafka.Message, err error) { + atomic.AddInt32(&numProcessedItems, 1) + })) + + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return int(atomic.LoadInt32(&numProcessedItems)) == numMsgs + }, pollOpts{ + exit: cancel, + }) + + require.Equal(t, numMsgs, int(atomic.LoadInt32(&numProcessedItems))) + + require.Len(t, lhState.preProMeta, numMsgs) + require.Len(t, lhState.postProMeta, numMsgs) + require.Len(t, lhState.preReadMeta, numMsgs) + require.Equal(t, 0, len(lhState.postAckMeta)) + + require.Equal(t, lhState.preProMeta[0].Topic, topicName) + require.Equal(t, lhState.preProMeta[0].GroupID, "xxx") + require.Equal(t, lhState.preProMeta[0].VirtualPartitionIndex, 0) + + require.Equal(t, lhState.postProMeta[0].Topic, topicName) + require.Equal(t, lhState.postProMeta[0].GroupID, "xxx") + require.Equal(t, lhState.postProMeta[0].VirtualPartitionIndex, 0) +} + +func TestWork_LifecycleHooksPostReadCanUpdateContext(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + numMsgs := 1 + msgs := zkafka.GetFakeMessages(topicName, numMsgs, "lydia", &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + + gomock.InOrder( + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[0], nil), + r.EXPECT().Read(gomock.Any()).AnyTimes().Return(nil, nil), + ) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(r, nil) + + lhMtx := sync.Mutex{} + lhState := FakeLifecycleState{ + numCalls: map[string]int{}, + } + lm := NewFakeLifecycleHooks(&lhMtx, &lhState) + lm.PostRead = func(ctx context.Context, meta zkafka.LifecyclePostReadMeta) (context.Context, error) { + return context.WithValue(ctx, "stewy", "hello"), nil + } + wf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l), zkafka.WithWorkLifecycleHooks(lm)) + var capturedContext context.Context + p := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + capturedContext = ctx + return nil + }, + } + + var numProcessedItems int32 + w := wf.Create(zkafka.ConsumerTopicConfig{Topic: topicName, GroupID: "xxx"}, + &p, + zkafka.WithOnDone(func(ctx context.Context, msg *zkafka.Message, err error) { + atomic.AddInt32(&numProcessedItems, 1) + })) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return int(atomic.LoadInt32(&numProcessedItems)) == numMsgs + }, pollOpts{ + exit: cancel, + }) + + require.Equal(t, capturedContext.Value("stewy"), "hello", "Expect context passed to process to include data injected at post read step") +} + +func TestWork_LifecycleHooksPostReadErrorDoesntHaltProcessing(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + numMsgs := 1 + msgs := zkafka.GetFakeMessages(topicName, numMsgs, "lydia", &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + + gomock.InOrder( + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[0], nil), + r.EXPECT().Read(gomock.Any()).AnyTimes().Return(nil, nil), + ) + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(r, nil) + + lhMtx := sync.Mutex{} + lhState := FakeLifecycleState{ + numCalls: map[string]int{}, + } + lm := NewFakeLifecycleHooks(&lhMtx, &lhState) + lm.PostRead = func(ctx context.Context, meta zkafka.LifecyclePostReadMeta) (context.Context, error) { + return ctx, errors.New("post read hook error") + } + wf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l), zkafka.WithWorkLifecycleHooks(lm)) + p := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return nil + }, + } + + var numProcessedItems int32 + w := wf.Create(zkafka.ConsumerTopicConfig{Topic: topicName, GroupID: "xxx"}, + &p, + zkafka.WithOnDone(func(ctx context.Context, msg *zkafka.Message, err error) { + atomic.AddInt32(&numProcessedItems, 1) + })) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return int(atomic.LoadInt32(&numProcessedItems)) == numMsgs + }, pollOpts{ + exit: cancel, + }) +} + +func TestWork_LifecycleHooksCalledForEachItem(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + numMsgs := 5 + msgs := zkafka.GetFakeMessages(topicName, numMsgs, struct{ name string }{name: "arish"}, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + + gomock.InOrder( + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[0], nil), + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[1], nil), + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[2], nil), + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[3], nil), + r.EXPECT().Read(gomock.Any()).Times(1).Return(msgs[4], nil), + r.EXPECT().Read(gomock.Any()).AnyTimes().Return(nil, nil), + ) + + qp := zkafka_mocks.NewMockClientProvider(ctrl) + qp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(r, nil) + + lhMtx := sync.Mutex{} + lhState := FakeLifecycleState{ + numCalls: map[string]int{}, + } + lh := NewFakeLifecycleHooks(&lhMtx, &lhState) + wf := zkafka.NewWorkFactory(qp, zkafka.WithLogger(l), zkafka.WithWorkLifecycleHooks(lh)) + p := fakeProcessor{} + + var numProcessedItems int32 + w := wf.Create(zkafka.ConsumerTopicConfig{Topic: topicName, GroupID: "xxx"}, + &p, zkafka.WithOnDone(func(ctx context.Context, msg *zkafka.Message, err error) { + atomic.AddInt32(&numProcessedItems, 1) + })) + + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return int(atomic.LoadInt32(&numProcessedItems)) == numMsgs + }, pollOpts{ + exit: cancel, + }) + + require.Equal(t, numMsgs, lhState.numCalls["pre-processing"]) + require.Equal(t, numMsgs, lhState.numCalls["post-processing"]) + require.Equal(t, 0, lhState.numCalls["post-ack"]) +} + +type FakeLifecycleState struct { + numCalls map[string]int + preProMeta []zkafka.LifecyclePreProcessingMeta + postProMeta []zkafka.LifecyclePostProcessingMeta + postAckMeta []zkafka.LifecyclePostAckMeta + preReadMeta []zkafka.LifecyclePostReadMeta + preWriteMeta []zkafka.LifecyclePreWriteMeta +} + +func NewFakeLifecycleHooks(mtx *sync.Mutex, state *FakeLifecycleState) zkafka.LifecycleHooks { + h := zkafka.LifecycleHooks{ + PostRead: func(ctx context.Context, meta zkafka.LifecyclePostReadMeta) (context.Context, error) { + mtx.Lock() + state.numCalls["pre-read"] += 1 + state.preReadMeta = append(state.preReadMeta, meta) + mtx.Unlock() + return ctx, nil + }, + PreProcessing: func(ctx context.Context, meta zkafka.LifecyclePreProcessingMeta) (context.Context, error) { + mtx.Lock() + state.numCalls["pre-processing"] += 1 + state.preProMeta = append(state.preProMeta, meta) + mtx.Unlock() + return ctx, nil + }, + PostProcessing: func(ctx context.Context, meta zkafka.LifecyclePostProcessingMeta) error { + mtx.Lock() + state.numCalls["post-processing"] += 1 + state.postProMeta = append(state.postProMeta, meta) + mtx.Unlock() + return nil + }, + PostAck: func(ctx context.Context, meta zkafka.LifecyclePostAckMeta) error { + mtx.Lock() + state.numCalls["post-ack"] += 1 + state.postAckMeta = append(state.postAckMeta, meta) + mtx.Unlock() + return nil + }, + PreWrite: func(ctx context.Context, meta zkafka.LifecyclePreWriteMeta) (zkafka.LifecyclePreWriteResp, error) { + mtx.Lock() + state.numCalls["pre-write"] += 1 + state.preWriteMeta = append(state.preWriteMeta, meta) + mtx.Unlock() + return zkafka.LifecyclePreWriteResp{}, nil + }, + } + + return h +} + +func getRandomMessage() *zkafka.Message { + return zkafka.GetFakeMessage(fmt.Sprintf("%d", rand.Intn(5)), nil, &zfmt.JSONFormatter{}, NoopOnDone) +} + +func TestWork_CircuitBreaker_WithoutBusyLoopBreaker_DoesNotWaitsForCircuitToOpen(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + msg := zkafka.GetFakeMessage("1", struct{ name string }{name: "arish"}, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).Return(msg, nil).AnyTimes() + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Return(r, nil).AnyTimes() + + l := zkafka.NoopLogger{} + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + + fanoutCount := atomic.Int64{} + processorCount := atomic.Int64{} + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + processorCount.Add(1) + return errors.New("an error occurred during processing") + }, + }, + zkafka.DisableBusyLoopBreaker(), + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + fanoutCount.Add(1) + }}), + zkafka.CircuitBreakAfter(1), + zkafka.CircuitBreakFor(10*time.Second), + ) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + start := time.Now() + go func() { w.Run(ctx, nil) }() + + pollWait(func() bool { + return fanoutCount.Load() >= 100 + }, pollOpts{ + exit: cancel, + timeoutExit: cancel, + }) + require.LessOrEqual(t, processorCount.Load(), int64(2), "circuit breaker should prevent processor from being called after circuit break opens, since circuit breaker won't close again until after test completes. At most two messages are read prior to circuit breaker opening") + require.LessOrEqual(t, time.Since(start), time.Second, "without busy loop breaker we expect fanout to called rapidly. Circuit break is open for 10 seconds. So asserting that fanout was called 100 times in a second is a rough assertion that busy loop breaker is not in effect. Typically these 100 calls should be on the order of micro or nanoseconds. But with resource contention in the pipeline we're more conservative with timing based assertions") +} + +func TestWork_CircuitBreaker_WaitsForCircuitToOpen(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + msg := zkafka.GetFakeMessage("1", struct{ name string }{name: "arish"}, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).Return(msg, nil).AnyTimes() + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Return(r, nil).AnyTimes() + + kwf := zkafka.NewWorkFactory(kcp) + + processCount := atomic.Int64{} + circuitBreakDuration := 10 * time.Millisecond + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + processCount.Add(1) + return errors.New("an error occurred during processing") + }, + }, + zkafka.CircuitBreakAfter(1), + zkafka.CircuitBreakFor(circuitBreakDuration), + ) + + ctx, cancel := context.WithCancel(ctx) + defer cancel() + start := time.Now() + go func() { + w.Run(ctx, nil) + }() + loopCount := int64(5) + for { + if processCount.Load() == loopCount { + cancel() + break + } + time.Sleep(time.Microsecond * 100) + } + require.GreaterOrEqual(t, circuitBreakDuration*time.Duration(loopCount), time.Since(start), "Total time should be greater than circuit break duration * loop count") +} + +// TestWork_DontDeadlockWhenCircuitBreakerIsInHalfOpen this test protects against a bug that was demonstrated in another worker library which implements similar behavior. +// Because of this, this test was written to protect against a regression similar to what was observed in that lib. +// +// This test aims to get the worker into a half open state (by returning processor errors) with short circuit breaker times. +// This test asserts we can process 10 messages in less than 10 seconds (should be able to process in about 1), and assumes +// if we can't a deadlock has occurred +func TestWork_DontDeadlockWhenCircuitBreakerIsInHalfOpen(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + qr := zkafka_mocks.NewMockReader(ctrl) + msg := zkafka.GetFakeMessage("1", struct{ name string }{name: "stewy"}, &zfmt.JSONFormatter{}, NoopOnDone) + gomock.InOrder( + qr.EXPECT().Read(gomock.Any()).Times(1).Return(msg, nil), + qr.EXPECT().Read(gomock.Any()).AnyTimes().Return(nil, nil), + ) + + cp := zkafka_mocks.NewMockClientProvider(ctrl) + cp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(qr, nil) + + wf := zkafka.NewWorkFactory(cp) + + p := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return errors.New("an error occurred during processing") + }, + } + + fanoutCount := atomic.Int64{} + w := wf.Create(zkafka.ConsumerTopicConfig{Topic: topicName}, + &p, + // go into half state almost immediately after processing the message. + zkafka.CircuitBreakFor(time.Microsecond), + // update so we enter open state immediately once one processing error occurs + zkafka.CircuitBreakAfter(1), + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + time.Sleep(time.Millisecond * 100) + fanoutCount.Add(1) + }}), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go func() { + w.Run(ctx, nil) + }() + + start := time.Now() + for { + // if we don't hit a deadlock we should get to 10 loops of Do execution quickly (especially since there's no messages to process after subsequent read) + if fanoutCount.Load() >= 10 { + cancel() + break + } + // take small breaks while polling + time.Sleep(time.Microsecond) + require.GreaterOrEqual(t, 10*time.Second, time.Since(start), "Process timeout likely not being respected. Likely entered a deadlock due to circuit breaker") + } +} + +// Test_Bugfix_WorkPoolCanBeRestartedAfterShutdown this test is in response to a bug +// that occurs during testing. The initial implementation of work (specifically a deprecated Do method) only allowed the worker pool to be started once. +// When the worker was torn down (by cancelling the context), it was unable to be restarted. +// This is primarily a vestigial concern, since the implementation starts a worker pool everytime run is called (instead of having to start it and then stop it and then potentially restart like Do) +// Some test patterns, were relying on the same work instance to be started and stopped multiple times across multiple tests. +func Test_Bugfix_WorkPoolCanBeRestartedAfterShutdown(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + mockReader := zkafka_mocks.NewMockReader(ctrl) + msg1 := zkafka.GetFakeMessage("abc", "def", &zfmt.StringFormatter{}, NoopOnDone) + mockReader.EXPECT().Read(gomock.Any()).Return(msg1, nil).AnyTimes() + mockReader.EXPECT().Close().Return(nil).AnyTimes() + + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(mockReader, nil) + mockClientProvider.EXPECT().Writer(gomock.Any(), gomock.Any()).AnyTimes().Return(nil, nil) + + kwf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(l)) + + processor := fakeProcessor{} + + w := kwf.Create( + zkafka.ConsumerTopicConfig{ + Topic: topicName, + DeadLetterTopicConfig: &zkafka.ProducerTopicConfig{ + ClientID: uuid.NewString(), + Topic: "topic2", + }, + }, + &processor, + ) + + t.Log("Starting first work.Run") + ctx, cancel := context.WithCancel(ctx) + defer cancel() + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + w.Run(context.Background(), ctx.Done()) + wg.Done() + }() + + // wait for at least 1 message to be processed and then cancel the context (which will stop worker) + // and break for loop + for { + if len(processor.ProcessedMessages()) >= 1 { + cancel() + break + } + time.Sleep(time.Millisecond) + } + // wait until worker fully completes and returns + wg.Wait() + t.Log("Completed first work.Run") + + // take a count of how many messages were processed. Because of concurrent processing it might be more than 1 + startCount := len(processor.ProcessedMessages()) + + // Start the worker again (make sure you don't pass in the canceled context). + ctx2, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + w.Run(context.Background(), ctx2.Done()) + }() + + t.Log("Started polling for second work.Run") + + // This is the assertion portion of the test. We're asserting the processing will continue + // and then message count will increase beyond what was originally counted. + // If we exit the test was a success. A bug will indefinitely block + pollWait(func() bool { + return len(processor.ProcessedMessages()) > startCount + }, pollOpts{ + exit: cancel, + timeoutExit: func() { + require.Failf(t, "Polling condition not met prior to test timeout", "Processing count %d, startcount %d", len(processor.ProcessedMessages()), startCount) + }, + maxWait: 10 * time.Second, + }) +} + +// Test_MsgOrderingIsMaintainedPerKeyWithAnyNumberOfVirtualPartitions +// given N messages ordered as follows [{key=0,val=0}, {key=1,val=0}, {key=2,val=0}, {key=0,val=1}, {key=1,val=1}, {key=2,val=1}, ... {key=0,val=N}, {key=1,val=N}, {key=2,val=N}] +// when a work is created with speedup +// then those messages are processed in order per key. To assert this, we track all the messages processed +// and then assert that the value is increasing per key (0, 1 and 2) +func Test_MsgOrderingIsMaintainedPerKeyWithAnyNumberOfVirtualPartitions(t *testing.T) { + defer recoverThenFail(t) + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + mockReader := zkafka_mocks.NewMockReader(ctrl) + var readerCalls []*gomock.Call + keyCount := 3 + msgCount := 200 + for i := 0; i < msgCount; i++ { + msg1 := zkafka.GetFakeMessage(strconv.Itoa(i%keyCount), strconv.Itoa(i), &zfmt.StringFormatter{}, NoopOnDone) + readerCalls = append(readerCalls, mockReader.EXPECT().Read(gomock.Any()).Return(msg1, nil)) + } + readerCalls = append(readerCalls, mockReader.EXPECT().Read(gomock.Any()).Return(nil, nil).AnyTimes()) + gomock.InOrder( + readerCalls..., + ) + mockReader.EXPECT().Close().Return(nil).AnyTimes() + + mockClientProvider := zkafka_mocks.NewMockClientProvider(ctrl) + mockClientProvider.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(mockReader, nil) + mockClientProvider.EXPECT().Writer(gomock.Any(), gomock.Any()).AnyTimes().Return(nil, nil) + + kwf := zkafka.NewWorkFactory(mockClientProvider, zkafka.WithLogger(l)) + + processor := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + time.Sleep(time.Millisecond * time.Duration(rand.Intn(100))) + return nil + }, + } + + w := kwf.Create( + zkafka.ConsumerTopicConfig{ + Topic: topicName, + DeadLetterTopicConfig: &zkafka.ProducerTopicConfig{ + ClientID: uuid.NewString(), + Topic: "topic2", + }, + }, + &processor, + zkafka.Speedup(10), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + w.Run(ctx, nil) + }() + + pollWait(func() bool { + return len(processor.ProcessedMessages()) == msgCount + }, pollOpts{ + exit: cancel, + }) + + keyToMsgs := make(map[string][]*zkafka.Message) + for _, m := range processor.ProcessedMessages() { + keyToMsgs[m.Key] = append(keyToMsgs[m.Key], m) + } + vals0 := make([]int, 0, len(keyToMsgs["0"])) + for _, m := range keyToMsgs["0"] { + i, err := strconv.Atoi(string(m.Value())) + require.NoError(t, err) + vals0 = append(vals0, i) + } + vals1 := make([]int, 0, len(keyToMsgs["1"])) + for _, m := range keyToMsgs["1"] { + i, err := strconv.Atoi(string(m.Value())) + require.NoError(t, err) + vals1 = append(vals1, i) + } + vals2 := make([]int, 0, len(keyToMsgs["2"])) + for _, m := range keyToMsgs["2"] { + i, err := strconv.Atoi(string(m.Value())) + require.NoError(t, err) + vals2 = append(vals2, i) + } + require.IsIncreasingf(t, vals0, "messages for key 0 are not sorted %v", vals0) + require.IsIncreasingf(t, vals1, "messages for key 1 are not sorted") + require.IsIncreasingf(t, vals2, "messages for key 2 are not sorted") +} + +func TestWork_LifecycleHookReaderPanicIsHandledAndMessagingProceeds(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + l := zkafka.NoopLogger{} + + testPanic := func(hooks zkafka.LifecycleHooks) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + qr := zkafka_mocks.NewMockReader(ctrl) + numMsgs := 1 + sentMsg := false + msg := zkafka.GetFakeMessage("1", struct{ name string }{name: "arish"}, &zfmt.JSONFormatter{}, NoopOnDone) + + qr.EXPECT().Read(gomock.Any()).AnyTimes().DoAndReturn(func(ctx context.Context) (*zkafka.Message, error) { + if !sentMsg { + sentMsg = true + return msg, nil + } + return nil, nil + }) + + qp := zkafka_mocks.NewMockClientProvider(ctrl) + qp.EXPECT().Reader(gomock.Any(), gomock.Any()).Times(1).Return(qr, nil) + + wf := zkafka.NewWorkFactory(qp, zkafka.WithLogger(l), zkafka.WithWorkLifecycleHooks(hooks)) + + p := fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return nil + }, + } + + m := sync.Mutex{} + var processedMsgs []*zkafka.Message + topicConfig := zkafka.ConsumerTopicConfig{ + ClientID: "test-config", + GroupID: "group", + Topic: "topic", + Formatter: zfmt.JSONFmt, + } + w := wf.Create(topicConfig, &p, + zkafka.WithOnDone(func(ctx context.Context, msg *zkafka.Message, err error) { + m.Lock() + processedMsgs = append(processedMsgs, msg) + m.Unlock() + }), + ) + + go func() { + w.Run(ctx, nil) + }() + + for { + m.Lock() + msgCount := len(processedMsgs) + m.Unlock() + + if msgCount == numMsgs { + cancel() + break + } + } + + require.Len(t, processedMsgs, numMsgs) + } + + testPanic(zkafka.LifecycleHooks{ + PreProcessing: func(ctx context.Context, meta zkafka.LifecyclePreProcessingMeta) (context.Context, error) { + panic("pre processing panic") + }, + }) + testPanic(zkafka.LifecycleHooks{ + PostProcessing: func(ctx context.Context, meta zkafka.LifecyclePostProcessingMeta) error { + panic("post processing panic") + }, + }) +} + +func TestWork_ShutdownCausesRunExit(t *testing.T) { + defer recoverThenFail(t) + ctx := context.Background() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + msg := zkafka.GetFakeMessage("1", struct{ name string }{name: "arish"}, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).Return(msg, nil).AnyTimes() + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Return(r, nil).AnyTimes() + + l := zkafka.NoopLogger{} + kwf := zkafka.NewWorkFactory(kcp, zkafka.WithLogger(l)) + + fanoutCount := atomic.Int64{} + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &fakeProcessor{}, + zkafka.WithLifecycleHooks(zkafka.LifecycleHooks{PostFanout: func(ctx context.Context) { + fanoutCount.Add(1) + }}), + ) + + settings := &workSettings{ + shutdownSig: make(chan struct{}, 1), + } + go func() { + pollWait(func() bool { + return fanoutCount.Load() >= 1 + }, pollOpts{ + maxWait: 10 * time.Second, + }) + close(settings.shutdownSig) + }() + + err := w.Run(ctx, settings.ShutdownSig()) + require.NoError(t, err) +} + +// $ go test -run=XXX -bench=BenchmarkWork_Run_CircuitBreaker_BusyLoopBreaker -cpuprofile profile_cpu.out +// $ go tool pprof --web profile_cpu.out +// $ go tool pprof -http=":8000" test.test ./profile_cpu.out +func BenchmarkWork_Run_CircuitBreaker_BusyLoopBreaker(b *testing.B) { + b.ReportAllocs() + ctx := context.Background() + + ctrl := gomock.NewController(b) + defer ctrl.Finish() + + msg := zkafka.GetFakeMessage("1", struct{ name string }{name: "arish"}, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).Return(msg, nil).AnyTimes() + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Return(r, nil).AnyTimes() + + kwf := zkafka.NewWorkFactory(kcp) + + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return errors.New("an error occurred during processing") + }, + }, + zkafka.Speedup(10), + zkafka.CircuitBreakAfter(100), + zkafka.CircuitBreakFor(30*time.Millisecond), + ) + + ctx, cancel := context.WithTimeout(ctx, time.Second) + defer cancel() + go func() { + w.Run(ctx, nil) + }() +} + +// $ go test -run=XXX -bench=BenchmarkWork_Run_CircuitBreaker_DisableBusyLoopBreaker -cpuprofile profile_cpu_disable.out +// $ go tool pprof --web profile_cpu_disable.out +// $go tool pprof -http=":8000" test.test ./profile_cpu_disable.out +func BenchmarkWork_Run_CircuitBreaker_DisableBusyLoopBreaker(b *testing.B) { + b.ReportAllocs() + ctx := context.Background() + + ctrl := gomock.NewController(b) + defer ctrl.Finish() + + msg := zkafka.GetFakeMessage("1", struct{ name string }{name: "arish"}, &zfmt.JSONFormatter{}, NoopOnDone) + r := zkafka_mocks.NewMockReader(ctrl) + r.EXPECT().Read(gomock.Any()).Return(msg, nil).AnyTimes() + + kcp := zkafka_mocks.NewMockClientProvider(ctrl) + kcp.EXPECT().Reader(gomock.Any(), gomock.Any()).Return(r, nil).AnyTimes() + + kwf := zkafka.NewWorkFactory(kcp) + + w := kwf.Create( + zkafka.ConsumerTopicConfig{Topic: topicName}, + &fakeProcessor{ + process: func(ctx context.Context, message *zkafka.Message) error { + return errors.New("an error occurred during processing") + }, + }, + zkafka.Speedup(10), + zkafka.CircuitBreakAfter(100), + zkafka.CircuitBreakFor(30*time.Millisecond), + zkafka.DisableBusyLoopBreaker(), + ) + + ctx, cancel := context.WithTimeout(ctx, time.Second) + defer cancel() + go func() { + w.Run(ctx, nil) + }() +} + +func recoverThenFail(t *testing.T) { + if r := recover(); r != nil { + fmt.Print(string(debug.Stack())) + t.Fatal(r) + } +} + +type fakeProcessor struct { + m sync.Mutex + processedMessages []*zkafka.Message + processedContexts []context.Context + process func(context.Context, *zkafka.Message) error +} + +func (p *fakeProcessor) Process(ctx context.Context, msg *zkafka.Message) error { + p.m.Lock() + p.processedMessages = append(p.processedMessages, msg) + p.processedContexts = append(p.processedContexts, ctx) + p.m.Unlock() + if p.process != nil { + return p.process(ctx, msg) + } + return nil +} + +func (p *fakeProcessor) ProcessedMessages() []*zkafka.Message { + p.m.Lock() + defer p.m.Unlock() + + var msgs []*zkafka.Message + for _, m := range p.processedMessages { + msgs = append(msgs, m) + } + return msgs +} + +func (p *fakeProcessor) ProcessedContexts() []context.Context { + p.m.Lock() + defer p.m.Unlock() + + var ctxs []context.Context + for _, ctx := range p.processedContexts { + ctxs = append(ctxs, ctx) + } + return ctxs +} + +type stdLogger struct { + includeDebug bool +} + +func (l stdLogger) Debugw(_ context.Context, msg string, keysAndValues ...interface{}) { + if l.includeDebug { + log.Printf("Debugw-"+msg, keysAndValues...) + } +} + +func (l stdLogger) Infow(_ context.Context, msg string, keysAndValues ...interface{}) { + log.Printf("Infow-"+msg, keysAndValues...) +} + +func (l stdLogger) Errorw(_ context.Context, msg string, keysAndValues ...interface{}) { + log.Printf("Errorw-"+msg, keysAndValues...) +} + +func (l stdLogger) Warnw(_ context.Context, msg string, keysAndValues ...interface{}) { + prefix := fmt.Sprintf("Warnw-%s-"+msg, time.Now().Format(time.RFC3339Nano)) + log.Printf(prefix, keysAndValues...) +} + +type workSettings struct { + shutdownSig chan struct{} +} + +func (w *workSettings) ShutdownSig() <-chan struct{} { + return w.shutdownSig +} + +func ptr[T any](v T) *T { + return &v +} + +type pollOpts struct { + exit func() + timeoutExit func() + pollPause time.Duration + maxWait time.Duration +} + +func pollWait(f func() bool, opts pollOpts) { + maxWait := time.Minute + pollPause := time.Millisecond + + if opts.pollPause != 0 { + pollPause = opts.pollPause + } + if opts.maxWait != 0 { + maxWait = opts.maxWait + } + + start := time.Now() + for { + if f() { + if opts.exit != nil { + opts.exit() + } + return + } + if time.Since(start) > maxWait { + if opts.timeoutExit != nil { + opts.timeoutExit() + } + break + } + time.Sleep(pollPause) + } +} diff --git a/test/writer_test.go b/test/writer_test.go new file mode 100644 index 0000000..bbee877 --- /dev/null +++ b/test/writer_test.go @@ -0,0 +1,67 @@ +//go:build integration +// +build integration + +package test + +import ( + "context" + "sync" + "testing" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + "github.com/zillow/zkafka" + mock_confluent "github.com/zillow/zkafka/mocks/confluent" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +func TestWriter_Write_LifecycleHooksCalled(t *testing.T) { + ctx, f := context.WithCancel(context.Background()) + defer f() + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + lhMtx := sync.Mutex{} + lhState := FakeLifecycleState{ + numCalls: map[string]int{}, + } + lh := NewFakeLifecycleHooks(&lhMtx, &lhState) + + bootstrapServer := getBootstrap() + + mockProducer := mock_confluent.NewMockKafkaProducer(ctrl) + mockProducer.EXPECT().Close().AnyTimes() + mockProducer.EXPECT().Produce(gomock.Any(), gomock.Any()).AnyTimes().DoAndReturn(func(msg *kafka.Message, deliveryChan chan kafka.Event) error { + go func() { + deliveryChan <- &kafka.Message{} + }() + return nil + }) + + client := zkafka.NewClient(zkafka.Config{BootstrapServers: []string{bootstrapServer}}, + zkafka.LoggerOption(stdLogger{}), + zkafka.WithClientLifecycleHooks(lh), + zkafka.WithProducerProvider(func(config map[string]any) (zkafka.KafkaProducer, error) { + return mockProducer, nil + }), + ) + defer func() { _ = client.Close() }() + + writer, err := client.Writer(ctx, zkafka.ProducerTopicConfig{ + ClientID: "writer", + Topic: "topic", + Formatter: zfmt.JSONFmt, + }) + require.NoError(t, err) + + msg := Msg{Val: "1"} + _, err = writer.Write(ctx, msg) + require.NoError(t, err) + + require.Equal(t, 0, lhState.numCalls["pre-processing"]) + require.Equal(t, 0, lhState.numCalls["post-processing"]) + require.Equal(t, 1, lhState.numCalls["post-ack"]) + require.Equal(t, 1, lhState.numCalls["pre-write"]) +} diff --git a/testhelper.go b/testhelper.go new file mode 100644 index 0000000..f7d18e6 --- /dev/null +++ b/testhelper.go @@ -0,0 +1,64 @@ +package zkafka + +import ( + "context" + "fmt" + "sync" + "time" + + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +func GetFakeMessage(key string, value any, fmt zfmt.Formatter, doneFunc func()) *Message { + wrapperFunc := func(c context.Context) { doneFunc() } + return GetFakeMessageWithContext(key, value, fmt, wrapperFunc) +} + +func GetFakeMessages(topic string, numMsgs int, value any, formatter zfmt.Formatter, doneFunc func()) []*Message { + msgs := make([]*Message, numMsgs) + wrapperFunc := func(c context.Context) { doneFunc() } + + for i := 0; i < numMsgs; i++ { + key := fmt.Sprint(i) + msgs[i] = GetFakeMessageWithContext(key, value, formatter, wrapperFunc) + msgs[i].Topic = topic + } + + return msgs +} + +func GetFakeMessageWithContext(key string, value any, fmt zfmt.Formatter, doneFunc func(context.Context)) *Message { + if b, err := fmt.Marshall(value); err == nil { + return &Message{ + Key: key, + Headers: nil, + value: b, + fmt: fmt, + doneFunc: doneFunc, + doneOnce: sync.Once{}, + TimeStamp: time.Now(), + } + } + return &Message{ + Key: key, + doneFunc: doneFunc, + doneOnce: sync.Once{}, + TimeStamp: time.Now(), + } +} + +func GetFakeMsgFromRaw(key *string, value []byte, fmt zfmt.Formatter, doneFunc func(context.Context)) *Message { + k := "" + if key != nil { + k = *key + } + return &Message{ + Key: k, + Headers: nil, + value: value, + fmt: fmt, + doneFunc: doneFunc, + doneOnce: sync.Once{}, + TimeStamp: time.Now(), + } +} diff --git a/testhelper_test.go b/testhelper_test.go new file mode 100644 index 0000000..8a87126 --- /dev/null +++ b/testhelper_test.go @@ -0,0 +1,47 @@ +package zkafka + +import ( + "testing" + + "github.com/stretchr/testify/require" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" +) + +func TestGetFakeMessage(t *testing.T) { + defer recoverThenFail(t) + + msg := GetFakeMessage("key", "value", &zfmt.JSONFormatter{}, nil) + expectedMessage := Message{ + Key: "key", + value: []byte("\"value\""), + } + require.Equal(t, expectedMessage.Key, msg.Key, "Expected generated zkafka.Message to use key from arg") + require.Equal(t, string(expectedMessage.value), string(msg.value), "Expected generated zkafka.Message to use value from arg") +} + +func TestGetFakeMessageFromRaw(t *testing.T) { + defer recoverThenFail(t) + + fmtr := &zfmt.JSONFormatter{} + val, err := fmtr.Marshall("value") + require.NoError(t, err) + msg := GetFakeMsgFromRaw(ptr("key"), val, fmtr, nil) + expectedMessage := Message{ + Key: "key", + value: []byte("\"value\""), + } + require.Equal(t, expectedMessage.Key, msg.Key, "Expected generated zkafka.Message to use key from arg") + require.Equal(t, string(expectedMessage.value), string(msg.value), "Expected generated zkafka.Message to use value from arg") +} + +func TestGetFakeMessage_WhenMarshallError(t *testing.T) { + + // pass in some invalid object for marshalling + msg := GetFakeMessage("key", make(chan int), &zfmt.JSONFormatter{}, nil) + expectedMessage := Message{ + Key: "key", + value: nil, + } + require.Equal(t, expectedMessage.Key, msg.Key, "Expected generated zkafka.Message to use key from arg") + require.Equal(t, string(expectedMessage.value), string(msg.value), "Expected generated zkafka.Message to use value from arg") +} diff --git a/wfoption.go b/wfoption.go new file mode 100644 index 0000000..bed50ea --- /dev/null +++ b/wfoption.go @@ -0,0 +1,61 @@ +package zkafka + +import ( + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/trace" +) + +// WorkFactoryOption interface to identify functional options +type WorkFactoryOption interface { + apply(s *WorkFactory) +} + +// WithLogger provides option to override the logger to use. default is noop +func WithLogger(l Logger) WorkFactoryOption { return loggerOption{l} } + +// WithTracerProvider provides option to specify the otel tracer provider used by the created work object. +// Defaults to nil (which means no tracing) +func WithTracerProvider(tp trace.TracerProvider) WorkFactoryOption { + return tracerProviderOption{tp: tp} +} + +// WithTextMapPropagator provides option to specify the otel text map propagator used by the created work object. +// Defaults to nil (which means no propagation of transport across transports) +func WithTextMapPropagator(p propagation.TextMapPropagator) WorkFactoryOption { + return wfPropagatorsOption{p} +} + +// WithWorkLifecycleHooks provides option to override the lifecycle hooks. Default is noop. +func WithWorkLifecycleHooks(h LifecycleHooks) WorkFactoryOption { + return LifecycleHooksOption{h} +} + +type loggerOption struct{ l Logger } + +func (s loggerOption) apply(wf *WorkFactory) { + if s.l != nil { + wf.logger = s.l + } +} + +type tracerProviderOption struct{ tp trace.TracerProvider } + +func (s tracerProviderOption) apply(wf *WorkFactory) { + if s.tp != nil { + wf.tp = s.tp + } +} + +type wfPropagatorsOption struct { + p propagation.TextMapPropagator +} + +func (s wfPropagatorsOption) apply(wf *WorkFactory) { + wf.p = s.p +} + +type LifecycleHooksOption struct{ h LifecycleHooks } + +func (h LifecycleHooksOption) apply(wf *WorkFactory) { + wf.lifecycle = h.h +} diff --git a/work.go b/work.go new file mode 100644 index 0000000..d9c2150 --- /dev/null +++ b/work.go @@ -0,0 +1,809 @@ +package zkafka + +import ( + "context" + "fmt" + "hash/fnv" + "strconv" + "sync" + "time" + + "github.com/google/uuid" + "github.com/pkg/errors" + "github.com/sony/gobreaker" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/codes" + "go.opentelemetry.io/otel/propagation" + semconv "go.opentelemetry.io/otel/semconv/v1.25.0" + "go.opentelemetry.io/otel/trace" + "golang.org/x/sync/errgroup" +) + +// Work continuously reads and processes messages from the queue (or queues) it's registered to listen to. +// Work executes the following steps +// 1) Read upto MaxNumberOfMessages from the queue(s) using the provided reader. +// 2) Offload the read message to a pool (one per queue) (processing from one queue doesn't block another) +// 3) Each pool has N go routines available to it for concurrent processing. Processing involves executing external code registered via a callback +// 4) On successful processing queue item for batch deletion. +// 5) On errors, apply circuit breaker if configured. + +// Work has a single public method `Run()` which continuously reads and process messages from the topic (or topics) it is registered to listen to. +// `Run()` executes the following steps +// +// 1. Read a kafka.Message using the provided reader. +// 2. Select the virtual partition pool allocated for a specific topic +// 3. Select and write the `kafka.Message` to the pool's virtual partition based on a hash of the `kafka.Message.Key` (virtual partition selection) +// 4. A goroutine is assigned for each virtual partition. Its responsibility is to continously read from its virtual partition, call the Process callback function, and then store the offset of the message. +// +// Additional responsibilities includes: +// 1. Logging +// 2. Executing callbacks of special lifecycle events (useful for observability like tracing/metrics) +// 3. Tripping circuit breaker for error conditions +// 4. Writing to dead letter topic, when configured. +type Work struct { + // kafka topic configuration + topicConfig ConsumerTopicConfig + + kafkaProvider ClientProvider + + // each message will be passed to this processor in a separate go routine. + processor processor + + // logger + logger Logger + + // metrics are handled in the lifecycle hooks. + lifecycle LifecycleHooks + + // reader is created once per work. + rdrMtx sync.RWMutex + reader Reader + + // poolSize is how many goroutines can process messages simultaneously. + // It defines the worker pool size. + // default is 1. + // Use Speedup to control this option + poolSize *uint16 + + messageBuffer chan struct{} + + // virtualPartitions are a list of channels each with an assigned worker goroutine. + // Each message is passed to a virtual partition based on the hash of the message key. + // The same message key will always be written to the same virtual partition. The virtual partition + // extends the concept of a kafka partition to a set of in memory channels. + virtualPartitions []chan workUnit + + // Circuit breaker to throttle reading from the topic. + cb *gobreaker.TwoStepCircuitBreaker + + // Number of consecutive failures allowed before opening the circuit. + // Use CircuitBreakAfter to control + cbAfter *uint32 + + // Duration for which a circuit is open. Use CircuitBreakFor to control + cbFor *time.Duration + + // Disable circuit breaking. Use DisableCircuitBreaker to control + disableCb bool + + // Busy loop breaker. When circuit breaker circuit is open, instead of consuming cpu in a busy loop + // We just block "Do" for the amount of time circuit is going to be open. + // This prevents immediate subsequent calls to Do which we know are going to be noop since circuit is open. + blb busyLoopBreaker + + // onDones is a list of optional callbacks that are called after the processing of a message + onDones []func(ctx context.Context, message *Message, err error) + + tracer trace.Tracer + p propagation.TextMapPropagator +} + +type processorError struct { + inner error +} + +func (w processorError) Error() string { + return fmt.Sprintf("error returned from processor: %s", w.inner.Error()) +} + +func (w processorError) Unwrap() error { + return w.inner +} + +// Run executes a pipeline with a single reader (possibly subscribed to multiple topics) +// fanning out read messages to virtual partitions (preserving message order) and subsequently being processed +// by the registered processor (user code which executes per kafka message). +// +// It returns either after context.Context cancellation or receiving a shutdown signal from settings (both of which +// will cause the awaited execReader and execProcessor methods to return +func (w *Work) Run(ctx context.Context, shutdown <-chan struct{}) error { + w.initiateProcessors(ctx) + + g := errgroup.Group{} + g.Go(func() error { + w.execReader(ctx, shutdown) + return nil + }) + g.Go(func() error { + w.execProcessors(ctx, shutdown) + return nil + }) + return g.Wait() +} + +func (w *Work) execReader(ctx context.Context, shutdown <-chan struct{}) { + defer func() { + w.closeProcessors(ctx) + }() + + for { + select { + case <-ctx.Done(): + return + case <-shutdown: + return + default: + w.fanOut(ctx, shutdown) + } + if w.lifecycle.PostFanout != nil { + w.lifecycle.PostFanout(ctx) + } + } +} + +// execProcessors checks whether the worker pool is in started state or not. If it isn't +// it starts the worker pool in a separate goroutine. The "worker pool" is a collection of goroutines each individually processing a +// virtual partition. When the worker pool goroutine exists, the flag is flipped again, allowing +// for a restart +func (w *Work) execProcessors(ctx context.Context, shutdown <-chan struct{}) { + wg := sync.WaitGroup{} + wg.Add(len(w.virtualPartitions)) + for i := range w.virtualPartitions { + i := i + go func() { + w.processVirtualPartition(ctx, i, shutdown) + wg.Done() + }() + } + wg.Wait() +} + +// initiateProcessors creates a buffered channel for each virtual partition, of size poolSize. That way +// a particular virtual partition never blocks because of its own capacity issue (and instead the goroutinepool is used +// to limit indefinte growth of processing goroutines). +func (w *Work) initiateProcessors(_ context.Context) { + poolSize := w.getPoolSize() + w.virtualPartitions = make([]chan workUnit, poolSize) + for i := 0; i < int(poolSize); i++ { + w.virtualPartitions[i] = make(chan workUnit, poolSize) + } +} + +func (w *Work) closeProcessors(_ context.Context) { + for _, p := range w.virtualPartitions { + close(p) + } +} + +func (w *Work) fanOut(ctx context.Context, shutdown <-chan struct{}) { + successFunc, err := w.cb.Allow() + // If circuit is open, Allow() returns error. + // If circuit is open, we don't read. + if err != nil { + w.logger.Warnw(ctx, "Kafka topic processing circuit open", + "topics", w.topicConfig.topics()) + + w.blb.wait() + return + } + msg, err := w.readMessage(ctx, shutdown) + if err != nil { + w.logger.Warnw(ctx, "Kafka worker read message failed", + "error", err, + "topics", w.topicConfig.topics()) + successFunc(false) + return + } + + if msg == nil { + successFunc(true) + return + } + if w.lifecycle.PostRead != nil { + ctx, err = w.lifecycle.PostRead(ctx, LifecyclePostReadMeta{ + Topic: msg.Topic, + GroupID: msg.GroupID, + Message: msg, + }) + if err != nil { + w.logger.Warnw(ctx, "Error in post read callback in worker", "offset", msg.Offset, "partition", msg.Partition, "topic", msg.Topic, "groupID", msg.GroupID) + } + } + w.logger.Debugw(ctx, "Kafka topic message received", "offset", msg.Offset, "partition", msg.Partition, "topic", msg.Topic, "groupID", msg.GroupID) + + index, err := selectPartitionIndex(msg.Key, msg.isKeyNil, len(w.virtualPartitions)) + if err != nil { + // selectPartitionIndex should never return errors (as long as len(w.virtualPartitions) > 0 which should always be the case + w.logger.Warnw(ctx, "Failed to selected virtual partition index. Choosing 0 index since it is guaranteed to exist", "error", err) + index = 0 + } + select { + case w.messageBuffer <- struct{}{}: + w.virtualPartitions[index] <- workUnit{ + ctx: ctx, + msg: msg, + successFunc: successFunc, + } + case <-shutdown: + w.removeInWork(msg) + break + case <-ctx.Done(): + w.removeInWork(msg) + break + } +} + +func (w *Work) readMessage(ctx context.Context, shutdown <-chan struct{}) (*Message, error) { + if err := w.ensureReader(ctx); err != nil { + return nil, err + } + msg, err := w.reader.Read(ctx) + if err != nil { + return nil, err + } + select { + case <-shutdown: + w.removeInWork(msg) + return nil, nil + case <-ctx.Done(): + w.removeInWork(msg) + return nil, nil + default: + break + } + return msg, nil +} + +// removeInWork is a cleanup function used when messages have been read +// but because wrapup is occurring are decidedly not processed. +// Internally, in progress work is tracked by commit managers and exit is delayed if they're not empty. +// It should be called when ctx cancellation or shutdown signal closure is causing work.Run exit, and a message has been read +func (w *Work) removeInWork(msg *Message) { + if msg == nil { + return + } + reader, ok := w.reader.(*KReader) + if ok && reader != nil { + reader.removeInWork(msg.topicPartition) + } +} + +// processVirtualPartition indefinitely listens for new work units written to its managed partition. +// As they become available, it processes them. Additionally, it is responsible for exiting for context cancellation. +// +// Its important that every message read from the partition is also released from the messageBuffer. Because processSingle has panic recovery, +// and shouldNotCircuitBreak is a tested library function, we can be sure that every message read from the partition will be released from the messageBuffer. +// If this invariant is broken, we could reduce throughput because it is limited by the availability in the messageBuffer +func (w *Work) processVirtualPartition(ctx context.Context, partitionIndex int, shutdown <-chan struct{}) { + partition := w.virtualPartitions[partitionIndex] + delayCalc := delayCalculator{} + for { + select { + case <-ctx.Done(): + return + case unit, ok := <-partition: + // partition has been closed and we should exit + if !ok { + return + } + msg := unit.msg + if msg == nil { + continue + } + + remainingDelay := delayCalc.remaining(w.processDelay(), msg.TimeStamp) + if !w.execDelay(ctx, shutdown, remainingDelay) { + // while waiting for processDelay we received some shutdown signal, so the message should be removed from in flight so it doesn't block during final rebalance + w.removeInWork(msg) + continue + } + err := w.processSingle(unit.ctx, msg, partitionIndex) + unit.successFunc(shouldNotCircuitBreak(err)) + <-w.messageBuffer + } + } +} + +func (w *Work) processDelay() time.Duration { + if w.topicConfig.ProcessDelayMillis == nil || *w.topicConfig.ProcessDelayMillis <= 0 { + return 0 + } + return time.Duration(*w.topicConfig.ProcessDelayMillis) * time.Millisecond +} + +// execDelay blocks, when given a positive processDelay, until that processDelay duration has passed or a signal indicates message processing should begin to exit exited +func (w *Work) execDelay(ctx context.Context, shutdown <-chan struct{}, delay time.Duration) bool { + if delay <= 0 { + return true + } + select { + case <-ctx.Done(): + return false + case <-shutdown: + return false + case <-time.After(delay): + return true + } +} + +func (w *Work) processSingle(ctx context.Context, msg *Message, partitionIndex int) (err error) { + defer func() { + if r := recover(); r != nil { + // in case of panic, we want to confirm message is marked as done. Most of the time this will be redundant, + // but it's possible that Done hasn't been called if panic happens during custom GetRequestContext extraction. + // It's safe to call Done multiple times + msg.Done() + // Panic for one message should not bring down the worker. Log and continue + w.logger.Errorw(ctx, "Kafka topic single message processing panicked", + "recover", r, + "kmsg", msg, + ) + switch x := r.(type) { + case error: + err = x + default: + err = errors.New("kafka topic single message processing panicked") + } + } + }() + + // send the done signal. Always do this. Otherwise, the message won't be committed + defer func() { + msg.DoneWithContext(ctx) + for _, onDone := range w.onDones { + if onDone != nil { + onDone(ctx, msg, err) + } + } + }() + + ctx = w.lifecyclePreProcessing(ctx, msg, partitionIndex) + ctx, span := w.startSpan(ctx, msg) + defer span.End() + + ctxCancel, cancel := context.WithTimeout(ctx, w.processTimeoutDuration()) + defer cancel() + + // In the case of a timeout we'll return an error indicating a timeout occurred. + // Additionally, the context will be canceled, so the processor is told it should release resources. + // If no timeout occurred, we'll return the processor result (either an error or nil). + err = func() error { + processResponses := make(chan error, 1) + + go func() { + defer func() { + if r := recover(); r != nil { + // Panic for one message should not bring down the worker. Log and continue + w.logger.Errorw(ctx, "Kafka topic single message processing panicked", + "recover", r, + "kmsg", msg, + ) + switch x := r.(type) { + case error: + processResponses <- x + default: + processResponses <- errors.New("kafka topic single message processing panicked") + } + } + }() + + begin := time.Now() + e := w.processor.Process(ctxCancel, msg) + w.lifecyclePostProcessing(ctx, msg, partitionIndex, begin, e) + + if e != nil { + processResponses <- processorError{inner: e} + } else { + processResponses <- nil + } + }() + + select { + case err2 := <-processResponses: + return err2 + case <-ctxCancel.Done(): + if errors.Is(ctxCancel.Err(), context.DeadlineExceeded) { + return errors.New("timeout occurred during kafka process") + } + if errors.Is(ctxCancel.Err(), context.Canceled) { + // an outside context will typically be canceled because of a sigterm or siginterrupt. This is often + // part of a natural teardown, and we won't error on this condition + x := ctxCancel.Err() + w.logger.Warnw(ctx, "Outside context canceled", "kmsg", msg, "error", x) + return nil + } + return errors.Wrap(ctxCancel.Err(), "processSingle execution canceled") + } + }() + + if err == nil { + return + } + span.RecordError(err) + span.SetStatus(codes.Error, err.Error()) + + if pError, ok := err.(processorError); ok { + // Because we assume workers will log their own internal errors once + // already, we try to ignore logging them twice by also logging them + // as errors in zkafka (also as this is not considered an 'error' + // in the zkafka library itself). + w.logger.Warnw(ctxCancel, "Kafka topic single message processing failed", + "error", pError.inner, + "kmsg", msg, + ) + } else { + w.logger.Errorw(ctxCancel, "Kafka topic single message processing failed", + "error", err, + "kmsg", msg, + ) + } + return err +} + +func (w *Work) lifecyclePreProcessing(ctx context.Context, msg *Message, partitionIndex int) context.Context { + if w.lifecycle.PreProcessing != nil { + lcPreMeta := LifecyclePreProcessingMeta{ + Topic: msg.Topic, + GroupID: w.topicConfig.GroupID, + VirtualPartitionIndex: partitionIndex, + TopicLag: time.Since(msg.TimeStamp), + Message: msg, + } + + var err error + ctx, err = w.lifecycle.PreProcessing(ctx, lcPreMeta) + if err != nil { + w.logger.Warnw(ctx, "Lifecycle pre-processing failed", "error", err, "meta", lcPreMeta) + } + } + + return ctx +} + +func (w *Work) lifecyclePostProcessing(ctx context.Context, msg *Message, partitionIndex int, begin time.Time, respErr error) { + if w.lifecycle.PostProcessing != nil { + lcPostMeta := LifecyclePostProcessingMeta{ + Topic: msg.Topic, + GroupID: w.topicConfig.GroupID, + VirtualPartitionIndex: partitionIndex, + ProcessingTime: time.Since(begin), + Msg: msg, + ResponseErr: respErr, + } + + lcErr := w.lifecycle.PostProcessing(ctx, lcPostMeta) + if lcErr != nil { + w.logger.Warnw(ctx, "Lifecycle post-processing failed", "error", lcErr, "meta", lcPostMeta) + } + } +} + +func (w *Work) startSpan(ctx context.Context, msg *Message) (context.Context, spanWrapper) { + if msg == nil || w.tracer == nil { + return ctx, spanWrapper{} + } + carrier := &msgCarrier{msg: msg} + + if w.p != nil { + ctx = w.p.Extract(ctx, carrier) + } + + // https://opentelemetry.io/docs/specs/semconv/attributes-registry/messaging/ + const base10 = 10 + offset := strconv.FormatInt(msg.Offset, base10) + opts := []trace.SpanStartOption{ + trace.WithAttributes( + semconv.MessagingSystemKafka, + semconv.MessagingMessageID(offset), + semconv.MessagingKafkaConsumerGroup(w.topicConfig.GroupID), + semconv.MessagingDestinationName(msg.Topic), + semconv.MessagingKafkaDestinationPartition(int(msg.Partition)), + semconv.MessagingKafkaMessageKey(msg.Key), + semconv.MessagingKafkaMessageOffset(int(msg.Offset)), + ), + trace.WithSpanKind(trace.SpanKindConsumer), + } + + operationName := "zkafka.process" + ctx, otelSpan := w.tracer.Start(ctx, operationName, opts...) + + return ctx, spanWrapper{otelSpan} +} + +func (w *Work) processTimeoutDuration() time.Duration { + if w.topicConfig.ProcessTimeoutMillis == nil { + return 60 * time.Second + } + return time.Duration(*w.topicConfig.ProcessTimeoutMillis) * time.Millisecond +} + +func (w *Work) getPoolSize() uint16 { + if w.poolSize == nil || *w.poolSize <= 0 { + return 1 + } + return *w.poolSize +} + +// ensureReader creates a KafkaReader and sets it to the Work field value. +// we don't do this in a sync.Once because during spinup sometimes things aren't ready and errors are returned, and +// we want to do a true retry instead of returning a cached error +func (w *Work) ensureReader(ctx context.Context) error { + w.rdrMtx.RLock() + if w.reader != nil { + w.rdrMtx.RUnlock() + return nil + } + w.rdrMtx.RUnlock() + + w.rdrMtx.Lock() + defer w.rdrMtx.Unlock() + + rdr, err := w.kafkaProvider.Reader(ctx, w.topicConfig) + if err != nil { + return err + } + + if rdr == nil { + return errors.New("nil reader received") + } + + w.reader = rdr + + return nil +} + +// ProcessError wraps an error that a processor encounters, while also exposing +// controls that allow for specifying how the error should be handled. +type ProcessError struct { + // Err is the actual error that the processor encountered. + Err error + // DisableCircuitBreak indicates that this error should be ignored for + // purposes of managing the circuit breaker. Any returned errors where + // this is set to true will not cause the processing of messages to slow. + DisableCircuitBreak bool + // DisableDLTWrite indicates that this message should not be written to + // a dead letter topic (if one is configured) as it cannot be retried + // successfully. + DisableDLTWrite bool +} + +func (p ProcessError) Error() string { + return fmt.Sprintf("err: %s", p.Err.Error()) +} + +func (p ProcessError) Unwrap() error { + return p.Err +} + +// shouldCircuitBreak checks for our bespoke error type, and if it is any other +// type ultimately results in just a nil check. +func shouldNotCircuitBreak(err error) bool { + // we check this in any case to avoid typed nil gotchas + if err == nil { + return true + } + + processError := &ProcessError{} + if ok := errors.As(err, processError); ok { + return processError.DisableCircuitBreak + } + + return false +} + +type processor interface { + // Process is called for each kafka message read. + Process(ctx context.Context, message *Message) error +} + +// WorkFactory creates a work object which reads messages from kafka topic and processes messages concurrently. +type WorkFactory struct { + kafkaProvider ClientProvider + logger Logger + tp trace.TracerProvider + p propagation.TextMapPropagator + lifecycle LifecycleHooks +} + +// NewWorkFactory initializes a new WorkFactory +func NewWorkFactory( + kafkaProvider ClientProvider, + options ...WorkFactoryOption, +) WorkFactory { + factory := WorkFactory{ + kafkaProvider: kafkaProvider, + logger: NoopLogger{}, + } + + for _, option := range options { + if option != nil { + option.apply(&factory) + } + } + return factory +} + +// Create creates a new Work instance. +func (f WorkFactory) Create(topicConfig ConsumerTopicConfig, processor processor, options ...WorkOption) *Work { + work := &Work{ + topicConfig: topicConfig, + kafkaProvider: f.kafkaProvider, + processor: processor, + logger: f.logger, + lifecycle: f.lifecycle, + tracer: getTracer(f.tp), + p: f.p, + } + + if topicConfig.DeadLetterTopicConfig != nil { + cfg := *topicConfig.DeadLetterTopicConfig + options = append(options, WithDeadLetterTopic(cfg)) + } + + for _, option := range options { + option.apply(work) + } + + poolSize := work.getPoolSize() + work.messageBuffer = make(chan struct{}, poolSize) + + cbSetting := gobreaker.Settings{} + + if work.disableCb { + cbSetting.ReadyToTrip = func(gobreaker.Counts) bool { return false } + } else { + if work.cbFor != nil && *work.cbFor > 0 { + cbSetting.Timeout = *work.cbFor + } + if work.cbAfter != nil && *work.cbAfter > 0 { + cbSetting.ReadyToTrip = func(c gobreaker.Counts) bool { return c.ConsecutiveFailures >= *work.cbAfter } + } + b := cbSetting.Timeout + if b == 0 { + b = 60 * time.Second + } + work.blb.maxPause = b + cbSetting.OnStateChange = func(name string, from, to gobreaker.State) { + switch to { + case gobreaker.StateOpen: + + // returned timer ignored. have no need to call Stop on it anyplace yet. + _ = time.AfterFunc(b, func() { work.blb.release() }) + case gobreaker.StateClosed: + work.blb.release() + } + } + } + work.cb = gobreaker.NewTwoStepCircuitBreaker(cbSetting) + + return work +} + +type busyLoopBreaker struct { + disabled bool + mtx sync.Mutex + waiters []chan struct{} + // maxPause indicates the max amount of a time a busyLoopBreaker will wait at the wait() + // site before returning. Can be used to guarantee that wait() doesn't block indefinitely + maxPause time.Duration +} + +func (b *busyLoopBreaker) wait() { + if b.disabled { + return + } + c := make(chan struct{}) + b.mtx.Lock() + b.waiters = append(b.waiters, c) + b.mtx.Unlock() + + timer := time.AfterFunc(b.maxPause, b.release) + // if wait is released externally, we'll want to release this timer's resources + defer timer.Stop() + + <-c +} + +func (b *busyLoopBreaker) release() { + b.mtx.Lock() + defer b.mtx.Unlock() + for _, v := range b.waiters { + close(v) + } + b.waiters = nil +} + +func selectPartitionIndex(key string, isKeyNil bool, partitionCount int) (int, error) { + if partitionCount <= 0 { + return 0, errors.New("partitionCount must be greater than 0") + } + if isKeyNil { + key = uuid.NewString() + } + h := fnv.New32a() + _, err := h.Write([]byte(key)) + if err != nil { + return 0, errors.Wrap(err, "failed to create partition index from seed string") + } + index := int(h.Sum32()) + return index % partitionCount, nil +} + +// workUnit encapsulates the work being written to a virtual partition. It includes +// the context passed in that current iteration of fanout(), the kafka message to be processed and the +// successFunc callback to be called when the work is done (indicating success or failure) +type workUnit struct { + ctx context.Context + msg *Message + successFunc func(bool) +} + +type spanWrapper struct { + span trace.Span +} + +func (s spanWrapper) End(options ...trace.SpanEndOption) { + if s.span == nil { + return + } + s.span.End(options...) +} + +func (s spanWrapper) RecordError(err error, options ...trace.EventOption) { + if s.span == nil { + return + } + s.span.RecordError(err, options...) +} + +func (s spanWrapper) SetStatus(code codes.Code, description string) { + if s.span == nil { + return + } + s.span.SetStatus(code, description) +} + +func (s spanWrapper) SetAttributes(kv ...attribute.KeyValue) { + if s.span == nil { + return + } + s.span.SetAttributes(kv...) +} + +func ptr[T any](v T) *T { + return &v +} + +type delayCalculator struct { + getNow func() time.Time +} + +// remaining calculates the remaining delay which hasn't been observed by subtracting the observed delay (using now-msgTimestamp) from some `target` delay. +// +// example: +// now=3:53, w.processDelay=5s +// timestamp=2:00 -> 0s delay. (delayed long enough). remainingDelay=5s - (3:53 - 2:00) => -1:52:55s. A negative processDelay doesn't end up pausing +// timestamp=3:48 => 0s delay. remainingDelay=5s-(3:53-3:48) =>0s. A 0 (more accurately <=0) processDelay doesn't end up pausing +// timetsamp=3:49 => 1s delay. remainingDelay=5s-(3:53-3:49) => 1s +// timestamp=3:53 => 5s delay. +// timestamp:3:54 => 5s delay. +// timestamp:4:00 => 5s delay (the result is capped by the `targetDelay` +func (c *delayCalculator) remaining(targetDelay time.Duration, msgTimeStamp time.Time) time.Duration { + if c.getNow == nil { + c.getNow = time.Now + } + now := c.getNow() + observedDelay := now.Sub(msgTimeStamp) + // this piece makes sure the return isn't possibly greater than the target + return min(targetDelay-observedDelay, targetDelay) +} diff --git a/work_test.go b/work_test.go new file mode 100644 index 0000000..b9267ba --- /dev/null +++ b/work_test.go @@ -0,0 +1,815 @@ +package zkafka + +import ( + "context" + "errors" + "fmt" + "log" + "math/rand" + "strconv" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/google/uuid" + "github.com/stretchr/testify/require" + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/trace/noop" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/golang/mock/gomock" + mock_confluent "github.com/zillow/zkafka/mocks/confluent" +) + +func TestWork_processTimeoutMillis(t *testing.T) { + type fields struct { + topicConfig ConsumerTopicConfig + } + tests := []struct { + name string + fields fields + want time.Duration + }{ + { + // this case shouldn't happen as zkafka should set a default when this field is nil. But for completeness we'll include it + name: "topic config has specified processTimeoutDuration", + fields: fields{ + topicConfig: ConsumerTopicConfig{ + ProcessTimeoutMillis: ptr(1000), + }, + }, + want: time.Second, + }, + { + // this case shouldn't happen as zkafka should set a default when this field is nil. But for completeness we'll include it + name: "topic config has missing processTimeoutDuration", + fields: fields{ + topicConfig: ConsumerTopicConfig{}, + }, + want: time.Second * 60, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + w := &Work{ + topicConfig: tt.fields.topicConfig, + } + if got := w.processTimeoutDuration(); got != tt.want { + t.Errorf("processTimeoutDuration() = %v, want %v", got, tt.want) + } + }) + } +} + +func TestWork_WithOptions(t *testing.T) { + defer recoverThenFail(t) + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + tp := noop.TracerProvider{} + propagator := propagation.TraceContext{} + + wf := NewWorkFactory(mockClientProvider{}, WithTracerProvider(tp), WithTextMapPropagator(propagator)) + + work := wf.Create(ConsumerTopicConfig{}, &timeDelayProcessor{}) + + require.Equal(t, tp.Tracer(""), work.tracer) +} + +// TestWork_ShouldCommitMessagesProperly asserts the behavior of committing kafka messages. +// Messages should be committed as they complete as long as there aren't lower offset messages still in progress. +// This tests specifies processing delay times such that low offsets finish after high offsets and asserts that the storeOffsets method +// isn't called until these lower offsets complete +// In this test case we read the following {partition, offsets}: +// {1,1}, {2,1}, {1,2}, {1,3} +// and we finish processing them in the following order at time t +// {2,1} t=0, {1,3} t=0, {1,2} t=90, {1,1} t=100 (it should be noted {2,1} and {1,3} have the same specified finish time) +// and could be swapped by chance. Also, the times are using a system clock, so are subject to some wiggle. Hopefully the delays +// are large enough that we don't run into weird behaviors and the assertions can remain unchanged. +// {2,1} comes and is the only message inwork for that partition. A commit is executed +// {1,3} comes in, but {1,2} and {1,1} are still inwork. No commit done as lower offsets are inwork +// {1,2} comes in. Same story as above +// {1,1} comes in, and partition 1 can be committed. We'll only do 1 commit for the largest one {1,3} +// We'll assert that we only see two commits {2,1} and {1,3} +func TestWork_ShouldCommitMessagesProperly(t *testing.T) { + defer recoverThenFail(t) + + type testInput struct { + // the message to be processed + // + msg kafka.Message + // a simulated process delay for the associated message + processDelay time.Duration + } + + topicName := "topic" + now := time.Now() + inputs := []testInput{ + { + msg: kafka.Message{ + TopicPartition: kafka.TopicPartition{Partition: 1, Offset: 1, Topic: &topicName}, + Timestamp: now, + }, + processDelay: time.Millisecond * 100, + }, + { + msg: kafka.Message{ + TopicPartition: kafka.TopicPartition{Partition: 2, Offset: 1, Topic: &topicName}, + Timestamp: now, + }, + processDelay: time.Millisecond * 0, + }, + { + msg: kafka.Message{ + TopicPartition: kafka.TopicPartition{Partition: 1, Offset: 2, Topic: &topicName}, + Timestamp: now, + }, + processDelay: time.Millisecond * 50, + }, + { + msg: kafka.Message{ + TopicPartition: kafka.TopicPartition{Partition: 1, Offset: 3, Topic: &topicName}, + Timestamp: now, + }, + processDelay: time.Millisecond * 0, + }, + } + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + consumer := mock_confluent.NewMockKafkaConsumer(ctrl) + consumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).Times(1) + var consumerCalls []*gomock.Call + + msgToDelay := make(map[key]time.Duration) + for i := range inputs { + input := inputs[i] + consumerCalls = append(consumerCalls, consumer.EXPECT().ReadMessage(gomock.Any()).Return(&input.msg, nil)) + msgToDelay[key{partition: input.msg.TopicPartition.Partition, offset: int64(input.msg.TopicPartition.Offset)}] = input.processDelay + } + consumerCalls = append(consumerCalls, consumer.EXPECT().ReadMessage(gomock.Any()).Return(nil, nil).AnyTimes()) + gomock.InOrder( + consumerCalls..., + ) + + m := sync.Mutex{} + storedOffsets := map[int][]kafka.TopicPartition{} + consumer.EXPECT().StoreOffsets(gomock.Any()).DoAndReturn(func(offsets []kafka.TopicPartition) ([]kafka.TopicPartition, error) { + m.Lock() + defer m.Unlock() + partition := int(offsets[0].Partition) + if _, ok := storedOffsets[partition]; !ok { + storedOffsets[partition] = nil + } + storedOffsets[partition] = append(storedOffsets[partition], offsets[0]) + return offsets, nil + }, + ).MaxTimes(len(inputs)) + + processor := timeDelayProcessor{ + msgToDelay: msgToDelay, + } + + l := NoopLogger{} + + r := KReader{ + tCommitMgr: newTopicCommitMgr(), + consumer: consumer, + topicConfig: ConsumerTopicConfig{ + ReadTimeoutMillis: ptr(1), + }, + logger: l, + } + + wf := WorkFactory{ + logger: l, + tp: noop.TracerProvider{}, + p: propagation.TraceContext{}, + } + + countMtx := sync.Mutex{} + processCount := 0 + var msgTimeStamp time.Time + // we need to be in a concurrent environment so that we can simulate work happening simultaneously and finishing in orders + // different from how the messages are read from the topic. + // Additionally, we'll specify a callback function which will update as messages are processed + work := wf.Create(ConsumerTopicConfig{}, &processor, Speedup(5), WithOnDone(func(ctx context.Context, m *Message, _ error) { + countMtx.Lock() + processCount += 1 + msgTimeStamp = m.TimeStamp + countMtx.Unlock() + })) + work.reader = &r + + // act + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + work.Run(ctx, nil) + }() + + go func() { + for { + if processCount == len(inputs) { + cancel() + } + } + }() + + // However, if there's a bug in our test (or code) this might never occur. + // We'll use this select case to specify a timeout for our tests + select { + case <-ctx.Done(): + break + } + + require.Equal(t, now, msgTimeStamp, "expected timestamp in kafka.Message to be mapped zkafka.Message") + + // These are the largest offsets that are processed. They should show up last (because larger offsets shouldn't be stored + // before smaller offsets) in stored offsets which is ordered + // by the call order to store offsets + expectedPartition1Offset := kafka.TopicPartition{Topic: &topicName, Partition: 1, Offset: 4} + expectedPartition2Offset := kafka.TopicPartition{Topic: &topicName, Partition: 2, Offset: 2} + // + assertContains(t, expectedPartition1Offset, storedOffsets[1]) + assertContains(t, expectedPartition2Offset, storedOffsets[2]) + // last storedOffset should be equal to our expectation + assertEqual(t, expectedPartition1Offset, storedOffsets[1][len(storedOffsets[1])-1]) + assertEqual(t, expectedPartition2Offset, storedOffsets[2][len(storedOffsets[2])-1]) + + partitions := []int32{1, 2} + for _, partition := range partitions { + c := r.tCommitMgr.get(topicName) + require.Contains(t, c.partitionToInWork, partition, "expect inwork message map to contain holder for visited partition") + require.Empty(t, c.partitionToInWork[partition].data, "all messages should be purged from heap") + require.Contains(t, c.partitionToCompleted, partition, "expect partitionToCompleted message map to contain holder for visited partition") + require.Empty(t, c.partitionToCompleted[partition].data, "all messages should be purged from heap") + } +} + +// TestWork_CommitManagerBeEmptyAfterAllProcessingCompletes asserts that the heaps tracked by the commitManager +// is empty after the conclusion of processing. This should be true as the commitManager is only responsible for +// tracking commits that have yet to be committed (typically because they finish out of order) +func TestWork_CommitManagerIsEmptyAfterAllProcessingCompletes(t *testing.T) { + defer recoverThenFail(t) + type testInput struct { + // the message to be processed + msg kafka.Message + // a simulated process delay for the associated message + processDelay time.Duration + } + + // arrange many kafka messages with random amounts of processing delay + var inputs []testInput + messageCount := 10000 + partitionCount := 3 + topicName := "topic-name" + for i := 0; i < messageCount; i++ { + randPartition := i % partitionCount + offset := i + randDelayMillis := rand.Intn(10) + inputs = append(inputs, testInput{ + msg: kafka.Message{ + TopicPartition: kafka.TopicPartition{Partition: int32(randPartition), Offset: kafka.Offset(offset), Topic: &topicName}, + Timestamp: time.Now(), + }, + processDelay: time.Duration(randDelayMillis) * time.Millisecond, + }) + } + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + consumer := mock_confluent.NewMockKafkaConsumer(ctrl) + consumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).Times(1) + var consumerCalls []*gomock.Call + + msgToDelay := make(map[key]time.Duration) + for i := range inputs { + input := inputs[i] + consumerCalls = append(consumerCalls, consumer.EXPECT().ReadMessage(gomock.Any()).Return(&input.msg, nil)) + msgToDelay[key{partition: input.msg.TopicPartition.Partition, offset: int64(input.msg.TopicPartition.Offset)}] = input.processDelay + } + // consumer.ReadMessage may get called multiple times after messages have been exhausted (this setup is for that scenario) + consumerCalls = append(consumerCalls, consumer.EXPECT().ReadMessage(gomock.Any()).Return(nil, nil).AnyTimes()) + // setup consumer so that it reads input.msg1, input.msg2....input.msgN + gomock.InOrder( + consumerCalls..., + ) + + consumer.EXPECT().StoreOffsets(gomock.Any()).MaxTimes(messageCount) + + processor := timeDelayProcessor{ + msgToDelay: msgToDelay, + } + + l := NoopLogger{} + + r := KReader{ + tCommitMgr: newTopicCommitMgr(), + consumer: consumer, + topicConfig: ConsumerTopicConfig{ + ReadTimeoutMillis: ptr(1), + }, + logger: l, + } + + wf := WorkFactory{ + logger: l, + tp: noop.TracerProvider{}, + p: propagation.TraceContext{}, + } + + countMtx := sync.Mutex{} + processCount := atomic.Int64{} + // we need to be in a concurrent environment so that we can simulate work happening simultaneously and finishing in orders + // different from how the messages are read from the topic. + // Additionally, we'll specify a callback function which will update as messages are processed + gopoolsize := 100 + work := wf.Create(ConsumerTopicConfig{}, &processor, Speedup(uint16(gopoolsize)), WithOnDone(func(_ context.Context, _ *Message, _ error) { + countMtx.Lock() + processCount.Add(1) + countMtx.Unlock() + })) + work.reader = &r + + ctx, cancel := context.WithCancel(context.Background()) + go func() { + work.Run(ctx, nil) + }() + + // we want to wait until all our input messages have been processed. + sig := make(chan struct{}) + go func() { + for { + if processCount.Load() == int64(len(inputs)) { + sig <- struct{}{} + } + time.Sleep(time.Millisecond) + } + }() + + // However, if there's a bug in our test (or code) this might never occur. + // We'll use this select case to specify a timeout for our tests + select { + case <-time.After(time.Second * 10): + cancel() + require.Fail(t, "test did not complete in expected time") + case <-sig: + cancel() + break + } + + for partition := 0; partition < partitionCount; partition++ { + c := r.tCommitMgr.get(topicName) + require.Contains(t, c.partitionToInWork, int32(partition), "expect inwork message map to contain holder for visited partition") + require.Empty(t, c.partitionToInWork[int32(partition)].data, "expect inwork message map to contain holder for visited partition") + require.Contains(t, c.partitionToCompleted, int32(partition), "expect partitionToCompleted message map to contain holder for visited partition") + require.Empty(t, c.partitionToInWork[int32(partition)].data, "expect inwork message map to contain holder for visited partition") + } +} + +// TestWork_WithDoneWithContext asserts that the context in the done +// callback matches the context of the processed message. +func TestWork_WithDoneWithContext(t *testing.T) { + defer recoverThenFail(t) + type testInput struct { + // the message to be processed + msg kafka.Message + // a simulated process delay for the associated message + processDelay time.Duration + } + + // arrange many kafka messages with random amounts of processing delay + var inputs []testInput + messageCount := 100 + partitionCount := 3 + topicName := "topic-name" + for i := 0; i < messageCount; i++ { + randPartition := i % partitionCount + offset := i + randDelayMillis := rand.Intn(10) + inputs = append(inputs, testInput{ + msg: kafka.Message{ + TopicPartition: kafka.TopicPartition{Partition: int32(randPartition), Offset: kafka.Offset(offset), Topic: &topicName}, + Timestamp: time.Now(), + Key: []byte(strconv.Itoa(i)), + }, + processDelay: time.Duration(randDelayMillis) * time.Millisecond, + }) + } + + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + consumer := mock_confluent.NewMockKafkaConsumer(ctrl) + consumer.EXPECT().SubscribeTopics(gomock.Any(), gomock.Any()).Times(1) + var consumerCalls []*gomock.Call + + msgToDelay := make(map[key]time.Duration) + for i := range inputs { + input := inputs[i] + consumerCalls = append(consumerCalls, consumer.EXPECT().ReadMessage(gomock.Any()).Return(&input.msg, nil)) + msgToDelay[key{partition: input.msg.TopicPartition.Partition, offset: int64(input.msg.TopicPartition.Offset)}] = input.processDelay + } + // consumer.ReadMessage may get called multiple times after messages have been exhausted (this setup is for that scenario) + consumerCalls = append(consumerCalls, consumer.EXPECT().ReadMessage(gomock.Any()).Return(nil, nil).AnyTimes()) + // setup consumer so that it reads input.msg1, input.msg2....input.msgN + gomock.InOrder( + consumerCalls..., + ) + + consumer.EXPECT().StoreOffsets(gomock.Any()).MaxTimes(messageCount) + + processor := timeDelayProcessor{ + msgToDelay: msgToDelay, + } + + r := KReader{ + tCommitMgr: newTopicCommitMgr(), + consumer: consumer, + topicConfig: ConsumerTopicConfig{ + ReadTimeoutMillis: ptr(1), + }, + logger: NoopLogger{}, + } + + wf := WorkFactory{ + logger: NoopLogger{}, + tp: noop.TracerProvider{}, + p: propagation.TraceContext{}, + } + + countMtx := sync.Mutex{} + processCount := atomic.Int64{} + + // Keep an array of the message keys seen (message keys were chosen to be string representation of ints) + // Check that all have been seen after all messages are processed. + msgsSeen := map[string]bool{} + + // we need to be in a concurrent environment so that we can simulate work happening simultaneously and finishing in orders + // different from how the messages are read from the topic. + // Additionally, we'll specify a callback function which will update as messages are processed + gopoolsize := 100 + work := wf.Create(ConsumerTopicConfig{}, &processor, Speedup(uint16(gopoolsize)), WithOnDone(func(ctx context.Context, msg *Message, _ error) { + countMtx.Lock() + processCount.Add(1) + msgsSeen[msg.Key] = true // Mark this context as seen + countMtx.Unlock() + })) + work.reader = &r + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + work.Run(ctx, nil) + }() + + // we want to wait until all our input messages have been processed. + for { + if processCount.Load() == int64(len(inputs)) { + break + } + time.Sleep(time.Microsecond * 100) + } + + // Make sure all messages were processed + for _, i := range inputs { + require.Contains(t, msgsSeen, string(i.msg.Key), "msg.Key not marked as seen (indicating it was not processed)") + } +} + +func Test_busyLoopBreaker_waitRespectsMaxPause(t *testing.T) { + defer recoverThenFail(t) + blb := busyLoopBreaker{ + mtx: sync.Mutex{}, + maxPause: time.Microsecond, + } + // if this doesn't respect maxPause it would pause here indefinitely + blb.wait() +} + +// Test_busyLoopBreaker_waitRespectsRelease asserts that calling release() cancels that wait occuring at the wait() site +func Test_busyLoopBreaker_waitRespectsRelease(t *testing.T) { + defer recoverThenFail(t) + blb := busyLoopBreaker{ + mtx: sync.Mutex{}, + maxPause: time.Second * 100, + } + + // call blb.Wait() and only once it relinquishes that wait will we signal that its finished. + // This signal can be used versus a timeout to assert + blbFinishedWait := make(chan struct{}) + go func() { + blb.wait() + blbFinishedWait <- struct{}{} + }() + + // wait a moment, so we can approximately guarantee that blb.wait has been called + time.Sleep(time.Millisecond * 100) + blb.release() + + select { + case <-time.After(time.Second * 5): + t.Fatalf("Test reached timeout of 5 seconds. blb.Release() method didnt return from blb.wait() site") + case <-blbFinishedWait: + break + } +} + +func Test_ShouldNotCircuitBreak(t *testing.T) { + type testCase struct { + err error + shouldNotCircuitBreak bool + description string + } + + tests := []testCase{ + { + err: nil, + shouldNotCircuitBreak: true, + description: "nil error should not trigger circuit breaker", + }, + { + err: errors.New("foobar"), + shouldNotCircuitBreak: false, + description: "generic error should trigger circuit breaker", + }, + { + err: ProcessError{ + Err: errors.New("foobar"), + DisableCircuitBreak: true, + }, + shouldNotCircuitBreak: true, + description: "processerror with circuit break flag set should not trigger circuit breaker", + }, + { + err: ProcessError{ + Err: errors.New("foobar"), + DisableCircuitBreak: false, + }, + shouldNotCircuitBreak: false, + description: "processerror with circuit break flag NOT set should trigger circuit breaker", + }, + { + err: processorError{ + inner: errors.New("foobar"), + }, + shouldNotCircuitBreak: false, + description: "workererror defaults to circuit break", + }, + { + err: processorError{ + inner: ProcessError{ + Err: errors.New("foobar"), + DisableCircuitBreak: true, + }, + }, + shouldNotCircuitBreak: true, + description: "workererror is properly unwrapped to check for processerror flag", + }, + { + err: processorError{ + inner: ProcessError{ + Err: errors.New("foobar"), + DisableCircuitBreak: false, + }, + }, + shouldNotCircuitBreak: false, + description: "workererror is properly unwrapped to check for processerror flag", + }, + } + + for _, test := range tests { + t.Run(test.description, func(t *testing.T) { + defer recoverThenFail(t) + require.Equalf(t, test.shouldNotCircuitBreak, shouldNotCircuitBreak(test.err), test.description) + }) + } +} + +// Test_selectPartitionIndex_SelectsDifferentForDifferentInputsSometimes tests that the selectPartitionIndex function +// will return different values for different inputs sometimes. This is a probabilistic test, so it's possible that +// there are no collisions in 10 runs, but the probability of that is (1/10)^10, so it's unlikely. +// This test is meant to catch any obvious issues with the implementation (always returning the same index). +func Test_selectPartitionIndex_SelectsDifferentForDifferentInputsSometimes(t *testing.T) { + for i := 0; i < 10; i++ { + str1 := uuid.NewString() + str2 := uuid.NewString() + index1, err := selectPartitionIndex(str1, false, 10) + require.NoError(t, err) + + index2, err := selectPartitionIndex(str2, false, 10) + require.NoError(t, err) + // break early, because we got generated different indexes for different inputs + if index1 != index2 { + return + } + } + t.Fatal(t, "10 executions of SelectIndex were run which, for a proper implementation of SelectIndex has a (1/10)^10 chance of generating collisions each time."+ + " Likely there's an issue with the implementation") +} + +func Test_selectPartitionIndex_SelectsDifferentForEmptyStringWithNilKeySometimes(t *testing.T) { + for i := 0; i < 10; i++ { + index1, err := selectPartitionIndex("", true, 10) + require.NoError(t, err) + + index2, err := selectPartitionIndex("", true, 10) + require.NoError(t, err) + // break early, because we got generated different indexes for different inputs + if index1 != index2 { + return + } + } + t.Fatal("10 executions of SelectIndex with empty key marked as nil were run which, for a proper implementation of SelectIndex has a (1/10)^10 chance of generating collisions each time." + + " Likely there's an issue with the implementation") +} + +func Test_selectPartitionIndex_SelectsSamePartitionWithEmptyKey(t *testing.T) { + for i := 0; i < 10; i++ { + index1, err := selectPartitionIndex("", false, 10) + require.NoError(t, err) + + index2, err := selectPartitionIndex("", false, 10) + require.NoError(t, err) + + require.Equal(t, index1, index2, "Expected that an empty string key always selects the same partition") + } +} + +func Test_calcDelay(t *testing.T) { + now := time.Now() + dc := delayCalculator{getNow: func() time.Time { return now }} + + type testCase struct { + name string + configuredDelay time.Duration + msgTimestamp time.Time + expectedDelay time.Duration + } + + testCases := []testCase{ + { + name: "timely-message-pays-full-delay-penalty", + configuredDelay: time.Hour, + msgTimestamp: now, + expectedDelay: time.Hour, + }, + { + name: "message-delayed-by-1-second-pays-delay-penalty-less-that-already-incurred-penalty", + configuredDelay: 5 * time.Second, + msgTimestamp: now.Add(-1 * time.Second), + expectedDelay: 4 * time.Second, + }, + { + name: "message-delayed-by-2-second-pays-delay-penalty-less-that-already-incurred-penalty", + configuredDelay: 5 * time.Second, + msgTimestamp: now.Add(-2 * time.Second), + expectedDelay: 3 * time.Second, + }, + { + name: "message-delayed-by-4-second-pays-delay-penalty-less-that-already-incurred-penalty", + configuredDelay: 5 * time.Second, + msgTimestamp: now.Add(-4 * time.Second), + expectedDelay: 1 * time.Second, + }, + { + name: "message-delayed-by-configureDela-second-pays-no-delay-penalty", + configuredDelay: 5 * time.Second, + msgTimestamp: now.Add(-5 * time.Second), + expectedDelay: 0 * time.Second, + }, + { + name: "future-messages-dont-incur-additional-delay", + configuredDelay: 5 * time.Second, + msgTimestamp: now.Add(1 * time.Second), + expectedDelay: 5 * time.Second, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + delay := dc.remaining(tc.configuredDelay, tc.msgTimestamp) + require.Equal(t, delay, tc.expectedDelay) + }) + } +} + +func Fuzz_selectPartitionIndexAlwaysReturnsErrorWhenPartitionCountIsNotPositive(f *testing.F) { + f.Add("hello", -3) + f.Fuzz(func(t *testing.T, s string, max int) { + if max > 0 { + t.Skip() + } + _, err := selectPartitionIndex(s, false, max) + require.Error(t, err) + }) +} + +func Fuzz_selectPartitionIndexNeverReturnsErrorWhenPartitionCountIsPositive(f *testing.F) { + f.Add("hello", 9) + f.Fuzz(func(t *testing.T, s string, max int) { + if max < 1 { + t.Skip() + } + index, err := selectPartitionIndex(s, false, max) + require.NoError(t, err) + require.Less(t, index, max) + }) +} + +func Fuzz_selectPartitionIndexReturnsSameIndexForSameString(f *testing.F) { + f.Add("hello") + f.Fuzz(func(t *testing.T, s string) { + index1, err := selectPartitionIndex(s, false, 10) + require.NoError(t, err) + index2, err := selectPartitionIndex(s, false, 10) + require.NoError(t, err) + require.Equal(t, index1, index2, "Selected index should be the same for the same string") + }) +} + +// Fuzz_AnySpeedupInputAlwaysCreatesABufferedChannel tests that the speedup parameter always creates a buffered channel +// for messageBuffer and virtualParititions +// messageBuffer is used to limit the number of outstanding messages that can be read from kafka which haven't been processed +func Fuzz_AnySpeedupInputAlwaysCreatesABufferedChannel(f *testing.F) { + f.Add(uint16(9)) + + f.Fuzz(func(t *testing.T, speedup uint16) { + wf := NewWorkFactory(mockClientProvider{}) + p := timeDelayProcessor{} + w := wf.Create(ConsumerTopicConfig{}, &p, Speedup(speedup)) + require.Greater(t, cap(w.messageBuffer), 0) + }) +} + +type stdLogger struct{} + +func (l stdLogger) Debugw(_ context.Context, msg string, keysAndValues ...any) { + // log.Printf("Debugw-"+msg, keysAndValues...) +} + +func (l stdLogger) Infow(_ context.Context, msg string, keysAndValues ...any) { + log.Printf("Infow-"+msg, keysAndValues...) +} + +func (l stdLogger) Errorw(_ context.Context, msg string, keysAndValues ...any) { + log.Printf("Errorw-"+msg, keysAndValues...) +} + +func (l stdLogger) Warnw(_ context.Context, msg string, keysAndValues ...any) { + log.Printf("Warnw-"+msg, keysAndValues...) +} + +type key struct { + partition int32 + offset int64 +} + +// timeDelayProcessor allows the simulation of processing delay on a per-message basis. +type timeDelayProcessor struct { + // msgToDelay stores how long a particular messages simulated delay should be. It uses the offset and partition to identify the processDelay + msgToDelay map[key]time.Duration +} + +func (m *timeDelayProcessor) Process(_ context.Context, message *Message) error { + timeDelay := m.msgToDelay[key{partition: message.Partition, offset: message.Offset}] + time.Sleep(timeDelay) + return nil +} + +type mockClientProvider struct{} + +func (mockClientProvider) Reader(ctx context.Context, topicConfig ConsumerTopicConfig, opts ...ReaderOption) (Reader, error) { + return nil, nil +} + +func (mockClientProvider) Writer(ctx context.Context, topicConfig ProducerTopicConfig, opts ...WriterOption) (Writer, error) { + return nil, nil +} + +func (mockClientProvider) Close() error { + return nil +} + +func assertContains(t *testing.T, wantIn kafka.TopicPartition, options []kafka.TopicPartition) { + t.Helper() + for _, want := range options { + if wantIn == want { + return + } + } + msg := fmt.Sprintf("expected wantIn to appear in provided options\nwantIn: %s\noptions: %+v\n", wantIn, options) + t.Fatal(msg) +} + +type workSettings struct { + shutdownSig chan struct{} +} + +func (w *workSettings) ShutdownSig() <-chan struct{} { + return w.shutdownSig +} diff --git a/workoption.go b/workoption.go new file mode 100644 index 0000000..dea7b3c --- /dev/null +++ b/workoption.go @@ -0,0 +1,143 @@ +package zkafka + +import ( + "context" + "errors" + "time" +) + +// WorkOption interface to identify functional options +type WorkOption interface { + apply(s *Work) +} + +// Speedup increases the concurrencyFactor for a worker. +// concurrencyFactor is how many go routines can be running in parallel. +// NOTE: it's strongly recommended to add more worker instances rather than using this option to speed up each worker. +func Speedup(times uint16) WorkOption { return speedupOption{times: times} } + +// CircuitBreakAfter these many consecutive failures +func CircuitBreakAfter(times uint32) WorkOption { + return circuitBreakAfterOption{times: times} +} + +// CircuitBreakFor sets the duration for which to keep the circuit open once broken +func CircuitBreakFor(duration time.Duration) WorkOption { + return circuitBreakForOption{duration: duration} +} + +// DisableCircuitBreaker disables the circuit breaker so that it never breaks +func DisableCircuitBreaker() WorkOption { + return disableCbOption{} +} + +// DisableBusyLoopBreaker disables the busy loop breaker which would block subsequent read calls till the circuit re-closes. +// Without blb we see increased cpu usage when circuit is open +func DisableBusyLoopBreaker() WorkOption { + return disableBlbOption{} +} + +// WithOnDone allows you to specify a callback function executed after processing of a kafka message +func WithOnDone(f func(ctx context.Context, message *Message, err error)) WorkOption { + return onDoneOption{f: f} +} + +func WithLifecycleHooks(h LifecycleHooks) WorkOption { + return lifeCycleOption{lh: h} +} + +// WithDeadLetterTopic allows you to specify a dead letter topic to forward messages to when work processing fails +func WithDeadLetterTopic(deadLetterTopicConfig ProducerTopicConfig) WorkOption { + return dltOption{dltConfig: deadLetterTopicConfig} +} + +type speedupOption struct{ times uint16 } + +func (s speedupOption) apply(w *Work) { + if s.times > 0 { + w.poolSize = &s.times + } +} + +type circuitBreakAfterOption struct{ times uint32 } + +func (c circuitBreakAfterOption) apply(w *Work) { + if c.times > 0 { + w.cbAfter = &c.times + } +} + +type circuitBreakForOption struct{ duration time.Duration } + +func (c circuitBreakForOption) apply(w *Work) { + if c.duration > 0 { + w.cbFor = &c.duration + } +} + +type disableCbOption struct{} + +func (d disableCbOption) apply(w *Work) { + w.disableCb = true +} + +type onDoneOption struct { + f func(ctx context.Context, message *Message, err error) +} + +func (d onDoneOption) apply(w *Work) { + if d.f != nil { + w.onDones = append(w.onDones, d.f) + } +} + +type lifeCycleOption struct { + lh LifecycleHooks +} + +func (o lifeCycleOption) apply(w *Work) { + w.lifecycle = o.lh +} + +type disableBlbOption struct{} + +func (d disableBlbOption) apply(w *Work) { + w.blb.disabled = true +} + +type dltOption struct { + dltConfig ProducerTopicConfig +} + +func (d dltOption) apply(w *Work) { + f := func(ctx context.Context, message *Message, errProc error) { + if message == nil { + return + } + + // even if we're going to skip forwarding a message to the DLT (because there was no error), + // establish a writer to the DLT early, so when the time comes the write is fast + writer, err := w.kafkaProvider.Writer(ctx, d.dltConfig) + if err != nil { + w.logger.Errorw(ctx, "Failed to get writer for dlt", "error", err, "offset", message.Offset, "partition", message.Partition, "topic", message.Topic) + return + } + + // only write to dlt if an error occurred + if errProc == nil { + return + } + + processError := ProcessError{} + if ok := errors.As(errProc, &processError); ok { + if processError.DisableDLTWrite { + return + } + } + + if _, err := writer.WriteRaw(ctx, &message.Key, message.value); err != nil { + w.logger.Errorw(ctx, "Failed to forward to DLT", "error", err, "offset", message.Offset, "partition", message.Partition, "topic", message.Topic) + } + } + w.onDones = append(w.onDones, f) +} diff --git a/writer.go b/writer.go new file mode 100644 index 0000000..b14bea1 --- /dev/null +++ b/writer.go @@ -0,0 +1,249 @@ +package zkafka + +//go:generate mockgen -package mock_confluent -destination=./mocks/confluent/kafka_producer.go . KafkaProducer + +import ( + "context" + "sync" + "time" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/pkg/errors" + "go.opentelemetry.io/otel/codes" + "go.opentelemetry.io/otel/propagation" + semconv "go.opentelemetry.io/otel/semconv/v1.25.0" + "go.opentelemetry.io/otel/trace" +) + +// Writer is the convenient interface for kafka KWriter +type Writer interface { + // Write sends messages to kafka with message key set as nil. + // The value arg passed to this method is marshalled by + // the configured formatter and used as the kafka message's value + Write(ctx context.Context, value any, opts ...WriteOption) (Response, error) + // WriteKey send message to kafka with a defined keys. + // The value arg passed to this method is marshalled by + // the configured formatter and used as the kafka message's value + WriteKey(ctx context.Context, key string, value any, opts ...WriteOption) (Response, error) + // WriteRaw sends messages to kafka. The caller is responsible for marshalling the data themselves. + WriteRaw(ctx context.Context, key *string, value []byte, opts ...WriteOption) (Response, error) + Close() +} + +// static type checking for the convenient Writer interface +var _ Writer = (*KWriter)(nil) + +type KafkaProducer interface { + Produce(msg *kafka.Message, deliveryChan chan kafka.Event) error + Close() +} + +var _ KafkaProducer = (*kafka.Producer)(nil) + +// KWriter is a kafka producer. KWriter should be initialized from the Client to be usable +type KWriter struct { + mu sync.Mutex + producer KafkaProducer + topicConfig ProducerTopicConfig + fmtter Formatter + logger Logger + tracer trace.Tracer + p propagation.TextMapPropagator + lifecycle LifecycleHooks + isClosed bool +} + +type keyValuePair struct { + key *string + value any +} + +func newWriter(conf Config, topicConfig ProducerTopicConfig, producer confluentProducerProvider) (*KWriter, error) { + confluentConfig := makeProducerConfig(conf, topicConfig) + p, err := producer(confluentConfig) + if err != nil { + return nil, err + } + fmtter, err := getFormatter(topicConfig) + if err != nil { + return nil, err + } + return &KWriter{ + producer: p, + fmtter: fmtter, + topicConfig: topicConfig, + logger: NoopLogger{}, + }, nil +} + +// Write sends messages to kafka with message key set as nil. +// The value arg passed to this method is marshalled by +// the configured formatter and used as the kafka message's value +func (w *KWriter) Write(ctx context.Context, value any, opts ...WriteOption) (Response, error) { + return w.write(ctx, keyValuePair{value: value}, opts...) +} + +// WriteKey send message to kafka with a defined keys. +// The value arg passed to this method is marshalled by +// the configured formatter and used as the kafka message's value +func (w *KWriter) WriteKey(ctx context.Context, key string, value any, opts ...WriteOption) (Response, error) { + return w.write(ctx, keyValuePair{ + key: &key, + value: value, + }, opts...) +} + +// WriteRaw allows you to write messages using a lower level API than Write and WriteKey. +// WriteRaw raw doesn't use a formatter to marshall the value data and instead takes the bytes as is and places them +// as the value for the kafka message +// It's convenient for forwarding message in dead letter operations. +func (w *KWriter) WriteRaw(ctx context.Context, key *string, value []byte, opts ...WriteOption) (Response, error) { + kafkaMessage := makeProducerMessageRaw(ctx, w.topicConfig.ClientID, w.topicConfig.Topic, key, value) + for _, opt := range opts { + opt.apply(&kafkaMessage) + } + if w.lifecycle.PreWrite != nil { + resp, err := w.lifecycle.PreWrite(ctx, LifecyclePreWriteMeta{}) + if err != nil { + w.logger.Warnw(ctx, "Lifecycle pre-write failed", "error", err) + } + kafkaMessage = addHeaders(kafkaMessage, resp.Headers) + } + + w.logger.Debugw(ctx, "write message", "message", kafkaMessage) + span := w.startSpan(ctx, &kafkaMessage) + defer span.End() + + deliveryChan := make(chan kafka.Event) + begin := time.Now() + err := w.producer.Produce(&kafkaMessage, deliveryChan) + if err != nil { + span.RecordError(err) + span.SetStatus(codes.Error, err.Error()) + return Response{}, errors.Wrap(err, "error writing message") + } + // wait on callback channel for kafka broker to ack written message + e := <-deliveryChan + + w.lifecyclePostAck(ctx, begin) + m := e.(*kafka.Message) + + span.SetAttributes( + semconv.MessagingMessageIDKey.Int64(int64(m.TopicPartition.Offset)), + semconv.MessagingKafkaDestinationPartitionKey.Int64(int64(m.TopicPartition.Partition)), + ) + + if m.TopicPartition.Error != nil { + w.logger.Debugw(ctx, "Delivery failed", "error", m.TopicPartition.Error) + return Response{}, errors.Wrap(m.TopicPartition.Error, "failed to produce kafka message") + } + return Response{Partition: m.TopicPartition.Partition, Offset: int64(m.TopicPartition.Offset)}, nil +} + +func (w *KWriter) lifecyclePostAck(ctx context.Context, begin time.Time) { + if w.lifecycle.PostAck != nil { + lcMeta := LifecyclePostAckMeta{ + Topic: w.topicConfig.Topic, + ProduceTime: begin, + } + + if err := w.lifecycle.PostAck(ctx, lcMeta); err != nil { + w.logger.Warnw(ctx, "Lifecycle post-ack failed", "error", err, "meta", lcMeta) + } + } +} + +func (w *KWriter) startSpan(ctx context.Context, msg *kafka.Message) spanWrapper { + if msg == nil || w.tracer == nil { + return spanWrapper{} + } + topic := "" + if msg.TopicPartition.Topic != nil { + topic = *msg.TopicPartition.Topic + } + + opts := []trace.SpanStartOption{ + trace.WithAttributes( + semconv.MessagingOperationPublish, + semconv.MessagingDestinationName(topic), + semconv.MessagingKafkaMessageKey(string(msg.Key)), + semconv.MessagingKafkaMessageOffset(int(msg.TopicPartition.Offset)), + semconv.MessagingKafkaDestinationPartition(int(msg.TopicPartition.Partition)), + ), + trace.WithSpanKind(trace.SpanKindProducer), + } + + operationName := "zkafka.write" + ctx, span := w.tracer.Start(ctx, operationName, opts...) + + // Inject the current span into the original message, so it can be used to + // propagate the span. + if w.p != nil { + carrier := &kMsgCarrier{msg: msg} + w.p.Inject(ctx, carrier) + } + return spanWrapper{span: span} +} + +func (w *KWriter) write(ctx context.Context, msg keyValuePair, opts ...WriteOption) (Response, error) { + if w.fmtter == nil { + return Response{}, errors.New("formatter is not supplied to produce kafka message") + } + value, err := w.fmtter.Marshall(msg.value) + if err != nil { + return Response{}, errors.Wrap(err, "failed to marshall producer message") + } + + return w.WriteRaw(ctx, msg.key, value, opts...) +} + +// Close terminates the writer gracefully and mark it as closed +func (w *KWriter) Close() { + w.mu.Lock() + defer w.mu.Unlock() + w.producer.Close() + w.isClosed = true +} + +// WriterOption is a function that modify the writer configurations +type WriterOption func(*KWriter) + +// WFormatterOption sets the formatter for this writer +func WFormatterOption(fmtter Formatter) WriterOption { + return func(w *KWriter) { + if fmtter != nil { + w.fmtter = fmtter + } + } +} + +// WriteOption is a function that modifies the kafka.Message to be transmitted +type WriteOption interface { + apply(s *kafka.Message) +} + +// WithHeaders allows for the specification of headers. Specified headers will override collisions. +func WithHeaders(headers map[string]string) WriteOption { return withHeaderOption{headers: headers} } + +type withHeaderOption struct { + headers map[string]string +} + +func (o withHeaderOption) apply(s *kafka.Message) { + updateHeaders := func(k, v string) { + header := kafka.Header{ + Key: k, + Value: []byte(v), + } + for i, h := range s.Headers { + if h.Key == k { + s.Headers[i] = header + return + } + } + s.Headers = append(s.Headers, header) + } + for k, v := range o.headers { + updateHeaders(k, v) + } +} diff --git a/writer_test.go b/writer_test.go new file mode 100644 index 0000000..2dda56b --- /dev/null +++ b/writer_test.go @@ -0,0 +1,599 @@ +package zkafka + +import ( + "context" + "errors" + "sync" + "testing" + + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + mock_confluent "github.com/zillow/zkafka/mocks/confluent" + "gitlab.zgtools.net/devex/archetypes/gomods/zfmt" + "go.opentelemetry.io/otel" + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/trace/noop" +) + +func TestWriter_Write(t *testing.T) { + recoverThenFail(t) + + ctrl := gomock.NewController(t) + p := mock_confluent.NewMockKafkaProducer(ctrl) + p.EXPECT().Produce(gomock.Any(), gomock.Any()).DoAndReturn(func(msg *kafka.Message, deliveryChan chan kafka.Event) error { + go func() { + deliveryChan <- &kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Partition: 1, + Offset: 1, + }, + } + }() + return nil + }).AnyTimes() + + type fields struct { + Mutex *sync.Mutex + Producer KafkaProducer + fmt zfmt.Formatter + } + type args struct { + ctx context.Context + value any + } + tests := []struct { + name string + fields fields + args args + want Response + wantErr bool + }{ + { + name: "formatter check at minimum", + fields: fields{}, + args: args{ctx: context.TODO(), value: "1"}, + want: Response{Partition: 0, Offset: 0}, + wantErr: true, + }, + { + name: "has formatter and producer", + fields: fields{ + fmt: &zfmt.StringFormatter{}, + Producer: p, + }, + args: args{ctx: context.TODO(), value: "1"}, + want: Response{Partition: 1, Offset: 1}, + }, + { + name: "has formatter, producer, incompatible message type", + fields: fields{ + fmt: &zfmt.StringFormatter{}, + Producer: p, + }, + args: args{ctx: context.TODO(), value: 5}, + want: Response{Partition: 1, Offset: 1}, + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + + w := &KWriter{ + producer: tt.fields.Producer, + fmtter: tt.fields.fmt, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + } + got, err := w.Write(tt.args.ctx, tt.args.value) + if tt.wantErr { + require.Error(t, err, "expected error for KWriter.Write()") + } else { + require.Equal(t, tt.want, got, "expected response for KWriter.Write()") + } + }) + } +} + +func TestWriter_WriteKey(t *testing.T) { + recoverThenFail(t) + ctrl := gomock.NewController(t) + + p := mock_confluent.NewMockKafkaProducer(ctrl) + p.EXPECT().Produce(gomock.Any(), gomock.Any()).DoAndReturn(func(msg *kafka.Message, deliveryChan chan kafka.Event) error { + go func() { + deliveryChan <- &kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Partition: 1, + Offset: 1, + }, + } + }() + return nil + }).AnyTimes() + + contextWithSpan, _ := otel.GetTracerProvider().Tracer("").Start(context.Background(), "sdf") + + type fields struct { + Mutex *sync.Mutex + Producer KafkaProducer + conf ProducerTopicConfig + fmt zfmt.Formatter + isClosed bool + } + type args struct { + ctx context.Context + key string + value any + } + tests := []struct { + name string + fields fields + args args + want Response + wantErr bool + }{ + { + name: "valid keyValPairs", + fields: fields{ + fmt: &zfmt.StringFormatter{}, + Producer: p, + }, + args: args{ctx: context.TODO(), key: "key1", value: "msg1"}, + want: Response{Partition: 1, Offset: 1}, + wantErr: false, + }, + { + name: "valid keyValPairs with partition spanning context", + fields: fields{ + fmt: &zfmt.StringFormatter{}, + Producer: p, + }, + args: args{ctx: contextWithSpan, key: "key1", value: "msg1"}, + want: Response{Partition: 1, Offset: 1}, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + defer recoverThenFail(t) + w := &KWriter{ + producer: tt.fields.Producer, + topicConfig: tt.fields.conf, + fmtter: tt.fields.fmt, + isClosed: tt.fields.isClosed, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + } + got, err := w.WriteKey(tt.args.ctx, tt.args.key, tt.args.value) + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + require.Equal(t, tt.want, got) + }) + } +} + +// TestWriter_WriteKeyReturnsImmediateError given a writer is used to try and produce a message +// when the producer returns an immediate error (as opposed) to on the channel +// then the error is bubbled up +// +// The producer doesn't typically return immediate errors (only if you're doing something obviously incorrect, librdkafka is too old for example). +// However, it was noticed, that there was a possible deadlock that occurred when the quick error was returned. This test assures that's +// no longer the behavior and the error is bubbled up +func TestWriter_WriteKeyReturnsImmediateError(t *testing.T) { + recoverThenFail(t) + ctrl := gomock.NewController(t) + + p := mock_confluent.NewMockKafkaProducer(ctrl) + p.EXPECT().Produce(gomock.Any(), gomock.Any()).DoAndReturn(func(msg *kafka.Message, deliveryChan chan kafka.Event) error { + return errors.New("not implemented error") + }).AnyTimes() + + defer recoverThenFail(t) + w := &KWriter{ + producer: p, + topicConfig: ProducerTopicConfig{}, + isClosed: false, + fmtter: &zfmt.JSONFormatter{}, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + } + _, err := w.WriteKey(context.Background(), "key", "val") + require.Error(t, err) +} + +func TestWriter_WritesMetrics(t *testing.T) { + recoverThenFail(t) + ctrl := gomock.NewController(t) + + p := mock_confluent.NewMockKafkaProducer(ctrl) + p.EXPECT().Produce(gomock.Any(), gomock.Any()).DoAndReturn(func(msg *kafka.Message, deliveryChan chan kafka.Event) error { + go func() { + deliveryChan <- &kafka.Message{} + }() + return nil + }).AnyTimes() + + th := testLifecycleHooks{} + hooks := LifecycleHooks{ + PreProcessing: func(ctx context.Context, meta LifecyclePreProcessingMeta) (context.Context, error) { + return th.PreProcessing(ctx, meta) + }, + PostProcessing: func(ctx context.Context, meta LifecyclePostProcessingMeta) error { + return th.PostProcessing(ctx, meta) + }, + PostAck: func(ctx context.Context, meta LifecyclePostAckMeta) error { + return th.PostAck(ctx, meta) + }, + } + + wr := &KWriter{ + producer: p, + topicConfig: ProducerTopicConfig{Topic: "orange"}, + lifecycle: hooks, + fmtter: &zfmt.StringFormatter{}, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + } + + _, err := wr.WriteKey(context.Background(), "apple", "mango") + require.NoError(t, err) + + // Pre- and Post-Processing are covered in read tests. We are testing writes here. + require.Len(t, th.preProMeta, 0) + require.Len(t, th.postProMeta, 0) + require.Len(t, th.postAckMeta, 1) + require.Equal(t, th.postAckMeta[0].Topic, "orange") +} + +func TestWriter_WriteSpecialCase(t *testing.T) { + recoverThenFail(t) + + ctrl := gomock.NewController(t) + + p1 := mock_confluent.NewMockKafkaProducer(ctrl) + p1.EXPECT().Produce(gomock.Any(), gomock.Any()).DoAndReturn(func(msg *kafka.Message, deliveryChan chan kafka.Event) error { + go func() { + deliveryChan <- &kafka.Message{ + TopicPartition: kafka.TopicPartition{ + Error: errors.New("an error"), + }, + } + }() + return errors.New("an error") + }).AnyTimes() + + type fields struct { + Mutex *sync.Mutex + Producer KafkaProducer + fmt zfmt.Formatter + } + type args struct { + ctx context.Context + value any + } + tests := []struct { + name string + fields fields + args args + want Response + wantErr bool + }{ + { + name: "partition message in the batch failed", + fields: fields{ + fmt: &zfmt.StringFormatter{}, + Producer: p1, + }, + args: args{ctx: context.TODO(), value: "mgs2"}, + want: Response{}, + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + w := &KWriter{ + producer: tt.fields.Producer, + fmtter: tt.fields.fmt, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + } + got, err := w.Write(tt.args.ctx, tt.args.value) + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + require.Equal(t, tt.want, got) + } + }) + } +} + +func TestWriter_PreWriteLifecycleHookCanAugmentHeaders(t *testing.T) { + recoverThenFail(t) + ctrl := gomock.NewController(t) + + p := mock_confluent.NewMockKafkaProducer(ctrl) + var capturedMsg *kafka.Message + p.EXPECT().Produce(gomock.Any(), gomock.Any()).DoAndReturn(func(msg *kafka.Message, deliveryChan chan kafka.Event) error { + go func() { + deliveryChan <- &kafka.Message{} + }() + capturedMsg = msg + return nil + }).AnyTimes() + + hooks := LifecycleHooks{ + PreWrite: func(ctx context.Context, meta LifecyclePreWriteMeta) (LifecyclePreWriteResp, error) { + return LifecyclePreWriteResp{ + Headers: map[string][]byte{ + "hello": []byte("world"), + }, + }, nil + }, + } + + wr := &KWriter{ + producer: p, + topicConfig: ProducerTopicConfig{Topic: "orange"}, + lifecycle: hooks, + fmtter: &zfmt.StringFormatter{}, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + } + + _, err := wr.WriteKey(context.Background(), "apple", "mango") + require.NoError(t, err) + + require.Contains(t, capturedMsg.Headers, kafka.Header{Key: "hello", Value: []byte("world")}) +} + +func TestWriter_WithHeadersWriteOptionCanAugmentHeaders(t *testing.T) { + recoverThenFail(t) + ctrl := gomock.NewController(t) + + p := mock_confluent.NewMockKafkaProducer(ctrl) + var capturedMsg *kafka.Message + p.EXPECT().Produce(gomock.Any(), gomock.Any()).DoAndReturn(func(msg *kafka.Message, deliveryChan chan kafka.Event) error { + go func() { + deliveryChan <- &kafka.Message{} + }() + capturedMsg = msg + return nil + }).AnyTimes() + + wr := &KWriter{ + producer: p, + topicConfig: ProducerTopicConfig{Topic: "orange"}, + fmtter: &zfmt.StringFormatter{}, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + } + + _, err := wr.WriteKey(context.Background(), "apple", "mango", WithHeaders(map[string]string{ + "hello": "world", + })) + require.NoError(t, err) + + require.Contains(t, capturedMsg.Headers, kafka.Header{Key: "hello", Value: []byte("world")}) +} + +func Test_WithHeadersUpdatesOnConflict(t *testing.T) { + recoverThenFail(t) + + opt := WithHeaders(map[string]string{ + "abc": "def", + }) + msg := &kafka.Message{ + Headers: []kafka.Header{ + { + Key: "abc", + Value: []byte("xxx"), + }, + }, + } + opt.apply(msg) + require.Len(t, msg.Headers, 1) + require.Equal(t, msg.Headers[0], kafka.Header{ + Key: "abc", + Value: []byte("def"), + }) +} + +func TestWriter_PreWriteLifecycleHookErrorDoesntHaltProcessing(t *testing.T) { + recoverThenFail(t) + ctrl := gomock.NewController(t) + + p := mock_confluent.NewMockKafkaProducer(ctrl) + p.EXPECT().Produce(gomock.Any(), gomock.Any()).DoAndReturn(func(msg *kafka.Message, deliveryChan chan kafka.Event) error { + go func() { + deliveryChan <- &kafka.Message{} + }() + return nil + }).AnyTimes() + + hooks := LifecycleHooks{ + PreWrite: func(ctx context.Context, meta LifecyclePreWriteMeta) (LifecyclePreWriteResp, error) { + return LifecyclePreWriteResp{}, errors.New("pre write hook") + }, + } + + wr := &KWriter{ + producer: p, + topicConfig: ProducerTopicConfig{Topic: "orange"}, + lifecycle: hooks, + fmtter: &zfmt.StringFormatter{}, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + } + + _, err := wr.WriteKey(context.Background(), "apple", "mango") + require.NoError(t, err) +} + +func TestWriter_Close(t *testing.T) { + recoverThenFail(t) + ctrl := gomock.NewController(t) + p1 := mock_confluent.NewMockKafkaProducer(ctrl) + p1.EXPECT().Close().AnyTimes() + p2 := mock_confluent.NewMockKafkaProducer(ctrl) + p2.EXPECT().Close().AnyTimes() + + type fields struct { + Mutex *sync.Mutex + Producer KafkaProducer + isClosed bool + } + tests := []struct { + name string + fields fields + wantErr bool + }{ + { + name: "basic closure", + fields: fields{ + Producer: p1, + }, + wantErr: false, + }, + { + name: "basic closure with fake close error", + fields: fields{ + Producer: p2, + }, + wantErr: true, + }, + } + // No parallel test since Close() mocks are sequential + for _, tt := range tests { + w := &KWriter{ + producer: tt.fields.Producer, + isClosed: tt.fields.isClosed, + logger: NoopLogger{}, + tracer: noop.TracerProvider{}.Tracer(""), + p: propagation.TraceContext{}, + } + w.Close() + require.True(t, w.isClosed, "KWriter.Close() should have been closed") + } +} + +func Test_newWriter(t *testing.T) { + type args struct { + conf Config + topicConfig ProducerTopicConfig + producerP confluentProducerProvider + } + tests := []struct { + name string + args args + wantErr bool + }{ + { + name: "custom formatter, no error. It is implied that user will supply formatter later", + args: args{ + topicConfig: ProducerTopicConfig{ + Formatter: zfmt.FormatterType("custom"), + }, + producerP: defaultConfluentProducerProvider{}.NewProducer, + }, + wantErr: false, + }, + { + name: "invalid formatter", + args: args{ + producerP: defaultConfluentProducerProvider{}.NewProducer, + topicConfig: ProducerTopicConfig{ + Formatter: zfmt.FormatterType("invalid_fmt"), + }, + }, + wantErr: true, + }, + { + name: "valid formatter but has error from confluent producer constructor", + args: args{ + producerP: mockConfluentProducerProvider{err: true}.NewProducer, + }, + wantErr: true, + }, + { + name: "minimum config with formatter", + args: args{ + producerP: defaultConfluentProducerProvider{}.NewProducer, + topicConfig: ProducerTopicConfig{ + Formatter: zfmt.StringFmt, + }, + }, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + recoverThenFail(t) + w, err := newWriter(tt.args.conf, tt.args.topicConfig, tt.args.producerP) + if tt.wantErr { + require.Error(t, err, "expected error for newWriter()") + } else { + require.NoError(t, err) + require.NotNil(t, w, "worker should be initialized") + } + }) + } +} + +func TestWriter_WithOptions(t *testing.T) { + recoverThenFail(t) + w := &KWriter{} + require.Nil(t, w.fmtter, "expected nil formatter") + + WFormatterOption(&zfmt.StringFormatter{})(w) + require.NotNil(t, w.fmtter, "expected non-nil formatter") +} + +func Test_writeAttributeCarrier_Set(t *testing.T) { + recoverThenFail(t) + km := kafka.Message{} + c := kMsgCarrier{ + msg: &km, + } + c.Set("hello", "world") + expected := kafka.Header{ + Key: "hello", + Value: []byte("world"), + } + assertEqual(t, km.Headers[0], expected) +} + +// testLifecycle ... +type testLifecycleHooks struct { + preProMeta []LifecyclePreProcessingMeta + postProMeta []LifecyclePostProcessingMeta + postAckMeta []LifecyclePostAckMeta +} + +func (l *testLifecycleHooks) PreProcessing(ctx context.Context, meta LifecyclePreProcessingMeta) (context.Context, error) { + l.preProMeta = append(l.preProMeta, meta) + return ctx, nil +} + +func (l *testLifecycleHooks) PostProcessing(ctx context.Context, meta LifecyclePostProcessingMeta) error { + l.postProMeta = append(l.postProMeta, meta) + return nil +} + +func (l *testLifecycleHooks) PostAck(ctx context.Context, meta LifecyclePostAckMeta) error { + l.postAckMeta = append(l.postAckMeta, meta) + return nil +}