diff --git a/.gitignore b/.gitignore index ae971837..e92006b4 100644 --- a/.gitignore +++ b/.gitignore @@ -1,3 +1,4 @@ tmp* *.*~ .tags* +vendor diff --git a/assignment.go b/assignment.go new file mode 100644 index 00000000..cf7a2651 --- /dev/null +++ b/assignment.go @@ -0,0 +1,13 @@ +package goka + +import ( + "fmt" +) + +// Assignment represents a partition:offset assignment for the current connection +type Assignment map[int32]int64 + +func (a *Assignment) string() string { + var am map[int32]int64 = *a + return fmt.Sprintf("Assignment %v", am) +} diff --git a/broker.go b/broker.go new file mode 100644 index 00000000..f8df67ce --- /dev/null +++ b/broker.go @@ -0,0 +1,11 @@ +package goka + +import "github.com/Shopify/sarama" + +// Broker is an interface for the sarama broker +type Broker interface { + Addr() string + Connected() (bool, error) + CreateTopics(request *sarama.CreateTopicsRequest) (*sarama.CreateTopicsResponse, error) + Open(conf *sarama.Config) error +} diff --git a/builders.go b/builders.go new file mode 100644 index 00000000..4908b8ea --- /dev/null +++ b/builders.go @@ -0,0 +1,88 @@ +package goka + +import ( + "hash" + + "github.com/Shopify/sarama" +) + +// ProducerBuilder create a Kafka producer. +type ProducerBuilder func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) + +// DefaultProducerBuilder creates a Kafka producer using the Sarama library. +func DefaultProducerBuilder(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { + config := globalConfig + config.ClientID = clientID + config.Producer.Partitioner = sarama.NewCustomHashPartitioner(hasher) + return NewProducer(brokers, &config) +} + +// ProducerBuilderWithConfig creates a Kafka consumer using the Sarama library. +func ProducerBuilderWithConfig(config *sarama.Config) ProducerBuilder { + return func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { + config.ClientID = clientID + config.Producer.Partitioner = sarama.NewCustomHashPartitioner(hasher) + return NewProducer(brokers, config) + } +} + +// TopicManagerBuilder creates a TopicManager to check partition counts and +// create tables. +type TopicManagerBuilder func(brokers []string) (TopicManager, error) + +// DefaultTopicManagerBuilder creates TopicManager using the Sarama library. +func DefaultTopicManagerBuilder(brokers []string) (TopicManager, error) { + config := globalConfig + config.ClientID = "goka-topic-manager" + return NewTopicManager(brokers, &config, NewTopicManagerConfig()) +} + +// TopicManagerBuilderWithConfig creates TopicManager using the Sarama library. +func TopicManagerBuilderWithConfig(config *sarama.Config, tmConfig *TopicManagerConfig) TopicManagerBuilder { + return func(brokers []string) (TopicManager, error) { + return NewTopicManager(brokers, config, tmConfig) + } +} + +// TopicManagerBuilderWithTopicManagerConfig creates TopicManager using the Sarama library. +func TopicManagerBuilderWithTopicManagerConfig(tmConfig *TopicManagerConfig) TopicManagerBuilder { + return func(brokers []string) (TopicManager, error) { + config := globalConfig + config.ClientID = "goka-topic-manager" + return NewTopicManager(brokers, &config, tmConfig) + } +} + +type ConsumerGroupBuilder func(brokers []string, group, clientID string) (sarama.ConsumerGroup, error) + +// DefaultConsumerGroupBuilder creates a Kafka consumer using the Sarama library. +func DefaultConsumerGroupBuilder(brokers []string, group, clientID string) (sarama.ConsumerGroup, error) { + config := globalConfig + config.ClientID = clientID + return sarama.NewConsumerGroup(brokers, group, &config) +} + +// ConsumerGroupBuilderWithConfig creates a sarama consumergroup using passed config +func ConsumerGroupBuilderWithConfig(config *sarama.Config) ConsumerGroupBuilder { + return func(brokers []string, group, clientID string) (sarama.ConsumerGroup, error) { + config.ClientID = clientID + return sarama.NewConsumerGroup(brokers, group, config) + } +} + +type SaramaConsumerBuilder func(brokers []string, clientID string) (sarama.Consumer, error) + +// DefaultSaramaConsumerBuilder creates a Kafka consumer using the Sarama library. +func DefaultSaramaConsumerBuilder(brokers []string, clientID string) (sarama.Consumer, error) { + config := globalConfig + config.ClientID = clientID + return sarama.NewConsumer(brokers, &config) +} + +// ConsumerBuilderWithConfig creates a sarama consumergroup using passed config +func SaramaConsumerBuilderWithConfig(config *sarama.Config) SaramaConsumerBuilder { + return func(brokers []string, clientID string) (sarama.Consumer, error) { + config.ClientID = clientID + return sarama.NewConsumer(brokers, config) + } +} diff --git a/codec/codec_test.go b/codec/codec_test.go deleted file mode 100644 index 1fae45a0..00000000 --- a/codec/codec_test.go +++ /dev/null @@ -1 +0,0 @@ -package codec diff --git a/config.go b/config.go new file mode 100644 index 00000000..1430838b --- /dev/null +++ b/config.go @@ -0,0 +1,57 @@ +package goka + +import ( + "time" + + "github.com/Shopify/sarama" +) + +var ( + globalConfig = *DefaultConfig() +) + +const ( + // size of sarama buffer for consumer and producer + defaultChannelBufferSize = 256 + + // time sarama-cluster assumes the processing of an event may take + defaultMaxProcessingTime = 1 * time.Second + + // producer flush configuration + defaultFlushFrequency = 100 * time.Millisecond + defaultFlushBytes = 64 * 1024 + defaultProducerMaxRetries = 10 +) + +// DefaultConfig creates a new config used by goka per default +// Use it to modify and pass to `goka.ReplaceGlobalConifg(...)` to modify +// goka's global config +func DefaultConfig() *sarama.Config { + config := sarama.NewConfig() + config.Version = sarama.V2_0_0_0 + + // consumer configuration + config.Consumer.Return.Errors = true + config.Consumer.MaxProcessingTime = defaultMaxProcessingTime + // this configures the initial offset for streams. Tables are always + // consumed from OffsetOldest. + config.Consumer.Offsets.Initial = sarama.OffsetNewest + // producer configuration + config.Producer.RequiredAcks = sarama.WaitForLocal + config.Producer.Compression = sarama.CompressionSnappy + config.Producer.Flush.Frequency = defaultFlushFrequency + config.Producer.Flush.Bytes = defaultFlushBytes + config.Producer.Return.Successes = true + config.Producer.Return.Errors = true + config.Producer.Retry.Max = defaultProducerMaxRetries + return config +} + +// ReplaceGlobalConfig registeres a standard config used during building if no +// other config is specified +func ReplaceGlobalConfig(config *sarama.Config) { + if config == nil { + panic("nil config registered as global config") + } + globalConfig = *config +} diff --git a/config_test.go b/config_test.go new file mode 100644 index 00000000..1c88a4f8 --- /dev/null +++ b/config_test.go @@ -0,0 +1,42 @@ +package goka + +import ( + "testing" + + "github.com/Shopify/sarama" + "github.com/lovoo/goka/internal/test" +) + +func TestConfig_DefaultConfig(t *testing.T) { + t.Run("equal", func(t *testing.T) { + cfg := DefaultConfig() + test.AssertTrue(t, cfg.Version == sarama.V2_0_0_0) + test.AssertTrue(t, cfg.Consumer.Return.Errors == true) + test.AssertTrue(t, cfg.Consumer.MaxProcessingTime == defaultMaxProcessingTime) + test.AssertTrue(t, cfg.Consumer.Offsets.Initial == sarama.OffsetNewest) + test.AssertTrue(t, cfg.Producer.RequiredAcks == sarama.WaitForLocal) + test.AssertTrue(t, cfg.Producer.Compression == sarama.CompressionSnappy) + test.AssertTrue(t, cfg.Producer.Flush.Frequency == defaultFlushFrequency) + test.AssertTrue(t, cfg.Producer.Flush.Bytes == defaultFlushBytes) + test.AssertTrue(t, cfg.Producer.Return.Successes == true) + test.AssertTrue(t, cfg.Producer.Return.Errors == true) + test.AssertTrue(t, cfg.Producer.Retry.Max == defaultProducerMaxRetries) + }) +} + +func TestConfig_ReplaceGlobalConfig(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + custom := DefaultConfig() + custom.Version = sarama.V0_8_2_0 + ReplaceGlobalConfig(custom) + test.AssertEqual(t, globalConfig.Version, custom.Version) + }) + t.Run("panic", func(t *testing.T) { + defer func() { + if r := recover(); r == nil { + t.Fatal("there was no panic") + } + }() + ReplaceGlobalConfig(nil) + }) +} diff --git a/context.go b/context.go index d8917488..fc38b206 100644 --- a/context.go +++ b/context.go @@ -7,11 +7,12 @@ import ( "sync" "time" - "github.com/lovoo/goka/kafka" + "github.com/Shopify/sarama" "github.com/lovoo/goka/multierr" - "github.com/lovoo/goka/storage" ) +type emitter func(topic string, key string, value []byte) *Promise + // Context provides access to the processor's table and emit capabilities to // arbitrary topics in kafka. // Upon arrival of a message from subscribed topics, the respective @@ -70,23 +71,29 @@ type Context interface { Context() context.Context } -type emitter func(topic string, key string, value []byte) *kafka.Promise - type cbContext struct { ctx context.Context graph *GroupGraph + // commit commits the message in the consumer session + commit func() + + emitter emitter + asyncFailer func(err error) + syncFailer func(err error) - commit func() - emitter emitter - failer func(err error) + // Headers as passed from sarama. Note that this field will be filled + // lazily after the first call to Headers + headers map[string][]byte - storage storage.Storage - pviews map[string]*partition - views map[string]*View + table *PartitionTable + // joins + pviews map[string]*PartitionTable + // lookup tables + views map[string]*View - pstats *PartitionStats + partProcStats *PartitionProcStats - msg *message + msg *sarama.ConsumerMessage done bool counters struct { emits int @@ -149,11 +156,7 @@ func (ctx *cbContext) emit(topic string, key string, value []byte) { } ctx.emitDone(err) }) - - s := ctx.pstats.Output[topic] - s.Count++ - s.Bytes += len(value) - ctx.pstats.Output[topic] = s + ctx.partProcStats.trackOutput(topic, len(value)) } func (ctx *cbContext) Delete() { @@ -164,7 +167,7 @@ func (ctx *cbContext) Delete() { // Value returns the value of the key in the group table. func (ctx *cbContext) Value() interface{} { - val, err := ctx.valueForKey(ctx.msg.Key) + val, err := ctx.valueForKey(ctx.Key()) if err != nil { ctx.Fail(err) } @@ -173,7 +176,7 @@ func (ctx *cbContext) Value() interface{} { // SetValue updates the value of the key in the group table. func (ctx *cbContext) SetValue(value interface{}) { - if err := ctx.setValueForKey(ctx.msg.Key, value); err != nil { + if err := ctx.setValueForKey(ctx.Key(), value); err != nil { ctx.Fail(err) } } @@ -184,7 +187,7 @@ func (ctx *cbContext) Timestamp() time.Time { } func (ctx *cbContext) Key() string { - return ctx.msg.Key + return string(ctx.msg.Key) } func (ctx *cbContext) Topic() Stream { @@ -200,7 +203,14 @@ func (ctx *cbContext) Partition() int32 { } func (ctx *cbContext) Headers() map[string][]byte { - return ctx.msg.Header + + if ctx.headers == nil { + ctx.headers = make(map[string][]byte) + for _, header := range ctx.msg.Headers { + ctx.headers[string(header.Key)] = header.Value + } + } + return ctx.headers } func (ctx *cbContext) Join(topic Table) interface{} { @@ -242,11 +252,11 @@ func (ctx *cbContext) Lookup(topic Table, key string) interface{} { // valueForKey returns the value of key in the processor state. func (ctx *cbContext) valueForKey(key string) (interface{}, error) { - if ctx.storage == nil { + if ctx.table == nil { return nil, fmt.Errorf("Cannot access state in stateless processor") } - data, err := ctx.storage.Get(key) + data, err := ctx.table.Get(key) if err != nil { return nil, fmt.Errorf("error reading value: %v", err) } else if data == nil { @@ -266,7 +276,7 @@ func (ctx *cbContext) deleteKey(key string) error { } ctx.counters.stores++ - if err := ctx.storage.Delete(key); err != nil { + if err := ctx.table.Delete(key); err != nil { return fmt.Errorf("error deleting key (%s) from storage: %v", key, err) } @@ -294,20 +304,22 @@ func (ctx *cbContext) setValueForKey(key string, value interface{}) error { } ctx.counters.stores++ - if err = ctx.storage.Set(key, encodedValue); err != nil { + if err = ctx.table.Set(key, encodedValue); err != nil { return fmt.Errorf("error storing value: %v", err) } table := ctx.graph.GroupTable().Topic() ctx.counters.emits++ - ctx.emitter(table, key, encodedValue).Then(func(err error) { + ctx.emitter(table, key, encodedValue).ThenWithMessage(func(msg *sarama.ProducerMessage, err error) { + if err == nil && msg != nil { + err = ctx.table.storeNewestOffset(msg.Offset) + } ctx.emitDone(err) }) - s := ctx.pstats.Output[table] - s.Count++ - s.Bytes += len(encodedValue) - ctx.pstats.Output[table] = s + // for a table write we're tracking both the diskwrites and the kafka output + ctx.partProcStats.trackOutput(table, len(encodedValue)) + ctx.table.trackMessageWrite(len(encodedValue)) return nil } @@ -334,6 +346,7 @@ func (ctx *cbContext) start() { // calls ctx.commit once all emits have successfully finished, or fails context // if some emit failed. +// this function must be called from a locked function. func (ctx *cbContext) tryCommit(err error) { if err != nil { _ = ctx.errors.Collect(err) @@ -346,18 +359,22 @@ func (ctx *cbContext) tryCommit(err error) { // commit if no errors, otherwise fail context if ctx.errors.HasErrors() { - ctx.failer(ctx.errors.NilOrError()) + ctx.asyncFailer(ctx.errors.NilOrError()) } else { ctx.commit() } - // no further callback will be called from this context + ctx.markDone() +} + +// markdone marks the context as done +func (ctx *cbContext) markDone() { ctx.wg.Done() } // Fail stops execution and shuts down the processor func (ctx *cbContext) Fail(err error) { - panic(err) + ctx.syncFailer(err) } func (ctx *cbContext) Context() context.Context { diff --git a/context_test.go b/context_test.go index f60a7028..6597af74 100644 --- a/context_test.go +++ b/context_test.go @@ -3,57 +3,58 @@ package goka import ( "errors" "fmt" - "regexp" "strings" "sync" "testing" "time" + "github.com/Shopify/sarama" + "github.com/golang/mock/gomock" + "github.com/lovoo/goka/codec" - "github.com/lovoo/goka/kafka" + "github.com/lovoo/goka/internal/test" "github.com/lovoo/goka/logger" - "github.com/lovoo/goka/mock" - - "github.com/facebookgo/ensure" - "github.com/golang/mock/gomock" ) func newEmitter(err error, done func(err error)) emitter { - return func(topic string, key string, value []byte) *kafka.Promise { - p := kafka.NewPromise() + return func(topic string, key string, value []byte) *Promise { + p := NewPromise() if done != nil { p.Then(done) } - return p.Finish(err) + return p.Finish(nil, err) } } func newEmitterW(wg *sync.WaitGroup, err error, done func(err error)) emitter { - return func(topic string, key string, value []byte) *kafka.Promise { + return func(topic string, key string, value []byte) *Promise { wg.Add(1) - p := kafka.NewPromise() + p := NewPromise() if done != nil { p.Then(done) } - return p.Finish(err) + return p.Finish(nil, err) } } func TestContext_Emit(t *testing.T) { - ack := 0 - emitted := 0 + var ( + ack = 0 + emitted = 0 + group Group = "some-group" + ) ctx := &cbContext{ - graph: DefineGroup(group), - commit: func() { ack++ }, - wg: &sync.WaitGroup{}, - pstats: newPartitionStats(), + graph: DefineGroup(group), + commit: func() { ack++ }, + wg: &sync.WaitGroup{}, + partProcStats: newPartitionProcStats(nil, []string{"emit-topic"}), } // after that the message is processed ctx.emitter = newEmitter(nil, func(err error) { emitted++ - ensure.Nil(t, err) + test.AssertNil(t, err) }) ctx.start() @@ -65,41 +66,46 @@ func TestContext_Emit(t *testing.T) { ctx.wg.Wait() // check everything is done - ensure.DeepEqual(t, emitted, 1) - ensure.DeepEqual(t, ack, 1) + test.AssertEqual(t, emitted, 1) + test.AssertEqual(t, ack, 1) } func TestContext_Timestamp(t *testing.T) { ts := time.Now() ctx := &cbContext{ - msg: &message{ + msg: &sarama.ConsumerMessage{ Timestamp: ts, }, } - ensure.DeepEqual(t, ctx.Timestamp(), ts) + test.AssertEqual(t, ctx.Timestamp(), ts) } func TestContext_EmitError(t *testing.T) { - ack := 0 - emitted := 0 - errToEmit := errors.New("some error") + var ( + ack = 0 + emitted = 0 + errToEmit = errors.New("some error") + group Group = "some-group" + ) + + failer := func(err error) { + test.AssertTrue(t, strings.Contains(err.Error(), errToEmit.Error())) + } // test error case ctx := &cbContext{ - graph: DefineGroup(group, Persist(new(codec.String))), - commit: func() { ack++ }, - wg: &sync.WaitGroup{}, - pstats: newPartitionStats(), - failer: func(err error) { - ensure.StringContains(t, err.Error(), errToEmit.Error()) - }, + graph: DefineGroup(group, Persist(new(codec.String))), + wg: &sync.WaitGroup{}, + partProcStats: newPartitionProcStats(nil, []string{"emit-topic"}), + syncFailer: failer, + asyncFailer: failer, } ctx.emitter = newEmitter(errToEmit, func(err error) { emitted++ - ensure.NotNil(t, err) - ensure.DeepEqual(t, err, errToEmit) + test.AssertNotNil(t, err) + test.AssertEqual(t, err, errToEmit) }) ctx.start() @@ -111,49 +117,49 @@ func TestContext_EmitError(t *testing.T) { ctx.wg.Wait() // check everything is done - ensure.DeepEqual(t, emitted, 1) + test.AssertEqual(t, emitted, 1) // nothing should be committed here - ensure.DeepEqual(t, ack, 0) + test.AssertEqual(t, ack, 0) } func TestContext_EmitToStateTopic(t *testing.T) { - ctx := &cbContext{graph: DefineGroup(group, Persist(c), Loop(c, cb))} + var ( + group Group = "some-group" + ) + + ctx := &cbContext{ + graph: DefineGroup(group, Persist(c), Loop(c, cb)), + syncFailer: func(err error) { panic(err) }, + } func() { - defer ensure.PanicDeepEqual(t, errors.New("cannot emit to table topic (use SetValue instead)")) + defer test.PanicAssertEqual(t, errors.New("cannot emit to table topic (use SetValue instead)")) ctx.Emit(Stream(tableName(group)), "key", []byte("value")) }() func() { - defer ensure.PanicDeepEqual(t, errors.New("cannot emit to loop topic (use Loopback instead)")) + defer test.PanicAssertEqual(t, errors.New("cannot emit to loop topic (use Loopback instead)")) ctx.Emit(Stream(loopName(group)), "key", []byte("value")) }() func() { - defer ensure.PanicDeepEqual(t, errors.New("cannot emit to empty topic")) + defer test.PanicAssertEqual(t, errors.New("cannot emit to empty topic")) ctx.Emit("", "key", []byte("value")) }() } -func PanicStringContains(t *testing.T, s string) { - if r := recover(); r != nil { - err := r.(error) - ensure.StringContains(t, err.Error(), s) - } else { - // there was no panic - t.Errorf("panic expected") - t.FailNow() - } -} - func TestContext_GetSetStateless(t *testing.T) { // ctx stateless since no storage passed - ctx := &cbContext{graph: DefineGroup("group"), msg: new(message)} + ctx := &cbContext{ + graph: DefineGroup("group"), + msg: new(sarama.ConsumerMessage), + syncFailer: func(err error) { panic(err) }, + } func() { - defer PanicStringContains(t, "stateless") + defer test.PanicAssertStringContains(t, "stateless") _ = ctx.Value() }() func() { - defer PanicStringContains(t, "stateless") + defer test.PanicAssertStringContains(t, "stateless") ctx.SetValue("whatever") }() } @@ -161,33 +167,41 @@ func TestContext_GetSetStateless(t *testing.T) { func TestContext_Delete(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - storage := mock.NewMockStorage(ctrl) - offset := int64(123) - ack := 0 - key := "key" + var ( + offset = int64(123) + key = "key" + ack = 0 + group Group = "some-group" + st = NewMockStorage(ctrl) + pt = &PartitionTable{ + st: &storageProxy{ + Storage: st, + }, + stats: newTableStats(), + } + ) + + st.EXPECT().Delete(key).Return(nil) ctx := &cbContext{ - graph: DefineGroup(group, Persist(new(codec.String))), - storage: storage, - wg: new(sync.WaitGroup), - commit: func() { ack++ }, - msg: &message{Offset: offset}, + graph: DefineGroup(group, Persist(new(codec.String))), + wg: new(sync.WaitGroup), + commit: func() { ack++ }, + msg: &sarama.ConsumerMessage{Offset: offset}, + table: pt, } - gomock.InOrder( - storage.EXPECT().Delete(key), - ) ctx.emitter = newEmitter(nil, nil) ctx.start() err := ctx.deleteKey(key) - ensure.Nil(t, err) + test.AssertNil(t, err) ctx.finish(nil) ctx.wg.Wait() - ensure.DeepEqual(t, ctx.counters, struct { + test.AssertEqual(t, ctx.counters, struct { emits int dones int stores int @@ -198,81 +212,100 @@ func TestContext_DeleteStateless(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - offset := int64(123) - key := "key" + var ( + offset = int64(123) + key = "key" + group Group = "some-group" + ) ctx := &cbContext{ graph: DefineGroup(group), wg: new(sync.WaitGroup), - msg: &message{Offset: offset}, + msg: &sarama.ConsumerMessage{Offset: offset}, } ctx.emitter = newEmitter(nil, nil) err := ctx.deleteKey(key) - ensure.Err(t, err, regexp.MustCompile("^Cannot access state in stateless processor$")) + test.AssertTrue(t, strings.Contains(err.Error(), "Cannot access state in stateless processor")) } func TestContext_DeleteStorageError(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - storage := mock.NewMockStorage(ctrl) - offset := int64(123) - key := "key" + var ( + offset = int64(123) + key = "key" + group Group = "some-group" + st = NewMockStorage(ctrl) + pt = &PartitionTable{ + st: &storageProxy{ + Storage: st, + }, + stats: newTableStats(), + } + retErr = errors.New("storage error") + ) + + st.EXPECT().Delete(key).Return(retErr) ctx := &cbContext{ - graph: DefineGroup(group, Persist(new(codec.String))), - storage: storage, - wg: new(sync.WaitGroup), - msg: &message{Offset: offset}, + graph: DefineGroup(group, Persist(new(codec.String))), + wg: new(sync.WaitGroup), + msg: &sarama.ConsumerMessage{Offset: offset}, + table: pt, } - gomock.InOrder( - storage.EXPECT().Delete(key).Return(fmt.Errorf("storage error")), - ) ctx.emitter = newEmitter(nil, nil) err := ctx.deleteKey(key) - ensure.Err(t, err, regexp.MustCompile("^error deleting key \\(key\\) from storage: storage error$")) + test.AssertTrue(t, strings.Contains(err.Error(), "error deleting key (key) from storage: storage error")) } func TestContext_Set(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - storage := mock.NewMockStorage(ctrl) - offset := int64(123) - ack := 0 - key := "key" - value := "value" + var ( + offset = int64(123) + ack = 0 + key = "key" + value = "value" + group Group = "some-group" + st = NewMockStorage(ctrl) + pt = &PartitionTable{ + st: &storageProxy{ + Storage: st, + }, + stats: newTableStats(), + } + ) + st.EXPECT().Set(key, []byte(value)).Return(nil) ctx := &cbContext{ - graph: DefineGroup(group, Persist(new(codec.String))), - storage: storage, - wg: new(sync.WaitGroup), - pstats: newPartitionStats(), - commit: func() { ack++ }, - msg: &message{Offset: offset}, + graph: DefineGroup(group, Persist(new(codec.String))), + wg: new(sync.WaitGroup), + commit: func() { ack++ }, + partProcStats: newPartitionProcStats(nil, []string{string(GroupTable(group))}), + msg: &sarama.ConsumerMessage{Key: []byte(key), Offset: offset}, + table: pt, } - gomock.InOrder( - storage.EXPECT().Set(key, []byte(value)).Return(nil), - ) ctx.emitter = newEmitter(nil, nil) ctx.start() err := ctx.setValueForKey(key, value) - ensure.Nil(t, err) + test.AssertNil(t, err) ctx.finish(nil) ctx.wg.Wait() - ensure.DeepEqual(t, ctx.counters, struct { + test.AssertEqual(t, ctx.counters, struct { emits int dones int stores int }{1, 1, 1}) - ensure.DeepEqual(t, ack, 1) + test.AssertEqual(t, ack, 1) } func TestContext_GetSetStateful(t *testing.T) { @@ -280,38 +313,47 @@ func TestContext_GetSetStateful(t *testing.T) { defer ctrl.Finish() var ( - storage = mock.NewMockStorage(ctrl) - key = "key" - value = "value" - offset = int64(123) - wg = new(sync.WaitGroup) + group Group = "some-group" + key = "key" + value = "value" + offset = int64(123) + wg = new(sync.WaitGroup) + st = NewMockStorage(ctrl) + pt = &PartitionTable{ + st: &storageProxy{ + Storage: st, + }, + stats: newTableStats(), + } ) + + st.EXPECT().Get(key).Return(nil, nil) + st.EXPECT().Set(key, []byte(value)).Return(nil) + st.EXPECT().Get(key).Return([]byte(value), nil) + graph := DefineGroup(group, Persist(new(codec.String))) ctx := &cbContext{ - pstats: newPartitionStats(), - wg: wg, - graph: graph, - msg: &message{Key: key, Offset: offset}, - storage: storage, - emitter: func(tp string, k string, v []byte) *kafka.Promise { + table: pt, + wg: wg, + graph: graph, + partProcStats: newPartitionProcStats(nil, []string{string(GroupTable(group))}), + msg: &sarama.ConsumerMessage{Key: []byte(key), Offset: offset}, + emitter: func(tp string, k string, v []byte) *Promise { wg.Add(1) - ensure.DeepEqual(t, tp, graph.GroupTable().Topic()) - ensure.DeepEqual(t, string(k), key) - ensure.DeepEqual(t, string(v), value) - return kafka.NewPromise().Finish(nil) + test.AssertEqual(t, tp, graph.GroupTable().Topic()) + test.AssertEqual(t, string(k), key) + test.AssertEqual(t, string(v), value) + return NewPromise().Finish(nil, nil) }, } - storage.EXPECT().Get(key).Return(nil, nil) val := ctx.Value() - ensure.True(t, val == nil) + test.AssertTrue(t, val == nil) - storage.EXPECT().Set(key, []byte(value)).Return(nil) ctx.SetValue(value) - storage.EXPECT().Get(key).Return([]byte(value), nil) val = ctx.Value() - ensure.DeepEqual(t, val, value) + test.AssertEqual(t, val, value) } func TestContext_SetErrors(t *testing.T) { @@ -319,52 +361,67 @@ func TestContext_SetErrors(t *testing.T) { defer ctrl.Finish() var ( - storage = mock.NewMockStorage(ctrl) - key = "key" - value = "value" - offset int64 = 123 - wg = new(sync.WaitGroup) - failed error - _ = failed // make linter happy + group Group = "some-group" + key = "key" + value = "value" + offset int64 = 123 + wg = new(sync.WaitGroup) + st = NewMockStorage(ctrl) + pt = &PartitionTable{ + st: &storageProxy{ + Storage: st, + }, + stats: newTableStats(), + } + failed error + _ = failed // make linter happy ) + failer := func(err error) { failed = err } + ctx := &cbContext{ - pstats: newPartitionStats(), - wg: wg, - graph: DefineGroup(group, Persist(new(codec.String))), - msg: &message{Key: key, Offset: offset}, - storage: storage, - failer: func(err error) { failed = err }, + table: pt, + partProcStats: newPartitionProcStats(nil, nil), + wg: wg, + graph: DefineGroup(group, Persist(new(codec.String))), + msg: &sarama.ConsumerMessage{Key: []byte(key), Offset: offset}, + syncFailer: failer, + asyncFailer: failer, } err := ctx.setValueForKey(key, nil) - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "cannot set nil") + test.AssertNotNil(t, err) + test.AssertTrue(t, strings.Contains(err.Error(), "cannot set nil")) err = ctx.setValueForKey(key, 123) // cannot encode 123 as string - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "error encoding") + test.AssertNotNil(t, err) + test.AssertTrue(t, strings.Contains(err.Error(), "error encoding")) + + st.EXPECT().Set(key, []byte(value)).Return(errors.New("some-error")) - storage.EXPECT().Set(key, []byte(value)).Return(fmt.Errorf("some error")) err = ctx.setValueForKey(key, value) - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "error storing") + test.AssertNotNil(t, err) + test.AssertTrue(t, strings.Contains(err.Error(), "some-error")) + // TODO(jb): check if still valid // finish with error - ctx.emitter = newEmitterW(wg, fmt.Errorf("some error X"), func(err error) { - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "error X") - }) - storage.EXPECT().Set(key, []byte(value)).Return(nil) - err = ctx.setValueForKey(key, value) - ensure.Nil(t, err) + // ctx.emitter = newEmitterW(wg, fmt.Errorf("some-error"), func(err error) { + // test.AssertNotNil(t, err) + // test.AssertTrue(t, strings.Contains(err.Error(), "some-error")) + // }) + // err = ctx.setValueForKey(key, value) + // test.AssertNil(t, err) } func TestContext_LoopbackNoLoop(t *testing.T) { // ctx has no loop set - ctx := &cbContext{graph: DefineGroup("group", Persist(c)), msg: new(message)} + ctx := &cbContext{ + graph: DefineGroup("group", Persist(c)), + msg: &sarama.ConsumerMessage{}, + syncFailer: func(err error) { panic(err) }, + } func() { - defer PanicStringContains(t, "loop") + defer test.PanicAssertStringContains(t, "loop") ctx.Loopback("some-key", "whatever") }() } @@ -382,20 +439,20 @@ func TestContext_Loopback(t *testing.T) { graph := DefineGroup("group", Persist(c), Loop(c, cb)) ctx := &cbContext{ - graph: graph, - msg: new(message), - pstats: newPartitionStats(), - emitter: func(tp string, k string, v []byte) *kafka.Promise { + graph: graph, + msg: &sarama.ConsumerMessage{}, + partProcStats: newPartitionProcStats([]string{"group-loop"}, []string{"group-loop"}), + emitter: func(tp string, k string, v []byte) *Promise { cnt++ - ensure.DeepEqual(t, tp, graph.LoopStream().Topic()) - ensure.DeepEqual(t, string(k), key) - ensure.DeepEqual(t, string(v), value) - return kafka.NewPromise() + test.AssertEqual(t, tp, graph.LoopStream().Topic()) + test.AssertEqual(t, string(k), key) + test.AssertEqual(t, string(v), value) + return NewPromise() }, } ctx.Loopback(key, value) - ensure.True(t, cnt == 1) + test.AssertTrue(t, cnt == 1) } func TestContext_Join(t *testing.T) { @@ -403,43 +460,46 @@ func TestContext_Join(t *testing.T) { defer ctrl.Finish() var ( - key = "key" - value = "value" - table Table = "table" - st = mock.NewMockStorage(ctrl) + key = "key" + value = "value" + table Table = "table" + errSome = errors.New("some-error") + st = NewMockStorage(ctrl) ) ctx := &cbContext{ graph: DefineGroup("group", Persist(c), Loop(c, cb), Join(table, c)), - msg: &message{Key: key}, - pviews: map[string]*partition{ - string(table): &partition{ + msg: &sarama.ConsumerMessage{Key: []byte(key)}, + pviews: map[string]*PartitionTable{ + string(table): &PartitionTable{ log: logger.Default(), st: &storageProxy{ Storage: st, }, + stats: newTableStats(), }, }, + syncFailer: func(err error) { panic(err) }, } st.EXPECT().Get(key).Return([]byte(value), nil) v := ctx.Join(table) - ensure.DeepEqual(t, v, value) + test.AssertEqual(t, v, value) func() { - defer PanicStringContains(t, errSome.Error()) + defer test.PanicAssertStringContains(t, errSome.Error()) st.EXPECT().Get(key).Return(nil, errSome) _ = ctx.Join(table) }() func() { - defer PanicStringContains(t, "not subs") + defer test.PanicAssertStringContains(t, "not subs") _ = ctx.Join("other-table") }() ctx.pviews = nil func() { - defer PanicStringContains(t, "not subs") + defer test.PanicAssertStringContains(t, "not subs") _ = ctx.Join(table) }() } @@ -449,65 +509,94 @@ func TestContext_Lookup(t *testing.T) { defer ctrl.Finish() var ( - key = "key" - value = "value" - table Table = "table" - st = mock.NewMockStorage(ctrl) + key = "key" + value = "value" + table Table = "table" + errSome = errors.New("some-error") + st = NewMockStorage(ctrl) ) ctx := &cbContext{ graph: DefineGroup("group", Persist(c), Loop(c, cb)), - msg: &message{Key: key}, + msg: &sarama.ConsumerMessage{Key: []byte(key)}, views: map[string]*View{ string(table): &View{ opts: &voptions{ tableCodec: c, hasher: DefaultHasher(), }, - partitions: []*partition{ - &partition{ + partitions: []*PartitionTable{ + &PartitionTable{ st: &storageProxy{ Storage: st, }, + stats: newTableStats(), }, }, }, }, + syncFailer: func(err error) { panic(err) }, } st.EXPECT().Get(key).Return([]byte(value), nil) v := ctx.Lookup(table, key) - ensure.DeepEqual(t, v, value) + test.AssertEqual(t, v, value) func() { - defer PanicStringContains(t, errSome.Error()) + defer test.PanicAssertStringContains(t, errSome.Error()) st.EXPECT().Get(key).Return(nil, errSome) _ = ctx.Lookup(table, key) }() func() { - defer PanicStringContains(t, "not subs") + defer test.PanicAssertStringContains(t, "not subs") _ = ctx.Lookup("other-table", key) }() ctx.views = nil func() { - defer PanicStringContains(t, "not subs") + defer test.PanicAssertStringContains(t, "not subs") _ = ctx.Lookup(table, key) }() } +func TestContext_Headers(t *testing.T) { + + // context without headers will return empty map + ctx := &cbContext{ + msg: &sarama.ConsumerMessage{Key: []byte("key")}, + } + headers := ctx.Headers() + test.AssertNotNil(t, headers) + test.AssertEqual(t, len(headers), 0) + + ctx = &cbContext{ + msg: &sarama.ConsumerMessage{Key: []byte("key"), Headers: []*sarama.RecordHeader{ + &sarama.RecordHeader{ + Key: []byte("key"), + Value: []byte("value"), + }, + }}, + } + headers = ctx.Headers() + test.AssertEqual(t, headers["key"], []byte("value")) +} + func TestContext_Fail(t *testing.T) { - ctx := new(cbContext) + ctx := &cbContext{ + syncFailer: func(err error) { + panic(fmt.Errorf("%#v", err)) + }, + } defer func() { err := recover() - ensure.NotNil(t, err) - ensure.True(t, strings.Contains(fmt.Sprintf("%v", err), "blubb")) + test.AssertNotNil(t, err) + test.AssertTrue(t, strings.Contains(fmt.Sprintf("%v", err), "blubb")) }() ctx.Fail(errors.New("blubb")) // this must not be executed. ctx.Fail should stop execution - ensure.True(t, false) + test.AssertTrue(t, false) } diff --git a/doc.go b/doc.go index 3a9f8360..cf9884ea 100644 --- a/doc.go +++ b/doc.go @@ -1,7 +1,7 @@ -//go:generate go-bindata -pkg templates -o web/templates/bindata.go web/templates/common/ web/templates/monitor/ web/templates/query/ web/templates/index -//go:generate mockgen -package mock -destination mock/storage.go github.com/lovoo/goka/storage Storage -//go:generate mockgen -package mock -destination mock/proxy.go -aux_files storage=storage/storage.go -source partition.go kafkaProxy -//go:generate mockgen -package mock -destination mock/kafka.go github.com/lovoo/goka/kafka Consumer,TopicManager,Producer +//go:generate go-bindata -pkg templates -o web/templates/bindata.go web/templates/common/... web/templates/monitor/... web/templates/query/... web/templates/index/... +//go:generate mockgen -self_package github.com/lovoo/goka -package goka -destination mockstorage.go github.com/lovoo/goka/storage Storage +//go:generate mockgen -self_package github.com/lovoo/goka -package goka -destination mocks.go github.com/lovoo/goka TopicManager,Producer,Broker +//go:generate mockgen -self_package github.com/lovoo/goka -package goka -destination mockssarama.go github.com/Shopify/sarama Client /* Package goka is a stateful stream processing library for Apache Kafka (version 0.9+) that eases diff --git a/emitter.go b/emitter.go index 4485b53f..2f9bbf40 100644 --- a/emitter.go +++ b/emitter.go @@ -1,27 +1,33 @@ package goka import ( + "errors" "fmt" "sync" +) - "github.com/lovoo/goka/kafka" +var ( + ErrEmitterAlreadyClosed error = errors.New("emitter already closed") ) // Emitter emits messages into a specific Kafka topic, first encoding the message with the given codec. type Emitter struct { codec Codec - producer kafka.Producer + producer Producer topic string - wg sync.WaitGroup + wg sync.WaitGroup + done chan struct{} } // NewEmitter creates a new emitter using passed brokers, topic, codec and possibly options. func NewEmitter(brokers []string, topic Stream, codec Codec, options ...EmitterOption) (*Emitter, error) { options = append( // default options comes first - []EmitterOption{}, + []EmitterOption{ + WithEmitterClientID(fmt.Sprintf("goka-emitter-%s", topic)), + }, // user-defined options (may overwrite default ones) options..., @@ -29,10 +35,7 @@ func NewEmitter(brokers []string, topic Stream, codec Codec, options ...EmitterO opts := new(eoptions) - err := opts.applyOptions(topic, codec, options...) - if err != nil { - return nil, fmt.Errorf(errApplyOptions, err) - } + opts.applyOptions(topic, codec, options...) prod, err := opts.builders.producer(brokers, opts.clientID, opts.hasher) if err != nil { @@ -43,11 +46,18 @@ func NewEmitter(brokers []string, topic Stream, codec Codec, options ...EmitterO codec: codec, producer: prod, topic: string(topic), + done: make(chan struct{}), }, nil } // Emit sends a message for passed key using the emitter's codec. -func (e *Emitter) Emit(key string, msg interface{}) (*kafka.Promise, error) { +func (e *Emitter) Emit(key string, msg interface{}) (*Promise, error) { + select { + case <-e.done: + return NewPromise().Finish(nil, ErrEmitterAlreadyClosed), nil + default: + } + var ( err error data []byte @@ -69,7 +79,7 @@ func (e *Emitter) Emit(key string, msg interface{}) (*kafka.Promise, error) { func (e *Emitter) EmitSync(key string, msg interface{}) error { var ( err error - promise *kafka.Promise + promise *Promise ) promise, err = e.Emit(key, msg) @@ -88,6 +98,7 @@ func (e *Emitter) EmitSync(key string, msg interface{}) error { // Finish waits until the emitter is finished producing all pending messages. func (e *Emitter) Finish() error { + close(e.done) e.wg.Wait() return e.producer.Close() } diff --git a/emitter_test.go b/emitter_test.go new file mode 100644 index 00000000..75464700 --- /dev/null +++ b/emitter_test.go @@ -0,0 +1,206 @@ +package goka + +import ( + "errors" + "hash" + "strconv" + "testing" + "time" + + "github.com/golang/mock/gomock" + "github.com/lovoo/goka/codec" + "github.com/lovoo/goka/internal/test" +) + +var ( + emitterTestClientID = "161" + emitterTestBrokers = []string{"0"} + emitterTestTopic = Stream("emitter-stream") + emitterIntCodec = new(codec.Int64) +) + +func createEmitter(t *testing.T, options ...EmitterOption) (*Emitter, *builderMock, *gomock.Controller) { + ctrl := NewMockController(t) + bm := newBuilderMock(ctrl) + emitter, _ := NewEmitter(emitterTestBrokers, emitterTestTopic, emitterIntCodec, append([]EmitterOption{ + WithEmitterClientID(emitterTestClientID), + WithEmitterTopicManagerBuilder(bm.getTopicManagerBuilder()), + WithEmitterProducerBuilder(bm.getProducerBuilder()), + WithEmitterHasher(func() hash.Hash32 { return newConstHasher(0) }), + }, options...)...) + return emitter, bm, ctrl +} + +func TestEmitter_NewEmitter(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + ctrl := NewMockController(t) + bm := newBuilderMock(ctrl) + emitter, err := NewEmitter(emitterTestBrokers, emitterTestTopic, emitterIntCodec, []EmitterOption{ + WithEmitterClientID(emitterTestClientID), + WithEmitterTopicManagerBuilder(bm.getTopicManagerBuilder()), + WithEmitterProducerBuilder(bm.getProducerBuilder()), + WithEmitterHasher(func() hash.Hash32 { return newConstHasher(0) }), + }...) + test.AssertNil(t, err) + test.AssertNotNil(t, emitter) + test.AssertTrue(t, emitter.codec == emitterIntCodec) + test.AssertEqual(t, emitter.producer, bm.producer) + test.AssertTrue(t, emitter.topic == string(emitterTestTopic)) + }) + t.Run("fail", func(t *testing.T) { + ctrl := NewMockController(t) + bm := newBuilderMock(ctrl) + defer ctrl.Finish() + emitter, err := NewEmitter(emitterTestBrokers, emitterTestTopic, emitterIntCodec, WithEmitterProducerBuilder(bm.getErrorProducerBuilder())) + test.AssertNotNil(t, err) + test.AssertNil(t, emitter) + }) +} + +func TestEmitter_Emit(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + emitter, bm, ctrl := createEmitter(t) + defer ctrl.Finish() + + var ( + key string = "some-key" + intVal int64 = 1312 + data []byte = []byte(strconv.FormatInt(intVal, 10)) + ) + + bm.producer.EXPECT().Emit(emitter.topic, key, data).Return(NewPromise().Finish(nil, nil)) + promise, err := emitter.Emit(key, intVal) + test.AssertNil(t, err) + test.AssertNotNil(t, promise) + }) + t.Run("fail_producer_emit", func(t *testing.T) { + emitter, bm, ctrl := createEmitter(t) + defer ctrl.Finish() + + var ( + key string = "some-key" + intVal int64 = 1312 + data []byte = []byte(strconv.FormatInt(intVal, 10)) + retErr error = errors.New("some-error") + ) + + bm.producer.EXPECT().Emit(emitter.topic, key, data).Return(NewPromise().Finish(nil, retErr)) + promise, err := emitter.Emit(key, intVal) + test.AssertNil(t, err) + test.AssertEqual(t, promise.err, retErr) + }) + t.Run("fail_closed", func(t *testing.T) { + emitter, bm, ctrl := createEmitter(t) + defer ctrl.Finish() + + var ( + key string = "some-key" + intVal int64 = 1312 + ) + + bm.producer.EXPECT().Close().Return(nil) + + emitter.Finish() + promise, err := emitter.Emit(key, intVal) + test.AssertNil(t, err) + test.AssertEqual(t, promise.err, ErrEmitterAlreadyClosed) + }) + t.Run("fail_encode", func(t *testing.T) { + emitter, _, _ := createEmitter(t) + + var ( + key string = "some-key" + intVal string = "1312" + ) + + _, err := emitter.Emit(key, intVal) + test.AssertNotNil(t, err) + }) +} + +func TestEmitter_EmitSync(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + emitter, bm, ctrl := createEmitter(t) + defer ctrl.Finish() + + var ( + key string = "some-key" + intVal int64 = 1312 + data []byte = []byte(strconv.FormatInt(intVal, 10)) + ) + + bm.producer.EXPECT().Emit(emitter.topic, key, data).Return(NewPromise().Finish(nil, nil)) + err := emitter.EmitSync(key, intVal) + test.AssertNil(t, err) + }) + t.Run("fail_producer_emit", func(t *testing.T) { + emitter, bm, ctrl := createEmitter(t) + defer ctrl.Finish() + + var ( + key string = "some-key" + intVal int64 = 1312 + data []byte = []byte(strconv.FormatInt(intVal, 10)) + retErr error = errors.New("some-error") + ) + + bm.producer.EXPECT().Emit(emitter.topic, key, data).Return(NewPromise().Finish(nil, retErr)) + err := emitter.EmitSync(key, intVal) + test.AssertEqual(t, err, retErr) + }) + t.Run("fail_closed", func(t *testing.T) { + emitter, bm, ctrl := createEmitter(t) + defer ctrl.Finish() + + var ( + key string = "some-key" + intVal int64 = 1312 + ) + + bm.producer.EXPECT().Close().Return(nil) + + emitter.Finish() + err := emitter.EmitSync(key, intVal) + test.AssertEqual(t, err, ErrEmitterAlreadyClosed) + }) + t.Run("fail_encode", func(t *testing.T) { + emitter, _, _ := createEmitter(t) + + var ( + key string = "some-key" + intVal string = "1312" + ) + + err := emitter.EmitSync(key, intVal) + test.AssertNotNil(t, err) + }) +} + +func TestEmitter_Finish(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + emitter, bm, ctrl := createEmitter(t) + defer ctrl.Finish() + + var ( + key string = "some-key" + intVal int64 = 1312 + data []byte = []byte(strconv.FormatInt(intVal, 10)) + msgCount int = 200 + ) + + bm.producer.EXPECT().Emit(emitter.topic, key, data).Return(NewPromise().Finish(nil, nil)).MaxTimes(msgCount) + bm.producer.EXPECT().Close().Return(nil) + + go func() { + for i := 0; i < msgCount; i++ { + _, err := emitter.Emit(key, intVal) + test.AssertNil(t, err) + // promise errors are not checked here since they are expected + } + }() + + time.Sleep(time.Nanosecond * 45) + err := emitter.Finish() + test.AssertNil(t, err) + }) +} diff --git a/examples/1-simplest/main.go b/examples/1-simplest/main.go index d48e54e8..4d565bb4 100644 --- a/examples/1-simplest/main.go +++ b/examples/1-simplest/main.go @@ -2,9 +2,12 @@ package main import ( "context" - "fmt" "log" + "os" + "os/signal" + "syscall" + "github.com/Shopify/sarama" "github.com/lovoo/goka" "github.com/lovoo/goka/codec" ) @@ -13,8 +16,18 @@ var ( brokers = []string{"localhost:9092"} topic goka.Stream = "example-stream" group goka.Group = "example-group" + + tmc *goka.TopicManagerConfig ) +func init() { + // This sets the default replication to 1. If you have more then one broker + // the default configuration can be used. + tmc = goka.NewTopicManagerConfig() + tmc.Table.Replication = 1 + tmc.Stream.Replication = 1 +} + // emits a single message and leave func runEmitter() { emitter, err := goka.NewEmitter(brokers, topic, new(codec.String)) @@ -26,7 +39,7 @@ func runEmitter() { if err != nil { log.Fatalf("error emitting message: %v", err) } - fmt.Println("message emitted") + log.Println("message emitted") } // process messages until ctrl-c is pressed @@ -54,16 +67,50 @@ func runProcessor() { goka.Persist(new(codec.Int64)), ) - p, err := goka.NewProcessor(brokers, g) + p, err := goka.NewProcessor(brokers, + g, + goka.WithTopicManagerBuilder(goka.TopicManagerBuilderWithTopicManagerConfig(tmc)), + goka.WithConsumerGroupBuilder(goka.DefaultConsumerGroupBuilder), + ) if err != nil { log.Fatalf("error creating processor: %v", err) } - if err = p.Run(context.Background()); err != nil { - log.Fatalf("error running processor: %v", err) + ctx, cancel := context.WithCancel(context.Background()) + done := make(chan struct{}) + go func() { + defer close(done) + if err = p.Run(ctx); err != nil { + log.Printf("error running processor: %v", err) + } + }() + + sigs := make(chan os.Signal) + go func() { + signal.Notify(sigs, syscall.SIGINT, syscall.SIGTERM, syscall.SIGKILL) + }() + + select { + case <-sigs: + case <-done: } + cancel() + <-done } func main() { + config := goka.DefaultConfig() + config.Consumer.Offsets.Initial = sarama.OffsetOldest + goka.ReplaceGlobalConfig(config) + + tm, err := goka.NewTopicManager(brokers, goka.DefaultConfig(), tmc) + if err != nil { + log.Fatalf("Error creating topic manager: %v", err) + } + err = tm.EnsureStreamExists(string(topic), 8) + if err != nil { + log.Printf("Error creating kafka topic %s: %v", topic, err) + } + runEmitter() // emits one message and stops runProcessor() // press ctrl-c to stop } diff --git a/examples/2-clicks/main.go b/examples/2-clicks/main.go index 451027c3..acf2e82b 100644 --- a/examples/2-clicks/main.go +++ b/examples/2-clicks/main.go @@ -87,7 +87,14 @@ func runProcessor() { goka.Input(topic, new(codec.String), process), goka.Persist(new(userCodec)), ) - p, err := goka.NewProcessor(brokers, g) + tmc := goka.NewTopicManagerConfig() + tmc.Table.Replication = 1 + tmc.Stream.Replication = 1 + p, err := goka.NewProcessor(brokers, + g, + goka.WithTopicManagerBuilder(goka.TopicManagerBuilderWithTopicManagerConfig(tmc)), + goka.WithConsumerGroupBuilder(goka.DefaultConsumerGroupBuilder), + ) if err != nil { panic(err) } diff --git a/examples/4-tests/example_test.go b/examples/4-tests/example_test.go index 2e6a8f5e..ff722087 100644 --- a/examples/4-tests/example_test.go +++ b/examples/4-tests/example_test.go @@ -5,9 +5,9 @@ import ( "fmt" "testing" - "github.com/facebookgo/ensure" "github.com/lovoo/goka" "github.com/lovoo/goka/codec" + "github.com/lovoo/goka/internal/test" "github.com/lovoo/goka/tester" ) @@ -15,7 +15,7 @@ import ( // One processor with only one input func Test_1Input(t *testing.T) { var ( - gkt = tester.New(t) + tt = tester.New(t) receivedMessage string ) @@ -25,17 +25,27 @@ func Test_1Input(t *testing.T) { receivedMessage = msg.(string) }), ), - goka.WithTester(gkt), + goka.WithTester(tt), ) - // start it - go proc.Run(context.Background()) + done := make(chan struct{}) + go func() { + defer close(done) + err := proc.Run(context.Background()) + if err != nil { + t.Fatalf("processor run failed with: %v", err) + } + }() // consume a message - gkt.Consume("input", "key", "some message") + tt.Consume("input", "key", "some message") // ensure the message was received - ensure.DeepEqual(t, receivedMessage, "some message") + test.AssertEqual(t, receivedMessage, "some message") + + // stop the processor and wait to finish + proc.Stop() + <-done } // Scenario (2) @@ -67,9 +77,55 @@ func Test_2InputOutput(t *testing.T) { // make sure received the message in the output key, value, valid := mt.Next() - ensure.True(t, valid) - ensure.DeepEqual(t, key, "key") - ensure.DeepEqual(t, value, "forwarded: some-message") + test.AssertTrue(t, valid) + test.AssertEqual(t, key, "key") + test.AssertEqual(t, value, "forwarded: some-message") +} + +func Test_SetTableValue(t *testing.T) { + var ( + gkt = tester.New(t) + ) + + // create a new processor, registering the tester + proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("group", + goka.Input("input", new(codec.Int64), func(ctx goka.Context, msg interface{}) { + ctx.SetValue(ctx.Value().(int64) + msg.(int64)) + }), + goka.Persist(new(codec.Int64)), + ), + goka.WithTester(gkt), + ) + + go proc.Run(context.Background()) + + gkt.SetTableValue("group-table", "value", int64(10)) + // start it + gkt.Consume("input", "value", int64(2)) + + // make sure it's correctly persisted in the state + value := gkt.TableValue("group-table", "value") + test.AssertEqual(t, value, int64(12)) +} + +func Test_JoinOutput(t *testing.T) { + + var ( + gkt = tester.New(t) + ) + + // create a new processor, registering the tester + proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("group", + goka.Input("input", new(codec.Int64), func(ctx goka.Context, msg interface{}) { + }), + goka.Output("output", new(codec.Int64)), + goka.Join("join", new(codec.Int64)), + ), + goka.WithTester(gkt), + ) + go proc.Run(context.Background()) + + gkt.Consume("input", "value", int64(2)) } // Scenario (3) @@ -97,7 +153,7 @@ func Test_3Persist(t *testing.T) { // make sure it's correctly persisted in the state value := gkt.TableValue("group-table", "key") - ensure.DeepEqual(t, value, "state: some-message") + test.AssertEqual(t, value, "state: some-message") } // Scenario (4) @@ -133,16 +189,17 @@ func Test_Subtest(t *testing.T) { // check it was emitted key, value, ok := mt.Next() - ensure.True(t, ok) - ensure.DeepEqual(t, key, "output-key") - ensure.DeepEqual(t, value, "forwarded: hello") + test.AssertTrue(t, ok) + test.AssertEqual(t, key, "output-key") + test.AssertEqual(t, value, "forwarded: hello") // we should be at the end - ensure.DeepEqual(t, mt.Hwm(), mt.NextOffset()) + test.AssertEqual(t, mt.Hwm(), int64(1)) + test.AssertEqual(t, mt.NextOffset(), int64(1)) // this is equivalent _, _, ok = mt.Next() - ensure.False(t, ok) + test.AssertFalse(t, ok) }) t.Run("test-2", func(t *testing.T) { // clear all values so we can start with an empty state @@ -153,7 +210,7 @@ func Test_Subtest(t *testing.T) { // do some state checks value := gkt.TableValue("group-table", "bob") - ensure.DeepEqual(t, value, "state: hello") + test.AssertEqual(t, value, "state: hello") }) } @@ -201,5 +258,5 @@ func Test_Chain(t *testing.T) { // the value should be persisted in the second processor's table value := gkt.TableValue("proc2-table", "bob") - ensure.DeepEqual(t, value, "persist: proc1-out: loop: hello world") + test.AssertEqual(t, value, "persist: proc1-out: loop: hello world") } diff --git a/examples/Makefile b/examples/Makefile index 32cee9b1..c641f2a8 100644 --- a/examples/Makefile +++ b/examples/Makefile @@ -1,45 +1,22 @@ -# Makefile to get Zookeeper and Kafka up and running - -ifdef DOCKER_HOST - DOCKER_IP=$(shell docker-machine ip) -else - DOCKER_IP=127.0.0.1 -endif - all: @echo targets: start restart stop -# delete all images and restart +# stop and start docker compose (zookeeper and kafka) restart: stop start @echo done -# start all images -start: run-zk run-kafka +# start docker compose (zookeeper and kafka) +start: compose-up @echo done -# stop all images (forces deletion of images) -stop: - @echo stopping Kafka and Zookeeper - @docker rm -f zookeeper kafka +# shutdown docker compose (zookeeper and kafka) +stop: compose-down + @echo done -run-zk: - @echo starting Zookeeper... - @docker run -d -t --name zookeeper \ - -p 2181:2181 \ - wurstmeister/zookeeper +compose-up: + @echo start docker compose + @docker-compose up -d -run-kafka: - @echo starting Kafka... - @docker run -d -t --name kafka \ - -e HOST_IP=kafka \ - -e KAFKA_ADVERTISED_HOST_NAME=${DOCKER_IP} \ - -e KAFKA_ADVERTISED_PORT=9092 \ - -e KAFKA_NUM_PARTITIONS=10 \ - -e KAFKA_DEFAULT_REPLICATION_FACTOR=1 \ - -e KAFKA_REPLICATION_FACTOR=1 \ - -p ${DOCKER_IP}:9092:9092 \ - -p ${DOCKER_IP}:9997:9997 \ - -e KAFKA_BROKER_ID=1 \ - -e ZK=zk -p 9092 \ - --link zookeeper:zk \ - wurstmeister/kafka:0.10.1.0 +compose-down: + @echo stop docker compose + @docker-compose down diff --git a/examples/create-kafka-commands.sh b/examples/create-kafka-commands.sh index a58c1dfa..f11be364 100644 --- a/examples/create-kafka-commands.sh +++ b/examples/create-kafka-commands.sh @@ -9,7 +9,7 @@ TARGET=$1 mkdir -p $TARGET # create Kafka scripts -SCRIPTS=$(docker run --rm -it --entrypoint /bin/bash wurstmeister/kafka -c "ls \$KAFKA_HOME/bin/*.sh") +SCRIPTS=$(docker run --rm -it --entrypoint /bin/bash confluentinc/cp-kafka:5.4.0 -c "compgen -c | grep -E '(zook*|kafka*)'") for SCRIPT in $SCRIPTS; do SCRIPT=$(echo $SCRIPT | tr -d '\r') FN=$TARGET/$(basename $SCRIPT) @@ -17,7 +17,7 @@ for SCRIPT in $SCRIPTS; do cat <<-EOF > $FN #!/bin/bash CMD="$SCRIPT \$@" - docker run --net=host --rm -it --entrypoint /bin/bash wurstmeister/kafka -c "\$CMD" + docker run --net=host --rm -it --entrypoint /bin/bash confluentinc/cp-kafka:5.4.0 -c "\$CMD" EOF chmod +x $FN done @@ -27,6 +27,6 @@ echo creating $TARGET/zkCli.sh cat <<-EOF > $TARGET/zkCli.sh #!/bin/bash CMD="bin/zkCli.sh \$@" - docker run --net=host --rm -it wurstmeister/zookeeper bash -c "\$CMD" + docker run --net=host --rm -it zookeeper:3.4.9 bash -c "\$CMD" EOF chmod +x $TARGET/zkCli.sh diff --git a/examples/docker-compose.yml b/examples/docker-compose.yml index 1c846758..de1b640a 100644 --- a/examples/docker-compose.yml +++ b/examples/docker-compose.yml @@ -1,7 +1,7 @@ version: '3' services: zoo1: - image: wurstmeister/zookeeper + image: zookeeper:3.4.9 hostname: zoo1 ports: - "2181:2181" @@ -9,48 +9,54 @@ services: ZOO_MY_ID: 1 ZOO_PORT: 2181 ZOO_SERVERS: server.1=zoo1:2888:3888 - kafka1: - image: wurstmeister/kafka:0.10.1.0 + image: confluentinc/cp-kafka:5.4.0 hostname: kafka1 ports: - "9092:9092" environment: - KAFKA_ADVERTISED_LISTENERS: "PLAINTEXT://kafka1:9092" + KAFKA_ADVERTISED_LISTENERS: LISTENER_DOCKER_INTERNAL://kafka1:19092,LISTENER_DOCKER_EXTERNAL://${DOCKER_HOST_IP:-127.0.0.1}:9092 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: LISTENER_DOCKER_INTERNAL:PLAINTEXT,LISTENER_DOCKER_EXTERNAL:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: LISTENER_DOCKER_INTERNAL KAFKA_ZOOKEEPER_CONNECT: "zoo1:2181" - KAFKA_DEFAULT_REPLICATION_FACTOR: ${REPLICATION_FACTOR} - KAFKA_NUM_PARTITIONS: ${NUM_PARTITIONS} - KAFKA_ADVERTISED_PORT: 9092 + KAFKA_AUTO_CREATE_TOPICS_ENABLE: "true" KAFKA_BROKER_ID: 1 + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_DEFAULT_REPLICATION_FACTOR: 2 + KAFKA_NUM_PARTITIONS: 6 depends_on: - zoo1 - kafka2: - image: wurstmeister/kafka:0.10.1.0 + image: confluentinc/cp-kafka:5.4.0 hostname: kafka2 ports: - "9093:9093" environment: - KAFKA_ADVERTISED_LISTENERS: "PLAINTEXT://kafka2:9093" + KAFKA_ADVERTISED_LISTENERS: LISTENER_DOCKER_INTERNAL://kafka2:19093,LISTENER_DOCKER_EXTERNAL://${DOCKER_HOST_IP:-127.0.0.1}:9093 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: LISTENER_DOCKER_INTERNAL:PLAINTEXT,LISTENER_DOCKER_EXTERNAL:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: LISTENER_DOCKER_INTERNAL KAFKA_ZOOKEEPER_CONNECT: "zoo1:2181" - KAFKA_DEFAULT_REPLICATION_FACTOR: ${REPLICATION_FACTOR} - KAFKA_NUM_PARTITIONS: ${NUM_PARTITIONS} - KAFKA_ADVERTISED_PORT: 9093 + KAFKA_AUTO_CREATE_TOPICS_ENABLE: "true" KAFKA_BROKER_ID: 2 + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_DEFAULT_REPLICATION_FACTOR: 2 + KAFKA_NUM_PARTITIONS: 6 depends_on: - zoo1 - kafka3: - image: wurstmeister/kafka:0.10.1.0 + image: confluentinc/cp-kafka:5.4.0 hostname: kafka3 ports: - "9094:9094" environment: - KAFKA_ADVERTISED_LISTENERS: "PLAINTEXT://kafka3:9094" + KAFKA_ADVERTISED_LISTENERS: LISTENER_DOCKER_INTERNAL://kafka3:19094,LISTENER_DOCKER_EXTERNAL://${DOCKER_HOST_IP:-127.0.0.1}:9094 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: LISTENER_DOCKER_INTERNAL:PLAINTEXT,LISTENER_DOCKER_EXTERNAL:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: LISTENER_DOCKER_INTERNAL KAFKA_ZOOKEEPER_CONNECT: "zoo1:2181" - KAFKA_DEFAULT_REPLICATION_FACTOR: ${REPLICATION_FACTOR} - KAFKA_NUM_PARTITIONS: ${NUM_PARTITIONS} - KAFKA_ADVERTISED_PORT: 9094 + KAFKA_AUTO_CREATE_TOPICS_ENABLE: "true" KAFKA_BROKER_ID: 3 + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_DEFAULT_REPLICATION_FACTOR: 2 + KAFKA_NUM_PARTITIONS: 6 depends_on: - - zoo1 \ No newline at end of file + - zoo1 diff --git a/examples/monitoring/main.go b/examples/monitoring/main.go index 14191b37..ab395ec8 100644 --- a/examples/monitoring/main.go +++ b/examples/monitoring/main.go @@ -4,12 +4,20 @@ import ( "context" "encoding/json" "fmt" + "log" "net/http" + "net/http/pprof" + "os" + "os/signal" + "runtime" + "syscall" "time" + "github.com/Shopify/sarama" "github.com/gorilla/mux" "github.com/lovoo/goka" "github.com/lovoo/goka/codec" + "github.com/lovoo/goka/multierr" "github.com/lovoo/goka/web/index" "github.com/lovoo/goka/web/monitor" "github.com/lovoo/goka/web/query" @@ -52,23 +60,31 @@ func (jc *userCodec) Decode(data []byte) (interface{}, error) { return &c, nil } -func runEmitter() { +func runEmitter(ctx context.Context) (rerr error) { emitter, err := goka.NewEmitter(brokers, topic, new(codec.String)) if err != nil { - panic(err) + rerr = err + return } - defer emitter.Finish() + defer func() { + rerr = emitter.Finish() + }() t := time.NewTicker(100 * time.Millisecond) defer t.Stop() var i int - for range t.C { - key := fmt.Sprintf("user-%d", i%50) - value := fmt.Sprintf("%s", time.Now()) - emitter.EmitSync(key, value) - i++ + for { + select { + case <-t.C: + key := fmt.Sprintf("user-%d", i%50) + value := fmt.Sprintf("%s", time.Now()) + emitter.EmitSync(key, value) + i++ + case <-ctx.Done(): + return + } } } @@ -84,7 +100,7 @@ func process(ctx goka.Context, msg interface{}) { ctx.SetValue(u) fmt.Printf("[proc] key: %s clicks: %d, msg: %v\n", ctx.Key(), u.Clicks, msg) } -func runStatelessProcessor(monitor *monitor.Server) { +func runStatelessProcessor(ctx context.Context, monitor *monitor.Server) error { g := goka.DefineGroup(group+"-stateless", goka.Input(topic, new(codec.String), @@ -94,21 +110,16 @@ func runStatelessProcessor(monitor *monitor.Server) { ) p, err := goka.NewProcessor(brokers, g) if err != nil { - panic(err) + return err } // attach the processor to the monitor monitor.AttachProcessor(p) - err = p.Run(context.Background()) - if err != nil { - panic(err) - } else { - fmt.Println("Processor stopped without errors") - } + return p.Run(ctx) } -func runJoinProcessor(monitor *monitor.Server) { +func runJoinProcessor(ctx context.Context, monitor *monitor.Server) error { g := goka.DefineGroup(group+"-join", goka.Input(topic, new(codec.String), @@ -123,25 +134,21 @@ func runJoinProcessor(monitor *monitor.Server) { u.Clicks++ ctx.SetValue(u) }), + goka.Lookup(goka.GroupTable(group), new(userCodec)), goka.Persist(new(userCodec)), ) p, err := goka.NewProcessor(brokers, g) if err != nil { - panic(err) + return err } // attach the processor to the monitor monitor.AttachProcessor(p) - err = p.Run(context.Background()) - if err != nil { - panic(err) - } else { - fmt.Println("Processor stopped without errors") - } + return p.Run(ctx) } -func runProcessor(monitor *monitor.Server, query *query.Server) { +func runProcessor(ctx context.Context, monitor *monitor.Server, query *query.Server) error { g := goka.DefineGroup(group, goka.Input(topic, new(codec.String), process), goka.Join(goka.GroupTable(goka.Group(string(group)+"-join")), new(codec.String)), @@ -149,54 +156,130 @@ func runProcessor(monitor *monitor.Server, query *query.Server) { ) p, err := goka.NewProcessor(brokers, g) if err != nil { - panic(err) + return err } // attach the processor to the monitor monitor.AttachProcessor(p) query.AttachSource("user-clicks", p.Get) - err = p.Run(context.Background()) + err = p.Run(ctx) if err != nil { - panic(err) - } else { - fmt.Println("Processor stopped without errors") + log.Printf("Error running processor: %v", err) } + return err } -func runView(root *mux.Router, monitor *monitor.Server) { +func runView(errg *multierr.ErrGroup, ctx context.Context, root *mux.Router, monitor *monitor.Server) error { view, err := goka.NewView(brokers, goka.GroupTable(group), new(userCodec), ) if err != nil { - panic(err) + return err } // attach the processor to the monitor monitor.AttachView(view) - go view.Run(context.Background()) + errg.Go(func() error { + return view.Run(ctx) + }) + + server := &http.Server{Addr: ":9095", Handler: root} - root.HandleFunc("/{key}", func(w http.ResponseWriter, r *http.Request) { - value, _ := view.Get(mux.Vars(r)["key"]) - data, _ := json.Marshal(value) - w.Write(data) + errg.Go(func() error { + + root.HandleFunc("/{key}", func(w http.ResponseWriter, r *http.Request) { + value, _ := view.Get(mux.Vars(r)["key"]) + data, _ := json.Marshal(value) + w.Write(data) + }) + fmt.Println("View opened at http://localhost:9095/") + err := server.ListenAndServe() + if err != http.ErrServerClosed { + return err + } + return nil + }) + errg.Go(func() error { + // wait for outer context to be finished + <-ctx.Done() + log.Printf("context cancelled, will shutdown server") + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + return server.Shutdown(ctx) }) - fmt.Println("View opened at http://localhost:9095/") - http.ListenAndServe(":9095", root) + return nil +} + +func pprofInit(root *mux.Router) { + runtime.SetBlockProfileRate(1) + runtime.SetMutexProfileFraction(1) + + sub := root.PathPrefix("/debug/pprof").Subrouter() + sub.HandleFunc("/", pprof.Index) + + sub.HandleFunc("/cmdline", pprof.Cmdline) + sub.Handle("/heap", pprof.Handler("heap")) + sub.Handle("/goroutine", pprof.Handler("goroutine")) + sub.Handle("/block", pprof.Handler("block")) + sub.Handle("/mutex", pprof.Handler("mutex")) + sub.HandleFunc("/profile", pprof.Profile) + sub.HandleFunc("/symbol", pprof.Symbol) + sub.HandleFunc("/trace", pprof.Trace) } func main() { + + cfg := goka.DefaultConfig() + cfg.Consumer.Offsets.Initial = sarama.OffsetOldest + cfg.Version = sarama.V2_4_0_0 + goka.ReplaceGlobalConfig(cfg) + root := mux.NewRouter() + pprofInit(root) monitorServer := monitor.NewServer("/monitor", root) queryServer := query.NewServer("/query", root) idxServer := index.NewServer("/", root) idxServer.AddComponent(monitorServer, "Monitor") idxServer.AddComponent(queryServer, "Query") - go runEmitter() - go runProcessor(monitorServer, queryServer) - go runStatelessProcessor(monitorServer) - go runJoinProcessor(monitorServer) - runView(root, monitorServer) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go func() { + waiter := make(chan os.Signal, 1) + signal.Notify(waiter, syscall.SIGINT, syscall.SIGTERM) + <-waiter + cancel() + }() + + errg, ctx := multierr.NewErrGroup(ctx) + errg.Go(func() error { + defer log.Printf("emitter done") + return runEmitter(ctx) + }) + errg.Go(func() error { + defer log.Printf("processor done") + return runProcessor(ctx, monitorServer, queryServer) + }) + errg.Go(func() error { + defer log.Printf("stateless processor done") + return runStatelessProcessor(ctx, monitorServer) + }) + errg.Go(func() error { + defer log.Printf("join procdessor done") + return runJoinProcessor(ctx, monitorServer) + }) + if err := runView(errg, ctx, root, monitorServer); err != nil { + log.Printf("Error running view, will shutdown: %v", err) + cancel() + } + + if err := errg.Wait().NilOrError(); err != nil { + log.Fatalf("Error running monitoring example: %v", err) + } else { + log.Printf("Example gracefully shutdown") + } } diff --git a/go.mod b/go.mod new file mode 100644 index 00000000..023d02db --- /dev/null +++ b/go.mod @@ -0,0 +1,13 @@ +module github.com/lovoo/goka + +go 1.13 + +require ( + github.com/Shopify/sarama v1.26.1 + github.com/golang/mock v1.4.0 + github.com/gorilla/mux v1.7.3 + github.com/syndtr/goleveldb v1.0.0 + golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e + gopkg.in/redis.v5 v5.2.9 + gopkg.in/yaml.v2 v2.2.8 +) diff --git a/go.sum b/go.sum new file mode 100644 index 00000000..7ab632e3 --- /dev/null +++ b/go.sum @@ -0,0 +1,113 @@ +github.com/Shopify/sarama v1.26.1/go.mod h1:NbSGBSSndYaIhRcBtY9V0U7AyH+x71bG668AuWys/yU= +github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= +github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= +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/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q= +github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= +github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= +github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= +github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= +github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= +github.com/frankban/quicktest v1.7.2 h1:2QxQoC1TS09S7fhCPsrvqYdvP1H5M1P1ih5ABm3BTYk= +github.com/frankban/quicktest v1.7.2/go.mod h1:jaStnuzAqU1AJdCO0l53JDCJrVDKcS03DbaAcR7Ks/o= +github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I= +github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/golang/mock v1.4.0 h1:Rd1kQnQu0Hq3qvJppYSG0HtP+f5LPPUiDswTLiEegLg= +github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/protobuf v1.2.0 h1:P3YflyNX/ehuJFLhxviNdFxQPkGK5cDcApsge1SqnvM= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db h1:woRePGFeVFfLKN/pOkfl+p/TAqKOfFu+7KPlMVpok/w= +github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= +github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/gorilla/mux v1.7.3 h1:gnP5JzjVOuiZD07fKKToCAOjS0yOpj/qPETTXCCS6hw= +github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= +github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= +github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= +github.com/klauspost/compress v1.9.8/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs= +github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs= +github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/gomega v1.4.3 h1:RE1xgDvH7imwFD45h+u2SgIfERHlS2yNG4DObb5BSKU= +github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= +github.com/pierrec/lz4 v2.4.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= +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/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563 h1:dY6ETXrvDG7Sa4vE8ZQG4yqWg6UnOcbqTAahkV813vQ= +github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/syndtr/goleveldb v1.0.0 h1:fBdIW9lB4Iz0n9khmH8w27SJ3QEJ7+IgjPEwGSZiFdE= +github.com/syndtr/goleveldb v1.0.0/go.mod h1:ZVVdQEZoIme9iO1Ch2Jdy24qqXrMMOU6lpPAyBWyWuQ= +github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= +github.com/xdg/stringprep v1.0.0/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3 h1:0GoQqolDA55aaLxZyTzK/Y2ePZzZTUrRacwib7cNsYQ= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d h1:+R4KGOnez64A81RvjARKc4UT5/tI9ujCIVX+P5KiHuI= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262 h1:qsl9y/CJx34tuA7QCPNp86JNJe4spst6Ff8MjvPUdPg= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= +gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= +gopkg.in/jcmturner/aescts.v1 v1.0.1 h1:cVVZBK2b1zY26haWB4vbBiZrfFQnfbTVrE3xZq6hrEw= +gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo= +gopkg.in/jcmturner/dnsutils.v1 v1.0.1 h1:cIuC1OLRGZrld+16ZJvvZxVJeKPsvd5eUIvxfoN5hSM= +gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q= +gopkg.in/jcmturner/goidentity.v3 v3.0.0 h1:1duIyWiTaYvVx3YX2CYtpJbUFd7/UuPYCfgXtQ3VTbI= +gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4= +gopkg.in/jcmturner/gokrb5.v7 v7.5.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM= +gopkg.in/jcmturner/rpc.v1 v1.1.0 h1:QHIUxTX1ISuAv9dD2wJ9HWQVuWDX/Zc0PfeC2tjc4rU= +gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= +gopkg.in/redis.v5 v5.2.9 h1:MNZYOLPomQzZMfpN3ZtD1uyJ2IDonTTlxYiV/pEApiw= +gopkg.in/redis.v5 v5.2.9/go.mod h1:6gtv0/+A4iM08kdRfocWYB3bLX2tebpNtfKlFT6H4mY= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= +gopkg.in/yaml.v2 v2.2.1 h1:mUhvW9EsL+naU5Q3cakzfE91YhliOondGd6ZrsDBHQE= +gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10= +gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/graph_test.go b/graph_test.go index 56474d51..2b88c09a 100644 --- a/graph_test.go +++ b/graph_test.go @@ -5,21 +5,24 @@ import ( "strings" "testing" - "github.com/facebookgo/ensure" "github.com/lovoo/goka/codec" + "github.com/lovoo/goka/internal/test" ) -var c = new(codec.String) +var ( + c = new(codec.String) + cb = func(ctx Context, msg interface{}) {} +) func TestGroupGraph_Validate(t *testing.T) { g := DefineGroup("group") err := g.Validate() - ensure.StringContains(t, err.Error(), "no input") + test.AssertStringContains(t, err.Error(), "no input") g = DefineGroup("group", Input("input-topic", c, cb)) err = g.Validate() - ensure.Nil(t, err) + test.AssertNil(t, err) g = DefineGroup("group", Input("input-topic", c, cb), @@ -27,7 +30,7 @@ func TestGroupGraph_Validate(t *testing.T) { Loop(c, cb), ) err = g.Validate() - ensure.StringContains(t, err.Error(), "more than one loop") + test.AssertStringContains(t, err.Error(), "more than one loop") g = DefineGroup("group", Input("input-topic", c, cb), @@ -35,42 +38,42 @@ func TestGroupGraph_Validate(t *testing.T) { Persist(c), ) err = g.Validate() - ensure.StringContains(t, err.Error(), "more than one group table") + test.AssertStringContains(t, err.Error(), "more than one group table") g = DefineGroup("group", Input(Stream(tableName("group")), c, cb), Persist(c), ) err = g.Validate() - ensure.StringContains(t, err.Error(), "group table") + test.AssertStringContains(t, err.Error(), "group table") g = DefineGroup("group", Input(Stream(loopName("group")), c, cb), Loop(c, cb), ) err = g.Validate() - ensure.StringContains(t, err.Error(), "loop stream") + test.AssertStringContains(t, err.Error(), "loop stream") g = DefineGroup("group", Input("input-topic", c, cb), Join(Table(loopName("group")), c), ) err = g.Validate() - ensure.StringContains(t, err.Error(), "loop stream") + test.AssertStringContains(t, err.Error(), "loop stream") g = DefineGroup("group", Input("input-topic", c, cb), Output(Stream(loopName("group")), c), ) err = g.Validate() - ensure.StringContains(t, err.Error(), "loop stream") + test.AssertStringContains(t, err.Error(), "loop stream") g = DefineGroup("group", Input("input-topic", c, cb), Lookup(Table(loopName("group")), c), ) err = g.Validate() - ensure.StringContains(t, err.Error(), "loop stream") + test.AssertStringContains(t, err.Error(), "loop stream") } @@ -82,7 +85,7 @@ func TestGroupGraph_codec(t *testing.T) { for _, topic := range []string{"input-topic", "input-topic2", "input-topic3"} { codec := g.codec(topic) - ensure.DeepEqual(t, codec, c) + test.AssertEqual(t, codec, c) } } @@ -95,7 +98,7 @@ func TestGroupGraph_callback(t *testing.T) { for _, topic := range []string{"input-topic", "input-topic2", "input-topic3"} { callback := g.callback(topic) - ensure.True(t, reflect.ValueOf(callback).Pointer() == reflect.ValueOf(cb).Pointer()) + test.AssertTrue(t, reflect.ValueOf(callback).Pointer() == reflect.ValueOf(cb).Pointer()) } } @@ -108,10 +111,10 @@ func TestGroupGraph_getters(t *testing.T) { Output("t5", c), Inputs(Streams{"t6", "t7"}, c, cb), ) - ensure.True(t, g.Group() == "group") - ensure.True(t, len(g.InputStreams()) == 4) - ensure.True(t, len(g.OutputStreams()) == 3) - ensure.True(t, g.LoopStream() == nil) + test.AssertTrue(t, g.Group() == "group") + test.AssertTrue(t, len(g.InputStreams()) == 4) + test.AssertTrue(t, len(g.OutputStreams()) == 3) + test.AssertTrue(t, g.LoopStream() == nil) g = DefineGroup("group", Input("t1", c, cb), @@ -121,10 +124,10 @@ func TestGroupGraph_getters(t *testing.T) { Output("t5", c), Loop(c, cb), ) - ensure.True(t, len(g.InputStreams()) == 2) - ensure.True(t, len(g.OutputStreams()) == 3) - ensure.True(t, g.GroupTable() == nil) - ensure.DeepEqual(t, g.LoopStream().Topic(), loopName("group")) + test.AssertTrue(t, len(g.InputStreams()) == 2) + test.AssertTrue(t, len(g.OutputStreams()) == 3) + test.AssertTrue(t, g.GroupTable() == nil) + test.AssertEqual(t, g.LoopStream().Topic(), loopName("group")) g = DefineGroup("group", Input("t1", c, cb), @@ -141,16 +144,16 @@ func TestGroupGraph_getters(t *testing.T) { Lookup("b2", c), Persist(c), ) - ensure.True(t, len(g.InputStreams()) == 2) - ensure.True(t, len(g.OutputStreams()) == 3) - ensure.True(t, len(g.JointTables()) == 4) - ensure.True(t, len(g.LookupTables()) == 2) - ensure.DeepEqual(t, g.GroupTable().Topic(), tableName("group")) + test.AssertTrue(t, len(g.InputStreams()) == 2) + test.AssertTrue(t, len(g.OutputStreams()) == 3) + test.AssertTrue(t, len(g.JointTables()) == 4) + test.AssertTrue(t, len(g.LookupTables()) == 2) + test.AssertEqual(t, g.GroupTable().Topic(), tableName("group")) } func TestGroupGraph_Inputs(t *testing.T) { topics := Inputs(Streams{"a", "b", "c"}, c, cb) - ensure.DeepEqual(t, topics.Topic(), "a,b,c") - ensure.True(t, strings.Contains(topics.String(), "a,b,c/*codec.String")) + test.AssertEqual(t, topics.Topic(), "a,b,c") + test.AssertTrue(t, strings.Contains(topics.String(), "a,b,c/*codec.String")) } diff --git a/integrationtest/processor_stuck.go b/integrationtest/processor_stuck.go index f37c477a..80386e65 100644 --- a/integrationtest/processor_stuck.go +++ b/integrationtest/processor_stuck.go @@ -1,121 +1,121 @@ package main -import ( - "context" - "fmt" - "log" - "sync/atomic" - "time" +// import ( +// "context" +// "fmt" +// "log" +// "sync/atomic" +// "time" - "github.com/lovoo/goka" - "github.com/lovoo/goka/codec" - "github.com/lovoo/goka/kafka" -) +// "github.com/lovoo/goka" +// "github.com/lovoo/goka/codec" +// "github.com/lovoo/goka/kafka" +// ) -func main() { +// func main() { - proc, err := goka.NewProcessor([]string{"localhost:9092"}, - goka.DefineGroup("processor-stuck-test", - goka.Input("input", new(codec.Int64), func(ctx goka.Context, msg interface{}) { - ctx.SetValue(msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - ctx.Emit("output", ctx.Key(), msg) - }), - goka.Output("output", new(codec.Int64)), - goka.Persist(new(codec.Int64)), - )) +// proc, err := goka.NewProcessor([]string{"localhost:9092"}, +// goka.DefineGroup("processor-stuck-test", +// goka.Input("input", new(codec.Int64), func(ctx goka.Context, msg interface{}) { +// ctx.SetValue(msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// ctx.Emit("output", ctx.Key(), msg) +// }), +// goka.Output("output", new(codec.Int64)), +// goka.Persist(new(codec.Int64)), +// )) - if err != nil { - log.Fatalf("Cannot start processor: %v", err) - } +// if err != nil { +// log.Fatalf("Cannot start processor: %v", err) +// } - ctx, cancel := context.WithCancel(context.Background()) - done := make(chan struct{}) - go func() { - defer close(done) +// ctx, cancel := context.WithCancel(context.Background()) +// done := make(chan struct{}) +// go func() { +// defer close(done) - log.Printf("Running processor") - procRunErr := proc.Run(ctx) - log.Printf("Processor finished with %v", procRunErr) - }() +// log.Printf("Running processor") +// procRunErr := proc.Run(ctx) +// log.Printf("Processor finished with %v", procRunErr) +// }() - log.Printf("wait 5 seconds before starting to emit") - time.Sleep(5 * time.Second) +// log.Printf("wait 5 seconds before starting to emit") +// time.Sleep(5 * time.Second) - for i := 0; i < 50; i++ { - go func() { +// for i := 0; i < 50; i++ { +// go func() { - cfg := kafka.NewConfig() - cfg.Producer.Retry.Max = 0 - cfg.Producer.Retry.Backoff = 1 * time.Millisecond - emitter, err := goka.NewEmitter([]string{"localhost:9092"}, "input", new(codec.Int64), - goka.WithEmitterProducerBuilder( - kafka.ProducerBuilderWithConfig(cfg), - ), - ) - if err != nil { - log.Fatalf("Error creating emitter: %v", err) - } +// cfg := kafka.NewConfig() +// cfg.Producer.Retry.Max = 0 +// cfg.Producer.Retry.Backoff = 1 * time.Millisecond +// emitter, err := goka.NewEmitter([]string{"localhost:9092"}, "input", new(codec.Int64), +// goka.WithEmitterProducerBuilder( +// kafka.ProducerBuilderWithConfig(cfg), +// ), +// ) +// if err != nil { +// log.Fatalf("Error creating emitter: %v", err) +// } - time.Sleep(2 * time.Second) - defer func() { - log.Printf("finishing") - emitter.Finish() - log.Printf("done") - }() +// time.Sleep(2 * time.Second) +// defer func() { +// log.Printf("finishing") +// emitter.Finish() +// log.Printf("done") +// }() - defer recover() - var done int64 - var emitted int64 - for i := 0; ; i++ { - if atomic.LoadInt64(&done) > 0 { - break - } +// defer recover() +// var done int64 +// var emitted int64 +// for i := 0; ; i++ { +// if atomic.LoadInt64(&done) > 0 { +// break +// } - // when the context is done, stop emitting - go func() { - <-ctx.Done() - atomic.AddInt64(&done, 1) - }() - emitted++ - if emitted%1000 == 0 { - log.Printf("emitted %d", emitted) - } - prom, err := emitter.Emit(fmt.Sprintf("%d", i), int64(i)) - if err != nil { - break - } - prom.Then(func(err error) { - if err != nil { - atomic.AddInt64(&done, 1) - } - }) - time.Sleep(10 * time.Millisecond) - } - }() - } +// // when the context is done, stop emitting +// go func() { +// <-ctx.Done() +// atomic.AddInt64(&done, 1) +// }() +// emitted++ +// if emitted%1000 == 0 { +// log.Printf("emitted %d", emitted) +// } +// prom, err := emitter.Emit(fmt.Sprintf("%d", i), int64(i)) +// if err != nil { +// break +// } +// prom.Then(func(err error) { +// if err != nil { +// atomic.AddInt64(&done, 1) +// } +// }) +// time.Sleep(10 * time.Millisecond) +// } +// }() +// } - log.Printf("waiting for the processor to shutdown") - <-done - log.Printf("processor is dead. Nice!") +// log.Printf("waiting for the processor to shutdown") +// <-done +// log.Printf("processor is dead. Nice!") - cancel() -} +// cancel() +// } diff --git a/internal/test/test.go b/internal/test/test.go new file mode 100644 index 00000000..a6c92819 --- /dev/null +++ b/internal/test/test.go @@ -0,0 +1,95 @@ +package test + +import ( + "reflect" + "regexp" + "runtime/debug" + "strings" +) + +type Fataler interface { + Fatalf(string, ...interface{}) +} + +func AssertNil(t Fataler, actual interface{}) { + value := reflect.ValueOf(actual) + if value.IsValid() { + if !value.IsNil() { + t.Fatalf("Expected value to be nil, but was not nil in %s", string(debug.Stack())) + } + } +} + +func AssertNotNil(t Fataler, actual interface{}) { + value := reflect.ValueOf(actual) + if !value.IsValid() || value.IsNil() { + t.Fatalf("Expected value to be not nil, but was nil in %s", string(debug.Stack())) + } +} + +func AssertTrue(t Fataler, value bool) { + if !value { + t.Fatalf("Expected value to be true, but was false in %s", string(debug.Stack())) + } +} +func AssertFalse(t Fataler, value bool) { + if value { + t.Fatalf("Expected value to be false, but was true in %s", string(debug.Stack())) + } +} + +func AssertEqual(t Fataler, actual, expected interface{}) { + if !reflect.DeepEqual(actual, expected) { + t.Fatalf("Expected values were not equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) + } +} + +func PanicAssertEqual(t Fataler, expected interface{}) { + if expected == nil { + panic("can't pass nil to test.PanicAssertEqual") + } + if actual := recover(); actual != nil { + if !reflect.DeepEqual(actual, expected) { + t.Fatalf("Expected values were equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) + } + } else { + t.Fatalf("panic expected") + } + +} + +func AssertNotEqual(t Fataler, actual, expected interface{}) { + if reflect.DeepEqual(actual, expected) { + t.Fatalf("Expected values were equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) + } +} + +func AssertFuncEqual(t Fataler, actual, expected interface{}) { + if !(reflect.ValueOf(actual).Pointer() == reflect.ValueOf(expected).Pointer()) { + t.Fatalf("Expected functions were equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) + } +} + +func AssertError(t Fataler, actual error, reg *regexp.Regexp) { + if actual == nil || reg == nil { + t.Fatalf("Error or regexp is nil.\nactual=%#v\nregexp=%#v in %s", actual, reg, string(debug.Stack())) + } + if !reg.MatchString(actual.(error).Error()) { + t.Fatalf("Expected but got.\nactual=%#v\nregexp=%#v in %s", actual, reg, string(debug.Stack())) + } +} + +func AssertStringContains(t Fataler, actual string, contains string) { + if !strings.Contains(actual, contains) { + t.Fatalf("Expected string to contain substring \nactual=%#v\nexpected=%#v in %s", actual, contains, string(debug.Stack())) + } +} + +func PanicAssertStringContains(t Fataler, s string) { + if r := recover(); r != nil { + err := r.(error) + AssertStringContains(t, err.Error(), s) + } else { + t.Fatalf("panic expected") + } +} diff --git a/iterator_test.go b/iterator_test.go index 537fcc48..0c0c4844 100644 --- a/iterator_test.go +++ b/iterator_test.go @@ -4,21 +4,21 @@ import ( "io/ioutil" "testing" - "github.com/facebookgo/ensure" "github.com/lovoo/goka/codec" + "github.com/lovoo/goka/internal/test" "github.com/lovoo/goka/storage" "github.com/syndtr/goleveldb/leveldb" ) func TestIterator(t *testing.T) { tmpdir, err := ioutil.TempDir("", "goka_storage_TestIterator") - ensure.Nil(t, err) + test.AssertNil(t, err) db, err := leveldb.OpenFile(tmpdir, nil) - ensure.Nil(t, err) + test.AssertNil(t, err) st, err := storage.New(db) - ensure.Nil(t, err) + test.AssertNil(t, err) kv := map[string]string{ "key-1": "val-1", @@ -27,13 +27,13 @@ func TestIterator(t *testing.T) { } for k, v := range kv { - ensure.Nil(t, st.Set(k, []byte(v))) + test.AssertNil(t, st.Set(k, []byte(v))) } - ensure.Nil(t, st.SetOffset(777)) + test.AssertNil(t, st.SetOffset(777)) iter, err := st.Iterator() - ensure.Nil(t, err) + test.AssertNil(t, err) it := &iterator{ iter: storage.NewMultiIterator([]storage.Iterator{iter}), @@ -44,8 +44,8 @@ func TestIterator(t *testing.T) { // accessing iterator before Next should only return nils val, err := it.Value() - ensure.True(t, val == nil) - ensure.Nil(t, err) + test.AssertTrue(t, val == nil) + test.AssertNil(t, err) for it.Next() { count++ @@ -56,13 +56,13 @@ func TestIterator(t *testing.T) { } val, err := it.Value() - ensure.Nil(t, err) - ensure.DeepEqual(t, expected, val.(string)) + test.AssertNil(t, err) + test.AssertEqual(t, expected, val.(string)) } if err := it.Err(); err != nil { t.Fatalf("unexpected iteration error: %v", err) } - ensure.DeepEqual(t, count, len(kv)) + test.AssertEqual(t, count, len(kv)) } diff --git a/kafka/builders.go b/kafka/builders.go deleted file mode 100644 index 52a9d061..00000000 --- a/kafka/builders.go +++ /dev/null @@ -1,80 +0,0 @@ -package kafka - -import ( - "hash" - - "github.com/Shopify/sarama" - cluster "github.com/bsm/sarama-cluster" -) - -// ConsumerBuilder creates a Kafka consumer. -type ConsumerBuilder func(brokers []string, group, clientID string) (Consumer, error) - -// DefaultConsumerBuilder creates a Kafka consumer using the Sarama library. -func DefaultConsumerBuilder(brokers []string, group, clientID string) (Consumer, error) { - config := NewConfig() - config.ClientID = clientID - return NewSaramaConsumer(brokers, group, config) -} - -// ConsumerBuilderWithConfig creates a Kafka consumer using the Sarama library. -func ConsumerBuilderWithConfig(config *cluster.Config) ConsumerBuilder { - return func(brokers []string, group, clientID string) (Consumer, error) { - config.ClientID = clientID - return NewSaramaConsumer(brokers, group, config) - } -} - -// ProducerBuilder create a Kafka producer. -type ProducerBuilder func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) - -// DefaultProducerBuilder creates a Kafka producer using the Sarama library. -func DefaultProducerBuilder(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { - config := NewConfig() - config.ClientID = clientID - config.Producer.Partitioner = sarama.NewCustomHashPartitioner(hasher) - return NewProducer(brokers, &config.Config) -} - -// ProducerBuilderWithConfig creates a Kafka consumer using the Sarama library. -func ProducerBuilderWithConfig(config *cluster.Config) ProducerBuilder { - return func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { - config.ClientID = clientID - config.Producer.Partitioner = sarama.NewCustomHashPartitioner(hasher) - return NewProducer(brokers, &config.Config) - } -} - -// TopicManagerBuilder creates a TopicManager to check partition counts and -// create tables. -type TopicManagerBuilder func(brokers []string) (TopicManager, error) - -// DefaultTopicManagerBuilder creates TopicManager using the Sarama library. -// This topic manager cannot create topics. -func DefaultTopicManagerBuilder(brokers []string) (TopicManager, error) { - return NewSaramaTopicManager(brokers, sarama.NewConfig()) -} - -// TopicManagerBuilderWithConfig creates TopicManager using the Sarama library. -// This topic manager cannot create topics. -func TopicManagerBuilderWithConfig(config *cluster.Config) TopicManagerBuilder { - return func(brokers []string) (TopicManager, error) { - return NewSaramaTopicManager(brokers, &config.Config) - } -} - -// ZKTopicManagerBuilder creates a TopicManager that connects with ZooKeeper to -// check partition counts and create tables. -func ZKTopicManagerBuilder(servers []string) TopicManagerBuilder { - return func([]string) (TopicManager, error) { - return NewTopicManager(servers, NewTopicManagerConfig()) - } -} - -// ZKTopicManagerBuilderWithConfig creates a TopicManager that connects with ZooKeeper to -// check partition counts and create tables given a topic configuration. -func ZKTopicManagerBuilderWithConfig(servers []string, config *TopicManagerConfig) TopicManagerBuilder { - return func([]string) (TopicManager, error) { - return NewTopicManager(servers, config) - } -} diff --git a/kafka/config.go b/kafka/config.go deleted file mode 100644 index a1e599aa..00000000 --- a/kafka/config.go +++ /dev/null @@ -1,33 +0,0 @@ -package kafka - -import ( - "github.com/Shopify/sarama" - cluster "github.com/bsm/sarama-cluster" -) - -// NewConfig creates a (bsm) sarama configuration with default values. -func NewConfig() *cluster.Config { - config := cluster.NewConfig() - config.Version = sarama.V0_10_1_0 - - // consumer configuration - config.Consumer.Return.Errors = true - config.Consumer.MaxProcessingTime = defaultMaxProcessingTime - // this configures the initial offset for streams. Tables are always - // consumed from OffsetOldest. - config.Consumer.Offsets.Initial = sarama.OffsetNewest - - // producer configuration - config.Producer.RequiredAcks = sarama.WaitForLocal - config.Producer.Compression = sarama.CompressionSnappy - config.Producer.Flush.Frequency = defaultFlushFrequency - config.Producer.Flush.Bytes = defaultFlushBytes - config.Producer.Return.Successes = true - config.Producer.Return.Errors = true - config.Producer.Retry.Max = defaultProducerMaxRetries - - // consumer group configuration - config.Group.Return.Notifications = true - - return config -} diff --git a/kafka/confluent/README.md b/kafka/confluent/README.md deleted file mode 100644 index 3e79dd34..00000000 --- a/kafka/confluent/README.md +++ /dev/null @@ -1,10 +0,0 @@ -This a wrapper around the confluent-kafka-go library. - -To use library -- Create processor or view with confluent consumer, eg, `goka.WithConsumerBuilder(confluent.NewConsumerBuilder(1000))` -- Install `librdkafka` in the compilation environment -- Compile the go binary with `-tags "confluent static"` -- Install `libssl1.0.0` `libsasl2-2` (or equivalent) in the execution environment - -Note that this is experimental, not well tested and features are missing (in particular `auto.commit` is set to true). - diff --git a/kafka/confluent/confluent.go b/kafka/confluent/confluent.go deleted file mode 100644 index dd8c1956..00000000 --- a/kafka/confluent/confluent.go +++ /dev/null @@ -1,323 +0,0 @@ -// +build confluent - -package confluent - -import ( - "fmt" - "log" - "strings" - - rdkafka "github.com/confluentinc/confluent-kafka-go/kafka" - "github.com/lovoo/goka" - "github.com/lovoo/goka/kafka" -) - -type topicPartitionInfo struct { - tp rdkafka.TopicPartition - hwm int64 - bofSent bool -} - -type confluent struct { - tablePartitions map[string]map[int32]topicPartitionInfo - streamPartitions map[int32][]rdkafka.TopicPartition - partitionMap map[int32]bool - - consumer confluentConsumer - events chan kafka.Event - groupTopics map[string]int64 - cmds chan interface{} - stop chan bool - done chan bool -} - -type addPartition struct { - topic string - partition int32 - initialOffset int64 -} - -type removePartition struct { - topic string - partition int32 -} - -type addGroupPartition struct { - partition int32 -} - -func NewConsumer(brokers []string, group string, bufsize int) (kafka.Consumer, error) { - consumer, err := rdkafka.NewConsumer( - &rdkafka.ConfigMap{ - "bootstrap.servers": strings.Join(brokers, ","), - "group.id": group, - "session.timeout.ms": 6000, - // TODO(diogo): implement Commit() - //"enable.auto.commit": false, - "go.events.channel.size": bufsize, - "go.events.channel.enable": true, - "go.application.rebalance.enable": true, - "default.topic.config": rdkafka.ConfigMap{"auto.offset.reset": "earliest"}, - }, - ) - if err != nil { - return nil, err - } - - c := &confluent{ - consumer: consumer, - tablePartitions: make(map[string]map[int32]topicPartitionInfo), - streamPartitions: make(map[int32][]rdkafka.TopicPartition), - partitionMap: make(map[int32]bool), - events: make(chan kafka.Event, 1024), - cmds: make(chan interface{}, 1024), - stop: make(chan bool), - done: make(chan bool), - } - - // start go routine - go c.run() - return c, nil -} - -// NewConsumerBuilder builds confluent-based consumers with channel size. -func NewConsumerBuilder(size int) goka.ConsumerBuilder { - return func(brokers []string, group, clientID string) (kafka.Consumer, error) { - consumer, err := NewConsumer(brokers, group, size) - if err != nil { - log.Fatalf("cannot create confluent consumer: %v", err) - } - return consumer, nil - } -} - -func (c *confluent) Events() <-chan kafka.Event { - return c.events -} - -func (c *confluent) Subscribe(topics map[string]int64) error { - log.Println("%% confluent %%", "subscribe", topics) - c.groupTopics = topics - - var tops []string - for topic := range topics { - tops = append(tops, string(topic)) - } - - err := c.consumer.SubscribeTopics(tops, nil) - if err != nil { - return err - } - return nil -} - -func (c *confluent) Commit(topic string, partition int32, offset int64) error { return nil } - -func (c *confluent) AddGroupPartition(partition int32) { - select { - case c.cmds <- &addGroupPartition{partition}: - case <-c.stop: - } -} - -func (c *confluent) AddPartition(topic string, partition int32, initialOffset int64) { - select { - case c.cmds <- &addPartition{topic, partition, initialOffset}: - case <-c.stop: - } -} - -func (c *confluent) RemovePartition(topic string, partition int32) { - select { - case c.cmds <- &removePartition{topic, partition}: - case <-c.stop: - } -} - -func (c *confluent) Close() error { - // stop go routines - close(c.stop) - <-c.done - - return nil -} - -func (c *confluent) run() { - defer close(c.done) - for { - select { - case ev := <-c.consumer.Events(): - // log.Println("%% confluent %%", "received event", ev) - switch e := ev.(type) { - case rdkafka.AssignedPartitions: - c.events <- c.rebalance(e) - - case rdkafka.RevokedPartitions: - c.consumer.Unassign() - - case *rdkafka.Message: - var ( - topic = *e.TopicPartition.Topic - partition = e.TopicPartition.Partition - ) - - headers := make(map[string][]byte) - for _, header := range e.Headers { - headers[header.Key] = header.Value - } - - c.events <- &kafka.Message{ - Topic: topic, - Partition: partition, - Offset: int64(e.TopicPartition.Offset), - Key: string(e.Key), - Value: e.Value, - Timestamp: e.Timestamp, - Header: headers, - } - - case rdkafka.PartitionEOF: - //log.Printf("%%%% confluent %%%% Reached %v\n", e) - c.events <- &kafka.EOF{ - Topic: *e.Topic, - Partition: e.Partition, - Hwm: int64(e.Offset), - } - - case rdkafka.Error: - c.events <- &kafka.Error{fmt.Errorf("error from rdkafka: %v", e)} - - default: - //log.Printf("HANDLE ME: %v", ev) - } - - case cmd := <-c.cmds: - switch cmd := cmd.(type) { - case *addPartition: - c.addPartition(cmd.topic, cmd.partition, cmd.initialOffset) - case *removePartition: - c.removePartition(cmd.topic, cmd.partition) - case *addGroupPartition: - c.addGroupPartition(cmd.partition) - default: - c.events <- &kafka.Error{fmt.Errorf("invalid command: %T", cmd)} - } - - case <-c.stop: - log.Println("%% confluent %% terminated") - return - } - } -} - -func (c *confluent) addGroupPartition(partition int32) { - log.Println("%% confluent %%", "adding group partition", partition) - c.partitionMap[partition] = true - c.reassign() -} - -func (c *confluent) addPartition(topic string, partition int32, initialOffset int64) { - log.Println("%% confluent %%", "adding topic partition", topic, partition, initialOffset) - if _, has := c.tablePartitions[topic]; !has { - c.tablePartitions[topic] = make(map[int32]topicPartitionInfo) - } - c.tablePartitions[topic][partition] = topicPartitionInfo{ - tp: rdkafka.TopicPartition{ - Topic: &topic, - Partition: partition, - Offset: rdkafka.Offset(initialOffset), - Error: nil, - }, - } - - // send BOF - l, h, err := c.consumer.QueryWatermarkOffsets(topic, partition, 500) - if err != nil { - select { - case c.events <- &kafka.Error{fmt.Errorf("error querying watermarks: %v", err)}: - case <-c.stop: - return - } - } - select { - case c.events <- &kafka.BOF{ - Topic: topic, - Partition: partition, - Offset: l, - Hwm: h, - }: - case <-c.stop: - return - } - - c.reassign() -} - -func (c *confluent) removePartition(topic string, partition int32) { - log.Println("%% confluent %%", "remove topic partition", topic, partition) - if _, has := c.tablePartitions[topic]; !has { - return - } - if _, has := c.tablePartitions[topic][partition]; !has { - return - } - delete(c.tablePartitions[topic], partition) - if len(c.tablePartitions[topic]) == 0 { - delete(c.tablePartitions, topic) - } - c.reassign() -} - -func (c *confluent) reassign() { - var tps []rdkafka.TopicPartition - for p, tp := range c.streamPartitions { - if c.partitionMap[p] { - tps = append(tps, tp...) - } - } - for _, m := range c.tablePartitions { - for _, tp := range m { - tps = append(tps, tp.tp) - } - } - c.consumer.Assign(tps) -} - -func (c *confluent) rebalance(e rdkafka.AssignedPartitions) *kafka.Assignment { - var ( - as = make(kafka.Assignment) - pm = c.partitionMap - ) - - c.partitionMap = make(map[int32]bool) - for _, p := range e.Partitions { - if p.Offset == -1001 { - off := c.groupTopics[*p.Topic] - as[p.Partition] = off - } else { - as[p.Partition] = int64(p.Offset) - } - - c.streamPartitions[p.Partition] = append(c.streamPartitions[p.Partition], p) - c.partitionMap[p.Partition] = pm[p.Partition] // keep already assigned partitions - } - return &as -} - -//go:generate mockgen -package confluent -destination confluent_mock_test.go -source=confluent.go confluentConsumer -type confluentConsumer interface { - Assign(partitions []rdkafka.TopicPartition) (err error) - Close() (err error) - Commit() ([]rdkafka.TopicPartition, error) - CommitMessage(m *rdkafka.Message) ([]rdkafka.TopicPartition, error) - CommitOffsets(offsets []rdkafka.TopicPartition) ([]rdkafka.TopicPartition, error) - Events() chan rdkafka.Event - GetMetadata(topic *string, allTopics bool, timeoutMs int) (*rdkafka.Metadata, error) - Poll(timeoutMs int) (event rdkafka.Event) - QueryWatermarkOffsets(topic string, partition int32, timeoutMs int) (low, high int64, err error) - String() string - Subscribe(topic string, rebalanceCb rdkafka.RebalanceCb) error - SubscribeTopics(topics []string, rebalanceCb rdkafka.RebalanceCb) (err error) - Unassign() (err error) - Unsubscribe() (err error) -} diff --git a/kafka/confluent/confluent_mock_test.go b/kafka/confluent/confluent_mock_test.go deleted file mode 100644 index d4d9e9f8..00000000 --- a/kafka/confluent/confluent_mock_test.go +++ /dev/null @@ -1,178 +0,0 @@ -// +build confluent - -// Automatically generated by MockGen. DO NOT EDIT! -// Source: confluent.go - -package confluent - -import ( - kafka "github.com/confluentinc/confluent-kafka-go/kafka" - gomock "github.com/golang/mock/gomock" -) - -// Mock of confluentConsumer interface -type MockconfluentConsumer struct { - ctrl *gomock.Controller - recorder *_MockconfluentConsumerRecorder -} - -// Recorder for MockconfluentConsumer (not exported) -type _MockconfluentConsumerRecorder struct { - mock *MockconfluentConsumer -} - -func NewMockconfluentConsumer(ctrl *gomock.Controller) *MockconfluentConsumer { - mock := &MockconfluentConsumer{ctrl: ctrl} - mock.recorder = &_MockconfluentConsumerRecorder{mock} - return mock -} - -func (_m *MockconfluentConsumer) EXPECT() *_MockconfluentConsumerRecorder { - return _m.recorder -} - -func (_m *MockconfluentConsumer) Assign(partitions []kafka.TopicPartition) error { - ret := _m.ctrl.Call(_m, "Assign", partitions) - ret0, _ := ret[0].(error) - return ret0 -} - -func (_mr *_MockconfluentConsumerRecorder) Assign(arg0 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Assign", arg0) -} - -func (_m *MockconfluentConsumer) Close() error { - ret := _m.ctrl.Call(_m, "Close") - ret0, _ := ret[0].(error) - return ret0 -} - -func (_mr *_MockconfluentConsumerRecorder) Close() *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Close") -} - -func (_m *MockconfluentConsumer) Commit() ([]kafka.TopicPartition, error) { - ret := _m.ctrl.Call(_m, "Commit") - ret0, _ := ret[0].([]kafka.TopicPartition) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -func (_mr *_MockconfluentConsumerRecorder) Commit() *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Commit") -} - -func (_m *MockconfluentConsumer) CommitMessage(m *kafka.Message) ([]kafka.TopicPartition, error) { - ret := _m.ctrl.Call(_m, "CommitMessage", m) - ret0, _ := ret[0].([]kafka.TopicPartition) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -func (_mr *_MockconfluentConsumerRecorder) CommitMessage(arg0 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "CommitMessage", arg0) -} - -func (_m *MockconfluentConsumer) CommitOffsets(offsets []kafka.TopicPartition) ([]kafka.TopicPartition, error) { - ret := _m.ctrl.Call(_m, "CommitOffsets", offsets) - ret0, _ := ret[0].([]kafka.TopicPartition) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -func (_mr *_MockconfluentConsumerRecorder) CommitOffsets(arg0 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "CommitOffsets", arg0) -} - -func (_m *MockconfluentConsumer) Events() chan kafka.Event { - ret := _m.ctrl.Call(_m, "Events") - ret0, _ := ret[0].(chan kafka.Event) - return ret0 -} - -func (_mr *_MockconfluentConsumerRecorder) Events() *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Events") -} - -func (_m *MockconfluentConsumer) GetMetadata(topic *string, allTopics bool, timeoutMs int) (*kafka.Metadata, error) { - ret := _m.ctrl.Call(_m, "GetMetadata", topic, allTopics, timeoutMs) - ret0, _ := ret[0].(*kafka.Metadata) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -func (_mr *_MockconfluentConsumerRecorder) GetMetadata(arg0, arg1, arg2 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "GetMetadata", arg0, arg1, arg2) -} - -func (_m *MockconfluentConsumer) Poll(timeoutMs int) kafka.Event { - ret := _m.ctrl.Call(_m, "Poll", timeoutMs) - ret0, _ := ret[0].(kafka.Event) - return ret0 -} - -func (_mr *_MockconfluentConsumerRecorder) Poll(arg0 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Poll", arg0) -} - -func (_m *MockconfluentConsumer) QueryWatermarkOffsets(topic string, partition int32, timeoutMs int) (int64, int64, error) { - ret := _m.ctrl.Call(_m, "QueryWatermarkOffsets", topic, partition, timeoutMs) - ret0, _ := ret[0].(int64) - ret1, _ := ret[1].(int64) - ret2, _ := ret[2].(error) - return ret0, ret1, ret2 -} - -func (_mr *_MockconfluentConsumerRecorder) QueryWatermarkOffsets(arg0, arg1, arg2 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "QueryWatermarkOffsets", arg0, arg1, arg2) -} - -func (_m *MockconfluentConsumer) String() string { - ret := _m.ctrl.Call(_m, "String") - ret0, _ := ret[0].(string) - return ret0 -} - -func (_mr *_MockconfluentConsumerRecorder) String() *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "String") -} - -func (_m *MockconfluentConsumer) Subscribe(topic string, rebalanceCb kafka.RebalanceCb) error { - ret := _m.ctrl.Call(_m, "Subscribe", topic, rebalanceCb) - ret0, _ := ret[0].(error) - return ret0 -} - -func (_mr *_MockconfluentConsumerRecorder) Subscribe(arg0, arg1 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Subscribe", arg0, arg1) -} - -func (_m *MockconfluentConsumer) SubscribeTopics(topics []string, rebalanceCb kafka.RebalanceCb) error { - ret := _m.ctrl.Call(_m, "SubscribeTopics", topics, rebalanceCb) - ret0, _ := ret[0].(error) - return ret0 -} - -func (_mr *_MockconfluentConsumerRecorder) SubscribeTopics(arg0, arg1 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "SubscribeTopics", arg0, arg1) -} - -func (_m *MockconfluentConsumer) Unassign() error { - ret := _m.ctrl.Call(_m, "Unassign") - ret0, _ := ret[0].(error) - return ret0 -} - -func (_mr *_MockconfluentConsumerRecorder) Unassign() *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Unassign") -} - -func (_m *MockconfluentConsumer) Unsubscribe() error { - ret := _m.ctrl.Call(_m, "Unsubscribe") - ret0, _ := ret[0].(error) - return ret0 -} - -func (_mr *_MockconfluentConsumerRecorder) Unsubscribe() *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Unsubscribe") -} diff --git a/kafka/confluent/confluent_test.go b/kafka/confluent/confluent_test.go deleted file mode 100644 index 97f19f37..00000000 --- a/kafka/confluent/confluent_test.go +++ /dev/null @@ -1,30 +0,0 @@ -// +build confluent - -package confluent - -import ( - "testing" - - "github.com/golang/mock/gomock" - "github.com/lovoo/goka/kafka" -) - -func newMockConfluent(consumer confluentConsumer) *confluent { - return &confluent{ - consumer: consumer, - events: make(chan kafka.Event), - stop: make(chan bool), - done: make(chan bool), - } -} - -func TestConfluent1(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - consumer := NewMockconfluentConsumer(ctrl) - c := newMockConfluent(consumer) - - consumer.EXPECT().SubscribeTopics([]string{"t1"}, nil).Return(nil) - c.Subscribe(map[string]int64{"t1": -1}) -} diff --git a/kafka/consumer.go b/kafka/consumer.go deleted file mode 100644 index a634b109..00000000 --- a/kafka/consumer.go +++ /dev/null @@ -1,118 +0,0 @@ -package kafka - -import ( - "time" - - "github.com/Shopify/sarama" - cluster "github.com/bsm/sarama-cluster" - "github.com/lovoo/goka/multierr" -) - -const ( - // size of sarama buffer for consumer and producer - defaultChannelBufferSize = 256 - - // time sarama-cluster assumes the processing of an event may take - defaultMaxProcessingTime = 1 * time.Second - - // producer flush configuration - defaultFlushFrequency = 100 * time.Millisecond - defaultFlushBytes = 64 * 1024 - defaultProducerMaxRetries = 10 -) - -const ( - // OffsetNewest defines the newest offset to read from using the consumer - OffsetNewest = -1 - // OffsetOldest defines the oldest offset to read from using the consumer - OffsetOldest = -2 -) - -// Consumer abstracts a kafka consumer -type Consumer interface { - Events() <-chan Event - - // group consume assumes co-partioned topics - // define input topics to consume - Subscribe(topics map[string]int64) error - // marks the consumer ready to start consuming the messages - AddGroupPartition(partition int32) - Commit(topic string, partition int32, offset int64) error - - // consume individual topic/partitions - AddPartition(topic string, partition int32, initialOffset int64) error - RemovePartition(topic string, partition int32) error - - // Close stops closes the events channel - Close() error -} - -type saramaConsumer struct { - groupConsumer *groupConsumer - simpleConsumer *simpleConsumer - events chan Event -} - -// NewSaramaConsumer creates a new Consumer using sarama -func NewSaramaConsumer(brokers []string, group string, config *cluster.Config) (Consumer, error) { - chsize := config.Config.ChannelBufferSize - if chsize == 0 { - chsize = defaultChannelBufferSize - } - events := make(chan Event, chsize) - - g, err := newGroupConsumer(brokers, group, events, config) - if err != nil { - return nil, err - } - - // since simple consumer only handle tables, be sure to start from oldest - simpleConfig := config.Config // copy config - simpleConfig.Consumer.Offsets.Initial = sarama.OffsetOldest - c, err := newSimpleConsumer(brokers, events, &simpleConfig) - if err != nil { - return nil, err - } - - return &saramaConsumer{ - groupConsumer: g, - simpleConsumer: c, - events: events, - }, nil -} - -func (c *saramaConsumer) Close() error { - // we want to close the events-channel regardless of any errors closing - // the consumers - defer close(c.events) - var errs multierr.Errors - if err := c.simpleConsumer.Close(); err != nil { - errs.Collect(err) - } - if err := c.groupConsumer.Close(); err != nil { - errs.Collect(err) - } - return errs.NilOrError() -} - -func (c *saramaConsumer) Events() <-chan Event { - return c.events -} - -// group consume assumes co-partioned topics -func (c *saramaConsumer) Subscribe(topics map[string]int64) error { - return c.groupConsumer.Subscribe(topics) -} -func (c *saramaConsumer) AddGroupPartition(partition int32) { - c.groupConsumer.AddGroupPartition(partition) -} -func (c *saramaConsumer) Commit(topic string, partition int32, offset int64) error { - return c.groupConsumer.Commit(topic, partition, offset) -} - -func (c *saramaConsumer) AddPartition(topic string, partition int32, initialOffset int64) error { - return c.simpleConsumer.AddPartition(topic, partition, int64(initialOffset)) -} -func (c *saramaConsumer) RemovePartition(topic string, partition int32) error { - return c.simpleConsumer.RemovePartition(topic, partition) -} diff --git a/kafka/event.go b/kafka/event.go deleted file mode 100644 index 76e7e5f6..00000000 --- a/kafka/event.go +++ /dev/null @@ -1,78 +0,0 @@ -package kafka - -import ( - "fmt" - "time" -) - -// Event abstracts different types of events from the kafka consumer like BOF/EOF/Error or an actual message -type Event interface { - string() string -} - -// Assignment represents a partition:offset assignment for the current connection -type Assignment map[int32]int64 - -func (a *Assignment) string() string { - var am map[int32]int64 = *a - return fmt.Sprintf("Assignment %v", am) -} - -// EOF marks the end of the log of a topic/partition. -type EOF struct { - Topic string - Partition int32 - Hwm int64 -} - -func (e *EOF) string() string { - return fmt.Sprintf("EOF %s/%d:%d", e.Topic, e.Partition, e.Hwm) -} - -// BOF marks the beginning of a topic/partition. -type BOF struct { - Topic string - Partition int32 - Offset int64 - Hwm int64 -} - -func (e *BOF) string() string { - return fmt.Sprintf("BOF %s/%d:%d->%d", e.Topic, e.Partition, e.Offset, e.Hwm) -} - -// Message represents a message from kafka containing -// extra information like topic, partition and offset for convenience -type Message struct { - Topic string - Partition int32 - Offset int64 - Timestamp time.Time - Header map[string][]byte - - Key string - Value []byte -} - -func (m *Message) string() string { - return fmt.Sprintf("Message %s/%d:%d %s=%v", m.Topic, m.Partition, m.Offset, m.Key, m.Value) -} - -// Error from kafka wrapped to be conform with the Event-Interface -type Error struct { - Err error -} - -func (e *Error) string() string { - return e.Err.Error() -} - -// NOP does not carry any information. Useful for debugging. -type NOP struct { - Topic string - Partition int32 -} - -func (n *NOP) string() string { - return "nop" -} diff --git a/kafka/group_consumer.go b/kafka/group_consumer.go deleted file mode 100644 index 8128fb63..00000000 --- a/kafka/group_consumer.go +++ /dev/null @@ -1,279 +0,0 @@ -package kafka - -import ( - "fmt" - "log" - "sync/atomic" - - "github.com/Shopify/sarama" - cluster "github.com/bsm/sarama-cluster" -) - -type groupConsumer struct { - brokers []string - config *cluster.Config - consumer clusterConsumer - - group string - partitionMap map[int32]bool - addPartition chan int32 - - events chan Event - stop chan bool - done chan bool - running int64 -} - -func newGroupConsumer(brokers []string, group string, events chan Event, config *cluster.Config) (*groupConsumer, error) { - return &groupConsumer{ - group: group, - brokers: brokers, - config: config, - partitionMap: make(map[int32]bool), - addPartition: make(chan int32, 2048), - events: events, - stop: make(chan bool), - done: make(chan bool), - }, nil -} - -func (c *groupConsumer) Close() error { - if atomic.LoadInt64(&c.running) == 0 { - // not running - return nil - } - close(c.stop) - <-c.done - if err := c.consumer.Close(); err != nil { - return fmt.Errorf("Failed to close consumer: %v", err) - } - return nil -} - -func (c *groupConsumer) Subscribe(topics map[string]int64) error { - var ts []string - for t := range topics { - ts = append(ts, string(t)) - } - upConsumer, err := cluster.NewConsumer(c.brokers, c.group, ts, c.config) - if err != nil { - return err - } - c.consumer = upConsumer - - go c.run() - - return nil -} - -func (c *groupConsumer) waitForRebalanceOK() bool { - for { - select { - case n := <-c.consumer.Notifications(): - if !c.checkRebalance(cluster.RebalanceStart, n.Type) { - continue - } - - select { - case nn := <-c.consumer.Notifications(): - if !c.checkRebalance(cluster.RebalanceOK, nn.Type) { - continue - } - - return c.handleRebalanceOK(nn) - case <-c.stop: - return false - } - case err := <-c.consumer.Errors(): - select { - case c.events <- &Error{err}: - case <-c.stop: - return false - } - case <-c.stop: - return false - } - } -} - -func (c *groupConsumer) checkRebalance(expected, actual cluster.NotificationType) bool { - if actual != expected { - select { - case c.events <- &Error{fmt.Errorf("expected %s but received %s", expected, actual)}: - case <-c.stop: - } - - return false - } - - return true -} - -func (c *groupConsumer) handleRebalanceOK(n *cluster.Notification) bool { - if n.Type != cluster.RebalanceOK { - // panic as this is a programming error - log.Panicf("GroupConsumer: unsupported notification type in handleRebalanceOK: %v/%s", n.Type, n.Type) - } - - // save partition map - m := c.partitionMap - c.partitionMap = make(map[int32]bool) - - // create assignment and update partitionMap - a := make(Assignment) - for _, v := range n.Current { - for _, p := range v { - a[p] = sarama.OffsetNewest - - // remember whether partition was added using m[p] - c.partitionMap[p] = m[p] - } - - break // copartitioned topics - } - - // send assignment - select { - case c.events <- &a: - return true - case <-c.stop: - return false - } -} - -// returns true if all partitions are registered. otherwise false -func (c *groupConsumer) partitionsRegistered() bool { - for _, v := range c.partitionMap { - if !v { - return false - } - } - return true -} - -func (c *groupConsumer) AddGroupPartition(partition int32) { - select { - case c.addPartition <- partition: - case <-c.stop: - } -} - -func (c *groupConsumer) waitForPartitions() bool { - defer c.ensureEmpty() - - // if all registered, start consuming - if c.partitionsRegistered() { - return true - } - - for { - select { - case par := <-c.addPartition: - c.partitionMap[par] = true - - // if all registered, start consuming - if c.partitionsRegistered() { - return true - } - - case <-c.stop: - return false - } - } -} - -func (c *groupConsumer) ensureEmpty() { - for { - select { - case <-c.addPartition: - default: - return - } - } -} - -func (c *groupConsumer) waitForMessages() bool { - for { - select { - case n := <-c.consumer.Notifications(): - if !c.checkRebalance(cluster.RebalanceStart, n.Type) { - continue - } - - select { - case nn := <-c.consumer.Notifications(): - if !c.checkRebalance(cluster.RebalanceOK, nn.Type) { - continue - } - - return c.handleRebalanceOK(nn) - case <-c.stop: - return false - } - case msg := <-c.consumer.Messages(): - - headers := make(map[string][]byte) - for _, header := range msg.Headers { - headers[string(header.Key)] = header.Value - } - - select { - case c.events <- &Message{ - Topic: msg.Topic, - Partition: msg.Partition, - Offset: msg.Offset, - Timestamp: msg.Timestamp, - Key: string(msg.Key), - Value: msg.Value, - Header: headers, - }: - case <-c.stop: - return false - } - - case err := <-c.consumer.Errors(): - select { - case c.events <- &Error{err}: - case <-c.stop: - return false - } - - case <-c.stop: - return false - } - } -} - -func (c *groupConsumer) run() { - atomic.AddInt64(&c.running, 1) - defer close(c.done) - - if !c.waitForRebalanceOK() { - return - } - - for { - if !c.waitForPartitions() { - return - } - - if !c.waitForMessages() { - return - } - } -} - -func (c *groupConsumer) Commit(topic string, partition int32, offset int64) error { - c.consumer.MarkPartitionOffset(topic, partition, offset, "") - return nil -} - -//go:generate mockgen -package mock -destination=mock/cluster_consumer.go -source=group_consumer.go clusterConsumer -type clusterConsumer interface { - Close() error - MarkPartitionOffset(topic string, partition int32, offset int64, metadata string) - - Notifications() <-chan *cluster.Notification - Messages() <-chan *sarama.ConsumerMessage - Errors() <-chan error -} diff --git a/kafka/group_consumer_test.go b/kafka/group_consumer_test.go deleted file mode 100644 index c0bec735..00000000 --- a/kafka/group_consumer_test.go +++ /dev/null @@ -1,618 +0,0 @@ -package kafka - -import ( - "errors" - "sync/atomic" - "testing" - "time" - - "github.com/lovoo/goka/kafka/mock" - - "github.com/Shopify/sarama" - cluster "github.com/bsm/sarama-cluster" - "github.com/facebookgo/ensure" - "github.com/golang/mock/gomock" -) - -var ( - brokers = []string{"localhost:9092"} - group = "group" - topic1 = "topic1" - topic2 = "topic2" - topics = map[string]int64{topic1: -1, topic2: -2} -) - -func TestGroupConsumer_Subscribe(t *testing.T) { - events := make(chan Event) - c, err := newGroupConsumer(brokers, group, events, nil) - ensure.Nil(t, err) - ctrl := gomock.NewController(t) - defer ctrl.Finish() - consumer := mock.NewMockclusterConsumer(ctrl) - c.consumer = consumer - - notifications := make(chan *cluster.Notification) - errs := make(chan error) - consumer.EXPECT().Notifications().Return(notifications) - consumer.EXPECT().Errors().Return(errs) - - wait := make(chan bool) - go func() { - c.run() - close(wait) - }() - - // wait for running - for { - if atomic.LoadInt64(&c.running) != 0 { - break - } - time.Sleep(10 * time.Millisecond) - } - - consumer.EXPECT().Close().Return(nil) - err = doTimed(t, func() { - err = c.Close() - ensure.Nil(t, err) - <-wait - }) - ensure.Nil(t, err) -} - -func TestGroupConsumer_Group(t *testing.T) { - events := make(chan Event) - c, err := newGroupConsumer(brokers, group, events, nil) - ensure.Nil(t, err) - ctrl := gomock.NewController(t) - defer ctrl.Finish() - consumer := mock.NewMockclusterConsumer(ctrl) - c.consumer = consumer - - notifications := make(chan *cluster.Notification, 1) - errs := make(chan error, 1) - consumer.EXPECT().Notifications().Return(notifications).Times(2) - consumer.EXPECT().Errors().Return(errs) - - wait := make(chan bool) - go func() { - c.run() - close(wait) - }() - - // test an error - err = errors.New("some error") - errs <- err - consumer.EXPECT().Notifications().Return(notifications) - consumer.EXPECT().Errors().Return(errs) - e := <-events - ensure.DeepEqual(t, e.(*Error).Err, err) - - // notification arrives - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{ - topic1: []int32{0, 1}, - }} - n := <-events - - ensure.DeepEqual(t, n, &Assignment{ - 0: -1, - 1: -1, - }) - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: false, 1: false, - }) - // add a partition - c.AddGroupPartition(0) - - consumer.EXPECT().Close().Return(nil) - err = doTimed(t, func() { - err := c.Close() - ensure.Nil(t, err) - <-wait - }) - ensure.Nil(t, err) - - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: true, 1: false, - }) -} - -func TestGroupConsumer_CloseOnNotifications(t *testing.T) { - events := make(chan Event) - c, err := newGroupConsumer(brokers, group, events, nil) - ensure.Nil(t, err) - ctrl := gomock.NewController(t) - defer ctrl.Finish() - consumer := mock.NewMockclusterConsumer(ctrl) - c.consumer = consumer - - notifications := make(chan *cluster.Notification) - errs := make(chan error) - consumer.EXPECT().Notifications().Return(notifications).AnyTimes() - consumer.EXPECT().Errors().Return(errs).AnyTimes() - - wait := make(chan bool) - go func() { - c.run() - close(wait) - }() - - // close on error - err = errors.New("some error") - errs <- err - - consumer.EXPECT().Close().Return(nil) - err = doTimed(t, func() { - err = c.Close() - ensure.Nil(t, err) - <-wait - }) - ensure.Nil(t, err) - - // close on notification - c, err = newGroupConsumer(brokers, group, events, nil) - ensure.Nil(t, err) - c.consumer = consumer - - wait = make(chan bool) - go func() { - c.run() - close(wait) - }() - - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{ - topic1: []int32{0, 1}, - }} - - consumer.EXPECT().Close().Return(nil) - err = doTimed(t, func() { - err = c.Close() - ensure.Nil(t, err) - <-wait - }) - ensure.Nil(t, err) - -} - -func TestGroupConsumer_GroupConsumeMessages(t *testing.T) { - events := make(chan Event) - c, err := newGroupConsumer(brokers, group, events, nil) - ensure.Nil(t, err) - ctrl := gomock.NewController(t) - defer ctrl.Finish() - consumer := mock.NewMockclusterConsumer(ctrl) - c.consumer = consumer - - notifications := make(chan *cluster.Notification, 1) - errs := make(chan error, 1) - consumer.EXPECT().Notifications().Return(notifications).Times(2) - consumer.EXPECT().Errors().Return(errs) - - wait := make(chan bool) - go func() { - c.run() - close(wait) - }() - - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{ - topic1: []int32{0, 1}, - }} - n := <-events - ensure.DeepEqual(t, n, &Assignment{ - 0: -1, - 1: -1, - }) - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: false, 1: false, - }) - - // add partitions (it will start consuming messages channel) - messages := make(chan *sarama.ConsumerMessage) - consumer.EXPECT().Notifications().Return(notifications) - consumer.EXPECT().Messages().Return(messages) - consumer.EXPECT().Errors().Return(errs) - c.AddGroupPartition(0) - c.AddGroupPartition(1) - - // test an error - err = errors.New("some error") - errs <- err - consumer.EXPECT().Notifications().Return(notifications) - consumer.EXPECT().Messages().Return(messages) - consumer.EXPECT().Errors().Return(errs) - e := <-events - ensure.DeepEqual(t, e.(*Error).Err, err) - - // process a message - var ( - key = []byte("key") - value = []byte("value") - ) - messages <- &sarama.ConsumerMessage{ - Topic: topic1, - Partition: 0, - Offset: 0, - Key: key, - Value: value, - } - - // goroutine will loop after we dequeue Events - consumer.EXPECT().Notifications().Return(notifications) - consumer.EXPECT().Messages().Return(messages) - consumer.EXPECT().Errors().Return(errs) - m := <-events - ensure.DeepEqual(t, m, &Message{ - Topic: topic1, - Partition: 0, - Offset: 0, - Key: string(key), - Value: value, - Header: make(map[string][]byte), - }) - - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: true, 1: true, - }) - - consumer.EXPECT().Close().Return(errors.New("some error")) - err = doTimed(t, func() { - err := c.Close() - ensure.NotNil(t, err) - <-wait - }) - ensure.Nil(t, err) - -} - -func TestGroupConsumer_CloseOnMessages(t *testing.T) { - events := make(chan Event) - c, err := newGroupConsumer(brokers, group, events, nil) - ensure.Nil(t, err) - ctrl := gomock.NewController(t) - defer ctrl.Finish() - consumer := mock.NewMockclusterConsumer(ctrl) - c.consumer = consumer - - notifications := make(chan *cluster.Notification) - errs := make(chan error) - consumer.EXPECT().Notifications().Return(notifications).Times(2) - consumer.EXPECT().Errors().Return(errs) - - wait := make(chan bool) - go func() { - c.run() - close(wait) - }() - - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{ - topic1: []int32{0, 1}, - }} - n := <-events - ensure.DeepEqual(t, n, &Assignment{ - 0: -1, - 1: -1, - }) - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: false, 1: false, - }) - - // add partitions (it will start consuming messages channel) - messages := make(chan *sarama.ConsumerMessage) - consumer.EXPECT().Notifications().Return(notifications) - consumer.EXPECT().Messages().Return(messages) - consumer.EXPECT().Errors().Return(errs) - c.AddGroupPartition(0) - c.AddGroupPartition(1) - - // process a message - var ( - key = []byte("key") - value = []byte("value") - ) - messages <- &sarama.ConsumerMessage{ - Topic: topic1, - Partition: 0, - Offset: 0, - Key: key, - Value: value, - } - - consumer.EXPECT().Close().Return(nil) - err = doTimed(t, func() { - err = c.Close() - ensure.Nil(t, err) - <-wait - }) - ensure.Nil(t, err) -} - -func TestGroupConsumer_CloseOnMessageErrors(t *testing.T) { - events := make(chan Event) - c, err := newGroupConsumer(brokers, group, events, nil) - ensure.Nil(t, err) - ctrl := gomock.NewController(t) - defer ctrl.Finish() - consumer := mock.NewMockclusterConsumer(ctrl) - c.consumer = consumer - - notifications := make(chan *cluster.Notification) - errs := make(chan error) - consumer.EXPECT().Notifications().Return(notifications).Times(2) - consumer.EXPECT().Errors().Return(errs) - - wait := make(chan bool) - go func() { - c.run() - close(wait) - }() - - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{ - topic1: []int32{0, 1}, - }} - n := <-events - ensure.DeepEqual(t, n, &Assignment{ - 0: -1, - 1: -1, - }) - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: false, 1: false, - }) - - // add partitions (it will start consuming messages channel) - messages := make(chan *sarama.ConsumerMessage) - consumer.EXPECT().Notifications().Return(notifications) - consumer.EXPECT().Messages().Return(messages) - consumer.EXPECT().Errors().Return(errs) - c.AddGroupPartition(0) - c.AddGroupPartition(1) - - err = errors.New("some error") - errs <- err - - consumer.EXPECT().Close().Return(nil) - err = doTimed(t, func() { - err = c.Close() - ensure.Nil(t, err) - <-wait - }) - ensure.Nil(t, err) -} - -func TestGroupConsumer_GroupNotificationsAfterMessages(t *testing.T) { - events := make(chan Event) - c, err := newGroupConsumer(brokers, group, events, nil) - ensure.Nil(t, err) - ctrl := gomock.NewController(t) - defer ctrl.Finish() - consumer := mock.NewMockclusterConsumer(ctrl) - c.consumer = consumer - - notifications := make(chan *cluster.Notification, 1) - errs := make(chan error, 1) - consumer.EXPECT().Notifications().Return(notifications).Times(2) - consumer.EXPECT().Errors().Return(errs) - - wait := make(chan bool) - go func() { - c.run() - close(wait) - }() - - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{ - topic1: []int32{0, 1}, - }} - n := <-events - ensure.DeepEqual(t, n, &Assignment{ - 0: -1, - 1: -1, - }) - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: false, 1: false, - }) - - // add partitions (it will start consuming messages channel) - messages := make(chan *sarama.ConsumerMessage) - consumer.EXPECT().Notifications().Return(notifications) - consumer.EXPECT().Messages().Return(messages) - consumer.EXPECT().Errors().Return(errs) - c.AddGroupPartition(0) - c.AddGroupPartition(1) - - // process a message - var ( - key = []byte("key") - value = []byte("value") - ) - messages <- &sarama.ConsumerMessage{ - Topic: topic1, - Partition: 0, - Offset: 0, - Key: key, - Value: value, - } - - // goroutine will loop after we dequeue Events - consumer.EXPECT().Notifications().Return(notifications).Times(2) - consumer.EXPECT().Messages().Return(messages) - consumer.EXPECT().Errors().Return(errs) - m := <-events - ensure.DeepEqual(t, m, &Message{ - Topic: topic1, - Partition: 0, - Offset: 0, - Key: string(key), - Value: value, - Header: make(map[string][]byte), - }) - - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: true, 1: true, - }) - - // new notification - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{ - topic1: []int32{0, 1, 2}, - }} - n = <-events - ensure.DeepEqual(t, n, &Assignment{ - 0: -1, - 1: -1, - 2: -1, - }) - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: true, 1: true, 2: false, - }) - - consumer.EXPECT().Close().Return(nil) - err = doTimed(t, func() { - err := c.Close() - ensure.Nil(t, err) - <-wait - }) - ensure.Nil(t, err) - -} - -func TestGroupConsumer_GroupEmptyNotifications(t *testing.T) { - events := make(chan Event) - c, err := newGroupConsumer(brokers, group, events, nil) - ensure.Nil(t, err) - ctrl := gomock.NewController(t) - defer ctrl.Finish() - consumer := mock.NewMockclusterConsumer(ctrl) - c.consumer = consumer - - notifications := make(chan *cluster.Notification) - messages := make(chan *sarama.ConsumerMessage) - errs := make(chan error) - consumer.EXPECT().Notifications().Return(notifications).AnyTimes() - consumer.EXPECT().Messages().Return(messages).AnyTimes() - consumer.EXPECT().Errors().Return(errs).AnyTimes() - - wait := make(chan bool) - go func() { - c.run() - close(wait) - }() - - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{}, - } - n := <-events - ensure.DeepEqual(t, n, &Assignment{}) - - err = doTimed(t, func() { - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{ - topic1: []int32{0, 1}, - }} - n = <-events - }) - ensure.Nil(t, err) - ensure.DeepEqual(t, n, &Assignment{ - 0: -1, - 1: -1, - }) - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: false, 1: false, - }) - - // add partitions (it will start consuming messages channel) - c.AddGroupPartition(0) - c.AddGroupPartition(1) - - // process a message - var ( - key = []byte("key") - value = []byte("value") - ) - messages <- &sarama.ConsumerMessage{ - Topic: topic1, - Partition: 0, - Offset: 0, - Key: key, - Value: value, - } - - // goroutine will loop after we dequeue Events - m := <-events - ensure.DeepEqual(t, m, &Message{ - Topic: topic1, - Partition: 0, - Offset: 0, - Key: string(key), - Value: value, - Header: make(map[string][]byte), - }) - - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: true, 1: true, - }) - - // new notification - notifications <- &cluster.Notification{Type: cluster.RebalanceStart} - notifications <- &cluster.Notification{ - Type: cluster.RebalanceOK, - Current: map[string][]int32{ - topic1: []int32{0, 1, 2}, - }} - n = <-events - ensure.DeepEqual(t, n, &Assignment{ - 0: -1, - 1: -1, - 2: -1, - }) - ensure.DeepEqual(t, c.partitionMap, map[int32]bool{ - 0: true, 1: true, 2: false, - }) - - consumer.EXPECT().Close().Return(nil) - err = doTimed(t, func() { - err := c.Close() - ensure.Nil(t, err) - <-wait - }) - ensure.Nil(t, err) - -} - -func doTimed(t *testing.T, do func()) error { - ch := make(chan bool) - go func() { - do() - close(ch) - }() - - select { - case <-time.After(1 * time.Second): - t.Fail() - return errors.New("function took too long to complete") - case <-ch: - } - - return nil -} diff --git a/kafka/mock/cluster_consumer.go b/kafka/mock/cluster_consumer.go deleted file mode 100644 index 7eec3e6c..00000000 --- a/kafka/mock/cluster_consumer.go +++ /dev/null @@ -1,103 +0,0 @@ -// Code generated by MockGen. DO NOT EDIT. -// Source: group_consumer.go - -// Package mock is a generated GoMock package. -package mock - -import ( - sarama "github.com/Shopify/sarama" - sarama_cluster "github.com/bsm/sarama-cluster" - gomock "github.com/golang/mock/gomock" - reflect "reflect" -) - -// MockclusterConsumer is a mock of clusterConsumer interface -type MockclusterConsumer struct { - ctrl *gomock.Controller - recorder *MockclusterConsumerMockRecorder -} - -// MockclusterConsumerMockRecorder is the mock recorder for MockclusterConsumer -type MockclusterConsumerMockRecorder struct { - mock *MockclusterConsumer -} - -// NewMockclusterConsumer creates a new mock instance -func NewMockclusterConsumer(ctrl *gomock.Controller) *MockclusterConsumer { - mock := &MockclusterConsumer{ctrl: ctrl} - mock.recorder = &MockclusterConsumerMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockclusterConsumer) EXPECT() *MockclusterConsumerMockRecorder { - return m.recorder -} - -// Close mocks base method -func (m *MockclusterConsumer) 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 *MockclusterConsumerMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockclusterConsumer)(nil).Close)) -} - -// MarkPartitionOffset mocks base method -func (m *MockclusterConsumer) MarkPartitionOffset(topic string, partition int32, offset int64, metadata string) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "MarkPartitionOffset", topic, partition, offset, metadata) -} - -// MarkPartitionOffset indicates an expected call of MarkPartitionOffset -func (mr *MockclusterConsumerMockRecorder) MarkPartitionOffset(topic, partition, offset, metadata interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MarkPartitionOffset", reflect.TypeOf((*MockclusterConsumer)(nil).MarkPartitionOffset), topic, partition, offset, metadata) -} - -// Notifications mocks base method -func (m *MockclusterConsumer) Notifications() <-chan *sarama_cluster.Notification { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Notifications") - ret0, _ := ret[0].(<-chan *sarama_cluster.Notification) - return ret0 -} - -// Notifications indicates an expected call of Notifications -func (mr *MockclusterConsumerMockRecorder) Notifications() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Notifications", reflect.TypeOf((*MockclusterConsumer)(nil).Notifications)) -} - -// Messages mocks base method -func (m *MockclusterConsumer) Messages() <-chan *sarama.ConsumerMessage { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Messages") - ret0, _ := ret[0].(<-chan *sarama.ConsumerMessage) - return ret0 -} - -// Messages indicates an expected call of Messages -func (mr *MockclusterConsumerMockRecorder) Messages() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Messages", reflect.TypeOf((*MockclusterConsumer)(nil).Messages)) -} - -// Errors mocks base method -func (m *MockclusterConsumer) Errors() <-chan error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Errors") - ret0, _ := ret[0].(<-chan error) - return ret0 -} - -// Errors indicates an expected call of Errors -func (mr *MockclusterConsumerMockRecorder) Errors() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Errors", reflect.TypeOf((*MockclusterConsumer)(nil).Errors)) -} diff --git a/kafka/mock/kazoo.go b/kafka/mock/kazoo.go deleted file mode 100644 index 49649b39..00000000 --- a/kafka/mock/kazoo.go +++ /dev/null @@ -1,185 +0,0 @@ -// Code generated by MockGen. DO NOT EDIT. -// Source: topic_manager.go - -// Package mock is a generated GoMock package. -package mock - -import ( - gomock "github.com/golang/mock/gomock" - kazoo_go "github.com/wvanbergen/kazoo-go" - reflect "reflect" -) - -// MockTopicManager is a mock of TopicManager interface -type MockTopicManager struct { - ctrl *gomock.Controller - recorder *MockTopicManagerMockRecorder -} - -// MockTopicManagerMockRecorder is the mock recorder for MockTopicManager -type MockTopicManagerMockRecorder struct { - mock *MockTopicManager -} - -// NewMockTopicManager creates a new mock instance -func NewMockTopicManager(ctrl *gomock.Controller) *MockTopicManager { - mock := &MockTopicManager{ctrl: ctrl} - mock.recorder = &MockTopicManagerMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockTopicManager) EXPECT() *MockTopicManagerMockRecorder { - return m.recorder -} - -// EnsureTableExists mocks base method -func (m *MockTopicManager) EnsureTableExists(topic string, npar int) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "EnsureTableExists", topic, npar) - ret0, _ := ret[0].(error) - return ret0 -} - -// EnsureTableExists indicates an expected call of EnsureTableExists -func (mr *MockTopicManagerMockRecorder) EnsureTableExists(topic, npar interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EnsureTableExists", reflect.TypeOf((*MockTopicManager)(nil).EnsureTableExists), topic, npar) -} - -// EnsureStreamExists mocks base method -func (m *MockTopicManager) EnsureStreamExists(topic string, npar int) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "EnsureStreamExists", topic, npar) - ret0, _ := ret[0].(error) - return ret0 -} - -// EnsureStreamExists indicates an expected call of EnsureStreamExists -func (mr *MockTopicManagerMockRecorder) EnsureStreamExists(topic, npar interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EnsureStreamExists", reflect.TypeOf((*MockTopicManager)(nil).EnsureStreamExists), topic, npar) -} - -// EnsureTopicExists mocks base method -func (m *MockTopicManager) EnsureTopicExists(topic string, npar, rfactor int, config map[string]string) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "EnsureTopicExists", topic, npar, rfactor, config) - ret0, _ := ret[0].(error) - return ret0 -} - -// EnsureTopicExists indicates an expected call of EnsureTopicExists -func (mr *MockTopicManagerMockRecorder) EnsureTopicExists(topic, npar, rfactor, config interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EnsureTopicExists", reflect.TypeOf((*MockTopicManager)(nil).EnsureTopicExists), topic, npar, rfactor, config) -} - -// Partitions mocks base method -func (m *MockTopicManager) Partitions(topic string) ([]int32, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Partitions", topic) - ret0, _ := ret[0].([]int32) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// Partitions indicates an expected call of Partitions -func (mr *MockTopicManagerMockRecorder) Partitions(topic interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Partitions", reflect.TypeOf((*MockTopicManager)(nil).Partitions), topic) -} - -// Close mocks base method -func (m *MockTopicManager) 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 *MockTopicManagerMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockTopicManager)(nil).Close)) -} - -// Mockkzoo is a mock of kzoo interface -type Mockkzoo struct { - ctrl *gomock.Controller - recorder *MockkzooMockRecorder -} - -// MockkzooMockRecorder is the mock recorder for Mockkzoo -type MockkzooMockRecorder struct { - mock *Mockkzoo -} - -// NewMockkzoo creates a new mock instance -func NewMockkzoo(ctrl *gomock.Controller) *Mockkzoo { - mock := &Mockkzoo{ctrl: ctrl} - mock.recorder = &MockkzooMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *Mockkzoo) EXPECT() *MockkzooMockRecorder { - return m.recorder -} - -// Topic mocks base method -func (m *Mockkzoo) Topic(topic string) *kazoo_go.Topic { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Topic", topic) - ret0, _ := ret[0].(*kazoo_go.Topic) - return ret0 -} - -// Topic indicates an expected call of Topic -func (mr *MockkzooMockRecorder) Topic(topic interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Topic", reflect.TypeOf((*Mockkzoo)(nil).Topic), topic) -} - -// Topics mocks base method -func (m *Mockkzoo) Topics() (kazoo_go.TopicList, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Topics") - ret0, _ := ret[0].(kazoo_go.TopicList) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// Topics indicates an expected call of Topics -func (mr *MockkzooMockRecorder) Topics() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Topics", reflect.TypeOf((*Mockkzoo)(nil).Topics)) -} - -// CreateTopic mocks base method -func (m *Mockkzoo) CreateTopic(topic string, npar, rep int, config map[string]string) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "CreateTopic", topic, npar, rep, config) - ret0, _ := ret[0].(error) - return ret0 -} - -// CreateTopic indicates an expected call of CreateTopic -func (mr *MockkzooMockRecorder) CreateTopic(topic, npar, rep, config interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateTopic", reflect.TypeOf((*Mockkzoo)(nil).CreateTopic), topic, npar, rep, config) -} - -// Close mocks base method -func (m *Mockkzoo) 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 *MockkzooMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*Mockkzoo)(nil).Close)) -} diff --git a/kafka/simple_consumer.go b/kafka/simple_consumer.go deleted file mode 100644 index 04deb513..00000000 --- a/kafka/simple_consumer.go +++ /dev/null @@ -1,253 +0,0 @@ -package kafka - -import ( - "fmt" - "sync" - - "github.com/Shopify/sarama" -) - -type topicPartition struct { - topic string - partition int32 -} - -type simpleConsumer struct { - client sarama.Client - consumer sarama.Consumer - partitions map[topicPartition]sarama.PartitionConsumer - m sync.Mutex - - events chan Event - dying chan bool - - wg sync.WaitGroup -} - -func newSimpleConsumer(brokers []string, events chan Event, config *sarama.Config) (*simpleConsumer, error) { - client, err := sarama.NewClient(brokers, config) - if err != nil { - return nil, fmt.Errorf("Cannot connect to kafka: %v", err) - } - - consumer, err := sarama.NewConsumerFromClient(client) - if err != nil { - return nil, fmt.Errorf("Cannot create consumer: %v", err) - } - - return &simpleConsumer{ - client: client, - consumer: consumer, - events: events, - dying: make(chan bool), - partitions: make(map[topicPartition]sarama.PartitionConsumer), - }, nil -} - -func (c *simpleConsumer) Close() error { - // stop any blocking writes to channels - close(c.dying) - - c.m.Lock() - defer c.m.Unlock() - for tp, pc := range c.partitions { - pc.AsyncClose() - delete(c.partitions, tp) - } - - // wait until all partition consumers have finished - c.wg.Wait() - - if err := c.consumer.Close(); err != nil { - return fmt.Errorf("Failed to close consumer: %v", err) - } - - if err := c.client.Close(); err != nil { - return fmt.Errorf("Failed to close client in consumer: %v", err) - } - return nil -} - -func (c *simpleConsumer) AddPartition(topic string, partition int32, offset int64) error { - c.m.Lock() - defer c.m.Unlock() - tp := topicPartition{topic, partition} - if _, has := c.partitions[tp]; has { - return fmt.Errorf("%s/%d already added", topic, partition) - } - - // find best offset - start, hwm, err := c.getOffsets(topic, partition, offset) - if err != nil { - return fmt.Errorf("error getting offsets %s/%d: %v", topic, partition, err) - } - - pc, err := c.consumer.ConsumePartition(topic, partition, start) - if err != nil { - return fmt.Errorf("error creating consumer for %s/%d: %v", topic, partition, err) - } - c.partitions[tp] = pc - - c.wg.Add(1) - go func() { - defer c.wg.Done() - defer func() { - if err := recover(); err != nil { - c.events <- &Error{ - Err: fmt.Errorf("panic: %v", err), - } - } - }() - c.run(pc, topic, partition, start, hwm) - }() - return nil -} - -func (c *simpleConsumer) run(pc sarama.PartitionConsumer, topic string, partition int32, start, hwm int64) { - // mark beginning of partition consumption - select { - case c.events <- &BOF{ - Topic: topic, - Partition: partition, - Offset: start, - Hwm: hwm, - }: - case <-c.dying: - return - } - - // generate EOF if nothing to consume - if start == hwm { - select { - case c.events <- &EOF{ - Topic: topic, - Partition: partition, - Hwm: hwm, - }: - case <-c.dying: - return - } - } - - count := 0 - // wait for messages to arrive - for { - select { - case m, ok := <-pc.Messages(): - if !ok { - // Partition was removed. Continue to loop until errors are also - // drained. - continue - } - - headers := make(map[string][]byte) - for _, header := range m.Headers { - headers[string(header.Key)] = header.Value - } - - select { - case c.events <- &Message{ - Topic: m.Topic, - Partition: m.Partition, - Offset: m.Offset, - Key: string(m.Key), - Value: m.Value, - Timestamp: m.Timestamp, - Header: headers, - }: - case <-c.dying: - return - } - - if m.Offset == pc.HighWaterMarkOffset()-1 { - select { - case c.events <- &EOF{ - Topic: m.Topic, - Partition: m.Partition, - Hwm: m.Offset + 1, - }: - case <-c.dying: - return - } - } - - count++ - if count%1000 == 0 && m.Offset >= hwm { // was this EOF? - select { - case c.events <- &EOF{ - Topic: m.Topic, - Partition: m.Partition, - Hwm: pc.HighWaterMarkOffset(), - }: - case <-c.dying: - return - } - } - case err, ok := <-pc.Errors(): - if !ok { - // Partition was removed. - return - } - select { - case c.events <- &Error{ - Err: err, - }: - case <-c.dying: - return - } - return - case <-c.dying: - // Only closed when simple_consumer was closed, not when partitions are removed. - return - } - } -} - -func (c *simpleConsumer) RemovePartition(topic string, partition int32) error { - tp := topicPartition{topic, partition} - c.m.Lock() - defer c.m.Unlock() - pc, has := c.partitions[tp] - if !has { - return fmt.Errorf("%s/%d was not added", topic, partition) - } - delete(c.partitions, tp) - - if err := pc.Close(); err != nil { - return fmt.Errorf("error closing consumer for %s/%d: %v", topic, partition, err) - } - - return nil -} - -func (c *simpleConsumer) getOffsets(topic string, partition int32, offset int64) (start, hwm int64, err error) { - // check if there is anything to consume in topic/partition - oldest, err := c.client.GetOffset(topic, partition, sarama.OffsetOldest) - if err != nil { - err = fmt.Errorf("Error reading oldest log offset from kafka: %v", err) - return - } - - // get HighWaterMark - hwm, err = c.client.GetOffset(topic, partition, sarama.OffsetNewest) - if err != nil { - err = fmt.Errorf("Error reading current log offset from kafka: %v", err) - return - } - - start = offset - - if offset == sarama.OffsetOldest { - start = oldest - } else if offset == sarama.OffsetNewest { - start = hwm - } - - if start > hwm { - start = hwm - } - if start < oldest { - start = oldest - } - return -} diff --git a/kafka/simple_consumer_test.go b/kafka/simple_consumer_test.go deleted file mode 100644 index 529e3a0e..00000000 --- a/kafka/simple_consumer_test.go +++ /dev/null @@ -1,313 +0,0 @@ -//go:generate mockgen -package mock -destination=mock/sarama.go github.com/Shopify/sarama Client,Consumer,PartitionConsumer - -package kafka - -import ( - "errors" - "testing" - - "github.com/lovoo/goka/kafka/mock" - - "github.com/Shopify/sarama" - "github.com/facebookgo/ensure" - "github.com/golang/mock/gomock" -) - -func TestSimpleConsumer_GetOffsets(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - client = mock.NewMockClient(ctrl) - topic = "topic" - partition int32 = 123 - offset int64 = 1234 - oldest int64 = 1233 - newest int64 = 1237 - start int64 - hwm int64 - err error - ) - - c := &simpleConsumer{ - client: client, - } - - // errors in GetOffset - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, errors.New("some error")) - _, _, err = c.getOffsets(topic, partition, offset) - ensure.NotNil(t, err) - - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(newest, errors.New("some error")) - _, _, err = c.getOffsets(topic, partition, offset) - ensure.NotNil(t, err) - - // oldest < offset < newest - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(newest, nil) - start, hwm, err = c.getOffsets(topic, partition, offset) - ensure.Nil(t, err) - ensure.True(t, start == offset) - ensure.True(t, hwm == newest) - - // offset < oldest < newest - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(newest, nil) - start, hwm, err = c.getOffsets(topic, partition, oldest-1) - ensure.Nil(t, err) - ensure.True(t, start == oldest) - ensure.True(t, hwm == newest) - - // oldest < newest < offset - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(newest, nil) - start, hwm, err = c.getOffsets(topic, partition, newest+1) - ensure.Nil(t, err) - ensure.True(t, start == newest) - ensure.True(t, hwm == newest) - - // sarama.OffsetOldest - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(newest, nil) - start, hwm, err = c.getOffsets(topic, partition, sarama.OffsetOldest) - ensure.Nil(t, err) - ensure.True(t, start == oldest) - ensure.True(t, hwm == newest) - - // sarama.OffsetNewest - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(newest, nil) - start, hwm, err = c.getOffsets(topic, partition, sarama.OffsetNewest) - ensure.Nil(t, err) - ensure.True(t, start == newest) - ensure.True(t, hwm == newest) -} - -func TestSimpleConsumer_AddPartition(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - client = mock.NewMockClient(ctrl) - consumer = mock.NewMockConsumer(ctrl) - pc = mock.NewMockPartitionConsumer(ctrl) - topic = "topic" - partition int32 = 123 - offset int64 = 1234 - oldest int64 = 1233 - hwm int64 = 1237 - messages = make(chan *sarama.ConsumerMessage) - ch = make(chan Event) - cherr = make(chan *sarama.ConsumerError) - ) - - c := &simpleConsumer{ - client: client, - consumer: consumer, - partitions: make(map[topicPartition]sarama.PartitionConsumer), - events: ch, - } - - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, errors.New("some error")) - err := c.AddPartition(topic, partition, offset) - ensure.NotNil(t, err) - - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(hwm, nil) - consumer.EXPECT().ConsumePartition(topic, partition, offset).Return(nil, errors.New("some error")) - err = c.AddPartition(topic, partition, offset) - ensure.NotNil(t, err) - - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(hwm, nil) - consumer.EXPECT().ConsumePartition(topic, partition, offset).Return(pc, nil) - pc.EXPECT().Messages().Return(messages).AnyTimes() - pc.EXPECT().Errors().Return(cherr).AnyTimes() - err = c.AddPartition(topic, partition, offset) - ensure.Nil(t, err) - m, ok := (<-ch).(*BOF) - ensure.True(t, ok) - ensure.DeepEqual(t, m, &BOF{Topic: topic, Partition: partition, Offset: offset, Hwm: hwm}) - - err = c.AddPartition(topic, partition, offset) - ensure.NotNil(t, err) - - doTimed(t, func() { - close(messages) - pc.EXPECT().AsyncClose() - consumer.EXPECT().Close().Return(errors.New("some error")) - c.dying = make(chan bool) - err = c.Close() - ensure.NotNil(t, err) - - consumer.EXPECT().Close().Return(nil) - client.EXPECT().Close().Return(errors.New("some error")) - c.dying = make(chan bool) - err = c.Close() - ensure.NotNil(t, err) - - consumer.EXPECT().Close().Return(nil) - client.EXPECT().Close().Return(nil) - c.dying = make(chan bool) - err = c.Close() - ensure.Nil(t, err) - }) -} - -func TestSimpleConsumer_RemovePartition(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - client = mock.NewMockClient(ctrl) - consumer = mock.NewMockConsumer(ctrl) - pc = mock.NewMockPartitionConsumer(ctrl) - topic = "topic" - partition int32 = 123 - offset int64 = 1234 - oldest int64 = 1233 - hwm int64 = 1237 - messages = make(chan *sarama.ConsumerMessage) - ch = make(chan Event) - cherr = make(chan *sarama.ConsumerError) - ) - - c := &simpleConsumer{ - client: client, - consumer: consumer, - partitions: make(map[topicPartition]sarama.PartitionConsumer), - events: ch, - } - - err := c.RemovePartition(topic, partition) - ensure.NotNil(t, err) - - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(hwm, nil) - consumer.EXPECT().ConsumePartition(topic, partition, offset).Return(pc, nil) - pc.EXPECT().Messages().Return(messages).AnyTimes() - pc.EXPECT().Errors().Return(cherr).AnyTimes() - err = c.AddPartition(topic, partition, offset) - ensure.Nil(t, err) - - m, ok := (<-ch).(*BOF) - ensure.True(t, ok) - ensure.DeepEqual(t, m, &BOF{Topic: topic, Partition: partition, Offset: offset, Hwm: hwm}) - - ensure.DeepEqual(t, c.partitions, map[topicPartition]sarama.PartitionConsumer{ - topicPartition{topic, partition}: pc, - }) - - // add partition again - err = c.AddPartition(topic, partition, offset) - ensure.NotNil(t, err) - - pc.EXPECT().Close() - err = c.RemovePartition(topic, partition) - ensure.Nil(t, err) - - doTimed(t, func() { - close(messages) - consumer.EXPECT().Close().Return(nil) - client.EXPECT().Close().Return(nil) - c.dying = make(chan bool) - err = c.Close() - ensure.Nil(t, err) - }) -} - -func TestSimpleConsumer_ErrorBlocked(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - client = mock.NewMockClient(ctrl) - consumer = mock.NewMockConsumer(ctrl) - pc = mock.NewMockPartitionConsumer(ctrl) - topic = "topic" - partition int32 = 123 - offset int64 = 1234 - oldest int64 = 1233 - hwm int64 = 1237 - messages = make(chan *sarama.ConsumerMessage) - ch = make(chan Event) - cherr = make(chan *sarama.ConsumerError) - ) - - c := &simpleConsumer{ - client: client, - consumer: consumer, - partitions: make(map[topicPartition]sarama.PartitionConsumer), - events: ch, - dying: make(chan bool), - } - - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, errors.New("some error")) - err := c.AddPartition(topic, partition, offset) - ensure.NotNil(t, err) - - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(hwm, nil) - consumer.EXPECT().ConsumePartition(topic, partition, offset).Return(nil, errors.New("some error")) - err = c.AddPartition(topic, partition, offset) - ensure.NotNil(t, err) - - client.EXPECT().GetOffset(topic, partition, sarama.OffsetOldest).Return(oldest, nil) - client.EXPECT().GetOffset(topic, partition, sarama.OffsetNewest).Return(hwm, nil) - consumer.EXPECT().ConsumePartition(topic, partition, offset).Return(pc, nil) - pc.EXPECT().Messages().Return(messages).AnyTimes() - pc.EXPECT().Errors().Return(cherr).AnyTimes() - err = c.AddPartition(topic, partition, offset) - ensure.Nil(t, err) - m, ok := (<-ch).(*BOF) - ensure.True(t, ok) - ensure.DeepEqual(t, m, &BOF{Topic: topic, Partition: partition, Offset: offset, Hwm: hwm}) - - err = c.AddPartition(topic, partition, offset) - ensure.NotNil(t, err) - - messages <- &sarama.ConsumerMessage{ - Key: []byte("somekey"), - Value: []byte("somevalue"), - Topic: "sometopic", - Partition: 123, - } - pc.EXPECT().HighWaterMarkOffset().Return(int64(0)) - mo, ok := (<-ch).(*Message) - ensure.True(t, ok) - ensure.DeepEqual(t, mo, &Message{Topic: "sometopic", Partition: 123, Key: "somekey", Value: []byte("somevalue"), Header: map[string][]byte{}}) - - messages <- &sarama.ConsumerMessage{ - Key: []byte("somekey"), - Value: []byte("somevalue"), - Topic: "sometopic", - Partition: 123, - Headers: []*sarama.RecordHeader{{ - Key: []byte("someHeader"), - Value: []byte("someHeaderValue"), - }}, - } - - pc.EXPECT().HighWaterMarkOffset().Return(int64(0)) - mo1, ok := (<-ch).(*Message) - ensure.True(t, ok) - ensure.DeepEqual(t, mo1, &Message{Topic: "sometopic", Partition: 123, Key: "somekey", Value: []byte("somevalue"), Header: map[string][]byte{"someHeader": []byte("someHeaderValue")}}) - - // we now write, but don't read events - messages <- &sarama.ConsumerMessage{ - Key: []byte("somekey"), - Value: []byte("somevalue"), - Topic: "sometopic", - Partition: 123, - } - - err = doTimed(t, func() { - pc.EXPECT().AsyncClose() - consumer.EXPECT().Close().Return(nil) - client.EXPECT().Close().Return(nil) - err := c.Close() - ensure.Nil(t, err) - }) - ensure.Nil(t, err) -} diff --git a/kafka/topic_manager.go b/kafka/topic_manager.go deleted file mode 100644 index 6e023f99..00000000 --- a/kafka/topic_manager.go +++ /dev/null @@ -1,292 +0,0 @@ -package kafka - -import ( - "fmt" - "strconv" - "strings" - "time" - - "github.com/Shopify/sarama" - kazoo "github.com/wvanbergen/kazoo-go" -) - -// TopicManager provides an interface to create/check topics and their partitions -type TopicManager interface { - // EnsureTableExists checks that a table (log-compacted topic) exists, or create one if possible - EnsureTableExists(topic string, npar int) error - // EnsureStreamExists checks that a stream topic exists, or create one if possible - EnsureStreamExists(topic string, npar int) error - // EnsureTopicExists checks that a topic exists, or create one if possible, - // enforcing the given configuration - EnsureTopicExists(topic string, npar, rfactor int, config map[string]string) error - - // Partitions returns the number of partitions of a topic, that are assigned to the running - // instance, i.e. it doesn't represent all partitions of a topic. - Partitions(topic string) ([]int32, error) - - // Close closes the topic manager - Close() error -} - -type saramaTopicManager struct { - brokers []string - client sarama.Client -} - -// NewSaramaTopicManager creates a new topic manager using the sarama library -func NewSaramaTopicManager(brokers []string, config *sarama.Config) (TopicManager, error) { - client, err := sarama.NewClient(brokers, config) - if err != nil { - return nil, fmt.Errorf("Error creating the kafka client: %v", err) - } - - return &saramaTopicManager{ - brokers: brokers, - client: client, - }, nil -} - -func (m *saramaTopicManager) Close() error { - return m.client.Close() -} - -func (m *saramaTopicManager) Partitions(topic string) ([]int32, error) { - return m.client.Partitions(topic) -} - -func (m *saramaTopicManager) EnsureStreamExists(topic string, npar int) error { - return m.EnsureTableExists(topic, npar) -} - -func (m *saramaTopicManager) EnsureTableExists(topic string, npar int) error { - par, err := m.client.Partitions(topic) - if err != nil { - return fmt.Errorf("could not ensure %s exists: %v", topic, err) - } - if len(par) != npar { - return fmt.Errorf("topic %s has %d partitions instead of %d", topic, len(par), npar) - } - return nil -} - -func (m *saramaTopicManager) EnsureTopicExists(topic string, npar, rfactor int, config map[string]string) error { - return fmt.Errorf("not implemented in SaramaTopicManager") -} - -// TopicManagerConfig contains the configuration to access the Zookeeper servers -// as well as the desired options of to create tables and stream topics. -type TopicManagerConfig struct { - Table struct { - Replication int - } - Stream struct { - Replication int - Retention time.Duration - } -} - -type topicManager struct { - zk kzoo - servers []string - config *TopicManagerConfig -} - -// NewTopicManagerConfig provides a default configuration for auto-creation -// with replication factor of 1 and rentention time of 1 hour. -func NewTopicManagerConfig() *TopicManagerConfig { - cfg := new(TopicManagerConfig) - cfg.Table.Replication = 2 - cfg.Stream.Replication = 2 - cfg.Stream.Retention = 1 * time.Hour - return cfg -} - -// NewTopicManager creates a new topic manager for managing topics with zookeeper -func NewTopicManager(servers []string, config *TopicManagerConfig) (TopicManager, error) { - servers, chroot, err := updateChroot(servers) - if err != nil { - return nil, err - } - - cfg := kazoo.NewConfig() - cfg.Chroot = chroot - - if config == nil { - config = NewTopicManagerConfig() - } - - kzoo, err := kazoo.NewKazoo(servers, cfg) - if err != nil { - return nil, fmt.Errorf("could not connect to ZooKeeper: %v", err) - } - - return &topicManager{ - zk: kzoo, - config: config, - }, nil -} - -func (m *topicManager) Close() error { - return m.zk.Close() -} - -func (m *topicManager) EnsureTableExists(topic string, npar int) error { - err := checkTopic( - m.zk, topic, npar, - m.config.Table.Replication, - map[string]string{"cleanup.policy": "compact"}, - false, - ) - if err != nil { - return err - } - // check number of partitions - return m.checkPartitions(topic, npar) -} - -func (m *topicManager) EnsureStreamExists(topic string, npar int) error { - retention := int(m.config.Stream.Retention.Nanoseconds() / time.Millisecond.Nanoseconds()) - err := checkTopic( - m.zk, topic, npar, - m.config.Stream.Replication, - map[string]string{"retention.ms": strconv.Itoa(retention)}, - false, - ) - if err != nil { - return err - } - return m.checkPartitions(topic, npar) -} - -func (m *topicManager) EnsureTopicExists(topic string, npar, rfactor int, config map[string]string) error { - if err := checkTopic(m.zk, topic, npar, rfactor, config, true); err != nil { - return err - } - return m.checkPartitions(topic, npar) -} - -func (m *topicManager) Partitions(topic string) ([]int32, error) { - tl, err := m.zk.Topics() - if err != nil { - return nil, err - } - t := tl.Find(topic) - if t == nil { - return nil, nil - } - - pl, err := t.Partitions() - if err != nil { - return nil, err - } - var partitions []int32 - for _, p := range pl { - partitions = append(partitions, p.ID) - } - return partitions, nil -} - -// ensure topic exists -func checkTopic(kz kzoo, topic string, npar int, rfactor int, cfg map[string]string, ensureConfig bool) error { - ok, err := hasTopic(kz, topic) - if err != nil { - return err - } - if !ok { - err = kz.CreateTopic(topic, npar, rfactor, cfg) - if err != nil { - return err - } - } - if !ensureConfig { - return nil - } - // topic exists, check if config the same - c, err := kz.Topic(topic).Config() - if err != nil { - return err - } - for k, v := range cfg { - if c[k] != v { - return fmt.Errorf("expected %s=%s, but found %s", k, cfg[k], c[k]) - } - } - return nil -} - -// returns true if topic exists, false otherwise -func hasTopic(kz kzoo, topic string) (bool, error) { - topics, err := kz.Topics() - if err != nil { - return false, err - } - for _, t := range topics { - if t.Name == topic { - return true, nil - } - } - return false, nil -} - -// check that the number of paritions match npar using kazoo library -func (m *topicManager) checkPartitions(topic string, npar int) error { - t := m.zk.Topic(topic) - - partitions, err := t.Partitions() - if err != nil { - return fmt.Errorf("Error fetching partitions for topic %s: %v", topic, err) - } - if len(partitions) != npar { - return fmt.Errorf("Topic %s does not have %d partitions", topic, npar) - } - return nil -} - -// check that the number of paritions match -func checkPartitions(client sarama.Client, topic string, npar int) error { - // check if topic has npar partitions - partitions, err := client.Partitions(topic) - if err != nil { - return fmt.Errorf("Error fetching partitions for topic %s: %v", topic, err) - } - if len(partitions) != npar { - return fmt.Errorf("Topic %s has %d partitions instead of %d", topic, len(partitions), npar) - } - return nil -} - -func updateChroot(servers []string) (servs []string, chroot string, err error) { - // find chroot in server addresses - for _, server := range servers { - for strings.HasSuffix(server, "/") { - server = server[:len(server)-1] - } - splt := strings.Split(server, "/") - if len(splt) == 1 { - // no chroot in address - servs = append(servs, server) - continue - } - if len(splt) > 2 { - err = fmt.Errorf("Could not parse %s properly", server) - return - } - servs = append(servs, splt[0]) - c := fmt.Sprintf("/%s", splt[1]) - if chroot == "" { - chroot = c - } else if c != chroot { - err = fmt.Errorf("Multiple chroot set (%s != %s)", c, chroot) - return - } - } - return -} - -//go:generate mockgen -package mock -destination mock/kazoo.go -source=topic_manager.go kzoo -type kzoo interface { - Topic(topic string) *kazoo.Topic - Topics() (kazoo.TopicList, error) - CreateTopic(topic string, npar int, rep int, config map[string]string) error - Close() error -} diff --git a/kafka/topic_manager_test.go b/kafka/topic_manager_test.go deleted file mode 100644 index a2a838dc..00000000 --- a/kafka/topic_manager_test.go +++ /dev/null @@ -1,160 +0,0 @@ -package kafka - -import ( - "fmt" - "testing" - - "github.com/lovoo/goka/kafka/mock" - - "github.com/facebookgo/ensure" - "github.com/golang/mock/gomock" - kazoo "github.com/wvanbergen/kazoo-go" -) - -func TestTopicManager_updateChroot(t *testing.T) { - chroot := "/test" - host := "somehost.com" - chroot2 := "/test2" - host2 := "somehost2.com" - - // chroot in one server - servers := []string{host + chroot} - s, c, err := updateChroot(servers) - ensure.Nil(t, err) - ensure.DeepEqual(t, c, chroot) - ensure.DeepEqual(t, s[0], host) - - // chroot in one out of multiple servers - servers = []string{host + chroot, host2} - s, c, err = updateChroot(servers) - ensure.Nil(t, err) - ensure.DeepEqual(t, c, chroot) - ensure.DeepEqual(t, s[0], host) - ensure.DeepEqual(t, s[1], host2) - - // chroot in multiple servers - servers = []string{host + chroot, host2 + chroot} - s, c, err = updateChroot(servers) - ensure.Nil(t, err) - ensure.DeepEqual(t, c, chroot) - ensure.DeepEqual(t, s[0], host) - ensure.DeepEqual(t, s[1], host2) - - // chroot in multiple servers but different - servers = []string{host + chroot, host2 + chroot2} - _, _, err = updateChroot(servers) - ensure.NotNil(t, err) - - // check chroot in servers but parse fails - servers = []string{host + chroot, host2 + "/hi/whatever"} - _, _, err = updateChroot(servers) - ensure.NotNil(t, err) - - // check chroot with trailing / - servers = []string{host + "/////"} - _, c, err = updateChroot(servers) - ensure.DeepEqual(t, c, "") - ensure.Nil(t, err) - - servers = []string{host + "/test/"} - _, c, err = updateChroot(servers) - ensure.DeepEqual(t, c, "/test") - ensure.Nil(t, err) - -} - -func TestCheckPartitions(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - client := mock.NewMockClient(ctrl) - - topic := "topic" - npar := 3 - - client.EXPECT().Partitions(topic).Return([]int32{0, 1, 2}, nil) - err := checkPartitions(client, topic, npar) - fmt.Println(err) - ensure.Nil(t, err) - - client.EXPECT().Partitions(topic).Return([]int32{0, 1}, nil) - err = checkPartitions(client, topic, npar) - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "partitions instead") - - client.EXPECT().Partitions(topic).Return([]int32{0, 1, 2}, fmt.Errorf("some error in the wire")) - err = checkPartitions(client, topic, npar) - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "Error fetching") -} - -func TestTopicManager_hasTopic(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - kz := mock.NewMockkzoo(ctrl) - - topic := "test" - - kz.EXPECT().Topics().Return(kazoo.TopicList{&kazoo.Topic{Name: topic}}, nil) - ok, err := hasTopic(kz, topic) - ensure.Nil(t, err) - ensure.True(t, ok) - - kz.EXPECT().Topics().Return(kazoo.TopicList{&kazoo.Topic{Name: "other"}}, nil) - ok, err = hasTopic(kz, topic) - ensure.Nil(t, err) - ensure.False(t, ok) - - kz.EXPECT().Topics().Return(kazoo.TopicList{&kazoo.Topic{Name: topic}}, fmt.Errorf("some error")) - _, err = hasTopic(kz, topic) - ensure.NotNil(t, err) -} - -func TestCheckTopic(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - kz := mock.NewMockkzoo(ctrl) - - topic := "test" - - kz.EXPECT().Topics().Return(kazoo.TopicList{&kazoo.Topic{Name: topic}}, nil) - ok, err := hasTopic(kz, topic) - ensure.Nil(t, err) - ensure.True(t, ok) - - kz.EXPECT().Topics().Return(kazoo.TopicList{&kazoo.Topic{Name: "other"}}, nil) - ok, err = hasTopic(kz, topic) - ensure.Nil(t, err) - ensure.False(t, ok) - - kz.EXPECT().Topics().Return(kazoo.TopicList{&kazoo.Topic{Name: topic}}, fmt.Errorf("some error")) - _, err = hasTopic(kz, topic) - ensure.NotNil(t, err) -} - -/* cannot fix this yet -type topicMock struct { - *kazoo.Topic -} - -func (t *topicMock) Partitions() (kazoo.PartitionList, error) { - return kazoo.PartitionList{}, nil -} -func TestTopicManager_checkPartitions(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - kz := mock_kazoo.NewMockKazoo(ctrl) - topic := "test" - npar := 3 - ac := &topicManager{ - servers: []string{"somehost"}, - config: NewTopicManagerConfig(), - } - - gomock.InOrder( - kz.EXPECT().Topic(topic).Return(&topicMock{&kazoo.Topic{Name: topic}}), - ) - err := ac.checkPartitions(kz, topic, npar) - ensure.Nil(t, err) -} -*/ diff --git a/logger/logger.go b/logger/logger.go index aa6ac7b4..97815957 100644 --- a/logger/logger.go +++ b/logger/logger.go @@ -1,6 +1,12 @@ package logger -import "log" +import ( + "fmt" + "log" + "strings" + + "github.com/Shopify/sarama" +) var ( defaultLogger = &std{} @@ -8,27 +14,112 @@ var ( // Logger is the interface Goka and its subpackages use for logging. type Logger interface { + + // Print will simply print the params + Print(...interface{}) + + // Print will simply print the params + Println(...interface{}) + // Printf will be used for informational messages. These can be thought of // having an 'Info'-level in a structured logger. Printf(string, ...interface{}) + + // Debugf is used for debugging messages, mostly for debugging goka itself. + // It is turned off unless goka is initialized + Debugf(string, ...interface{}) + // Panicf will be only called an unexpected programming error such as a type // assertion which should never fail. Regular errors will be returned out // from the library. Panicf(string, ...interface{}) + + // PrefixedLogger returns a logger that prefixes all messages with passed prefix + Prefix(string) Logger } // std bridges the logger calls to the standard library log. -type std struct{} +type std struct { + debug bool + prefixPath []string + prefix string +} + +func (s *std) Print(msgs ...interface{}) { + log.Print(msgs...) +} +func (s *std) Println(msgs ...interface{}) { + log.Print(msgs...) +} func (s *std) Printf(msg string, args ...interface{}) { - log.Printf(msg, args...) + log.Printf(fmt.Sprintf("%s%s", s.prefix, msg), args...) +} + +func (s *std) Debugf(msg string, args ...interface{}) { + if s.debug { + log.Printf(fmt.Sprintf("%s%s", s.prefix, msg), args...) + } } func (s *std) Panicf(msg string, args ...interface{}) { - log.Panicf(msg, args...) + log.Panicf(fmt.Sprintf("%s%s", s.prefix, msg), args...) +} + +func (s *std) Prefix(prefix string) Logger { + return s.StackPrefix(prefix).(*std) } // Default returns the standard library logger func Default() Logger { return defaultLogger } + +// Debug enables or disables debug logging using the global logger. +func Debug(gokaDebug, saramaDebug bool) { + defaultLogger.debug = gokaDebug + if saramaDebug { + SetSaramaLogger(&std{debug: true}) + } +} + +func SetSaramaLogger(logger Logger) { + sarama.Logger = logger +} + +// EmptyPrefixer encapsulates a prefixer that is initially without a prefix +func EmptyPrefixer() Prefixer { + return &std{} +} + +// Prefixer abstracts the functionality of stacking the prefix for a custom logger implementation +type Prefixer interface { + CurrentPrefix() string + StackPrefix(prefix string) Prefixer +} + +func (s *std) CurrentPrefix() string { + return s.prefix +} +func (s *std) StackPrefix(prefix string) Prefixer { + var prefPath []string + // append existing path + prefPath = append(prefPath, s.prefixPath...) + + // if new is not empty, append to path + if prefix != "" { + prefPath = append(prefPath, prefix) + } + + // make new prefix + newPrefix := strings.Join(prefPath, " > ") + if newPrefix != "" { + newPrefix = "[" + newPrefix + "] " + } + + return &std{ + prefixPath: prefPath, + prefix: newPrefix, + debug: s.debug, + } +} diff --git a/mock/proxy.go b/mock/proxy.go deleted file mode 100644 index 7b83e24d..00000000 --- a/mock/proxy.go +++ /dev/null @@ -1,85 +0,0 @@ -// Code generated by MockGen. DO NOT EDIT. -// Source: partition.go - -// Package mock is a generated GoMock package. -package mock - -import ( - gomock "github.com/golang/mock/gomock" - reflect "reflect" -) - -// MockkafkaProxy is a mock of kafkaProxy interface -type MockkafkaProxy struct { - ctrl *gomock.Controller - recorder *MockkafkaProxyMockRecorder -} - -// MockkafkaProxyMockRecorder is the mock recorder for MockkafkaProxy -type MockkafkaProxyMockRecorder struct { - mock *MockkafkaProxy -} - -// NewMockkafkaProxy creates a new mock instance -func NewMockkafkaProxy(ctrl *gomock.Controller) *MockkafkaProxy { - mock := &MockkafkaProxy{ctrl: ctrl} - mock.recorder = &MockkafkaProxyMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockkafkaProxy) EXPECT() *MockkafkaProxyMockRecorder { - return m.recorder -} - -// Add mocks base method -func (m *MockkafkaProxy) Add(arg0 string, arg1 int64) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Add", arg0, arg1) - ret0, _ := ret[0].(error) - return ret0 -} - -// Add indicates an expected call of Add -func (mr *MockkafkaProxyMockRecorder) Add(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Add", reflect.TypeOf((*MockkafkaProxy)(nil).Add), arg0, arg1) -} - -// Remove mocks base method -func (m *MockkafkaProxy) Remove(arg0 string) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Remove", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// Remove indicates an expected call of Remove -func (mr *MockkafkaProxyMockRecorder) Remove(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Remove", reflect.TypeOf((*MockkafkaProxy)(nil).Remove), arg0) -} - -// AddGroup mocks base method -func (m *MockkafkaProxy) AddGroup() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "AddGroup") -} - -// AddGroup indicates an expected call of AddGroup -func (mr *MockkafkaProxyMockRecorder) AddGroup() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddGroup", reflect.TypeOf((*MockkafkaProxy)(nil).AddGroup)) -} - -// Stop mocks base method -func (m *MockkafkaProxy) Stop() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Stop") -} - -// Stop indicates an expected call of Stop -func (mr *MockkafkaProxyMockRecorder) Stop() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stop", reflect.TypeOf((*MockkafkaProxy)(nil).Stop)) -} diff --git a/mockautoconsumers.go b/mockautoconsumers.go new file mode 100644 index 00000000..5aabfb60 --- /dev/null +++ b/mockautoconsumers.go @@ -0,0 +1,619 @@ +package goka + +import ( + "context" + "fmt" + "strings" + "sync" + "sync/atomic" + "testing" + + "github.com/Shopify/sarama" + "github.com/lovoo/goka/multierr" +) + +const ( + AnyOffset int64 = -1000 +) + +var ( + errOutOfExpectations = fmt.Errorf("error out of expectations") + errPartitionConsumerNotStarted = fmt.Errorf("error partition consumer not started") +) + +// MockAutoConsumer implements sarama's Consumer interface for testing purposes. +// Before you can start consuming from this consumer, you have to register +// topic/partitions using ExpectConsumePartition, and set expectations on them. +type MockAutoConsumer struct { + l sync.Mutex + t *testing.T + config *sarama.Config + partitionConsumers map[string]map[int32]*MockAutoPartitionConsumer + metadata map[string][]int32 +} + +// NewAutoMockAutoConsumer returns a new mock Consumer instance. The t argument should +// be the *testing.T instance of your test method. An error will be written to it if +// an expectation is violated. The config argument can be set to nil. +func NewMockAutoConsumer(t *testing.T, config *sarama.Config) *MockAutoConsumer { + if config == nil { + config = sarama.NewConfig() + } + + c := &MockAutoConsumer{ + t: t, + config: config, + partitionConsumers: make(map[string]map[int32]*MockAutoPartitionConsumer), + } + return c +} + +/////////////////////////////////////////////////// +// Consumer interface implementation +/////////////////////////////////////////////////// + +// ConsumePartition implements the ConsumePartition method from the sarama.Consumer interface. +// Before you can start consuming a partition, you have to set expectations on it using +// ExpectConsumePartition. You can only consume a partition once per consumer. +func (c *MockAutoConsumer) ConsumePartition(topic string, partition int32, offset int64) (sarama.PartitionConsumer, error) { + c.l.Lock() + defer c.l.Unlock() + + if c.partitionConsumers[topic] == nil || c.partitionConsumers[topic][partition] == nil { + c.t.Errorf("No expectations set for %s/%d", topic, partition) + return nil, errOutOfExpectations + } + + pc := c.partitionConsumers[topic][partition] + if pc.consumed { + return nil, sarama.ConfigurationError("The topic/partition is already being consumed") + } + + if pc.offset != AnyOffset && pc.offset != offset { + c.t.Errorf("Unexpected offset when calling ConsumePartition for %s/%d. Expected %d, got %d.", topic, partition, pc.offset, offset) + } + + pc.consumed = true + return pc, nil +} + +// Topics returns a list of topics, as registered with SetTopicMetadata +func (c *MockAutoConsumer) Topics() ([]string, error) { + c.l.Lock() + defer c.l.Unlock() + + if c.metadata == nil { + c.t.Errorf("Unexpected call to Topics. Initialize the mock's topic metadata with SetTopicMetadata.") + return nil, sarama.ErrOutOfBrokers + } + + var result []string + for topic := range c.metadata { + result = append(result, topic) + } + return result, nil +} + +// Partitions returns the list of parititons for the given topic, as registered with SetTopicMetadata +func (c *MockAutoConsumer) Partitions(topic string) ([]int32, error) { + c.l.Lock() + defer c.l.Unlock() + + if c.metadata == nil { + c.t.Errorf("Unexpected call to Partitions. Initialize the mock's topic metadata with SetTopicMetadata.") + return nil, sarama.ErrOutOfBrokers + } + if c.metadata[topic] == nil { + return nil, sarama.ErrUnknownTopicOrPartition + } + + return c.metadata[topic], nil +} + +func (c *MockAutoConsumer) HighWaterMarks() map[string]map[int32]int64 { + c.l.Lock() + defer c.l.Unlock() + + hwms := make(map[string]map[int32]int64, len(c.partitionConsumers)) + for topic, partitionConsumers := range c.partitionConsumers { + hwm := make(map[int32]int64, len(partitionConsumers)) + for partition, pc := range partitionConsumers { + hwm[partition] = pc.HighWaterMarkOffset() + } + hwms[topic] = hwm + } + + return hwms +} + +// Close implements the Close method from the sarama.Consumer interface. It will close +// all registered PartitionConsumer instances. +func (c *MockAutoConsumer) Close() error { + c.l.Lock() + defer c.l.Unlock() + + for _, partitions := range c.partitionConsumers { + for _, partitionConsumer := range partitions { + _ = partitionConsumer.Close() + } + } + + return nil +} + +/////////////////////////////////////////////////// +// Expectation API +/////////////////////////////////////////////////// + +// SetTopicMetadata sets the clusters topic/partition metadata, +// which will be returned by Topics() and Partitions(). +func (c *MockAutoConsumer) SetTopicMetadata(metadata map[string][]int32) { + c.l.Lock() + defer c.l.Unlock() + + c.metadata = metadata +} + +// ExpectConsumePartition will register a topic/partition, so you can set expectations on it. +// The registered PartitionConsumer will be returned, so you can set expectations +// on it using method chaining. Once a topic/partition is registered, you are +// expected to start consuming it using ConsumePartition. If that doesn't happen, +// an error will be written to the error reporter once the mock consumer is closed. It will +// also expect that the +func (c *MockAutoConsumer) ExpectConsumePartition(topic string, partition int32, offset int64) *MockAutoPartitionConsumer { + c.l.Lock() + defer c.l.Unlock() + + if c.partitionConsumers[topic] == nil { + c.partitionConsumers[topic] = make(map[int32]*MockAutoPartitionConsumer) + } + + if c.partitionConsumers[topic][partition] == nil { + c.partitionConsumers[topic][partition] = &MockAutoPartitionConsumer{ + t: c.t, + topic: topic, + partition: partition, + offset: offset, + messages: make(chan *sarama.ConsumerMessage, c.config.ChannelBufferSize), + errors: make(chan *sarama.ConsumerError, c.config.ChannelBufferSize), + } + } + + return c.partitionConsumers[topic][partition] +} + +/////////////////////////////////////////////////// +// PartitionConsumer mock type +/////////////////////////////////////////////////// + +// MockAutoPartitionConsumer implements sarama's PartitionConsumer interface for testing purposes. +// It is returned by the mock Consumers ConsumePartitionMethod, but only if it is +// registered first using the Consumer's ExpectConsumePartition method. Before consuming the +// Errors and Messages channel, you should specify what values will be provided on these +// channels using YieldMessage and YieldError. +type MockAutoPartitionConsumer struct { + highWaterMarkOffset int64 // must be at the top of the struct because https://golang.org/pkg/sync/atomic/#pkg-note-BUG + l sync.Mutex + t *testing.T + topic string + partition int32 + offset int64 + messages chan *sarama.ConsumerMessage + errors chan *sarama.ConsumerError + singleClose sync.Once + consumed bool + errorsShouldBeDrained bool + messagesShouldBeDrained bool +} + +/////////////////////////////////////////////////// +// PartitionConsumer interface implementation +/////////////////////////////////////////////////// + +// AsyncClose implements the AsyncClose method from the sarama.PartitionConsumer interface. +func (pc *MockAutoPartitionConsumer) AsyncClose() { + pc.singleClose.Do(func() { + close(pc.messages) + close(pc.errors) + pc.consumed = false + }) +} + +// Close implements the Close method from the sarama.PartitionConsumer interface. It will +// verify whether the partition consumer was actually started. +func (pc *MockAutoPartitionConsumer) Close() error { + if !pc.consumed { + pc.t.Errorf("Expectations set on %s/%d, but no partition consumer was started.", pc.topic, pc.partition) + return errPartitionConsumerNotStarted + } + + if pc.errorsShouldBeDrained && len(pc.errors) > 0 { + pc.t.Errorf("Expected the errors channel for %s/%d to be drained on close, but found %d errors.", pc.topic, pc.partition, len(pc.errors)) + } + + if pc.messagesShouldBeDrained && len(pc.messages) > 0 { + pc.t.Errorf("Expected the messages channel for %s/%d to be drained on close, but found %d messages.", pc.topic, pc.partition, len(pc.messages)) + } + + pc.AsyncClose() + + var ( + closeErr error + wg sync.WaitGroup + ) + + wg.Add(1) + go func() { + defer wg.Done() + + var errs = make(sarama.ConsumerErrors, 0) + for err := range pc.errors { + errs = append(errs, err) + } + + if len(errs) > 0 { + closeErr = errs + } + }() + + wg.Add(1) + go func() { + defer wg.Done() + for range pc.messages { + // drain + } + }() + + wg.Wait() + return closeErr +} + +// Errors implements the Errors method from the sarama.PartitionConsumer interface. +func (pc *MockAutoPartitionConsumer) Errors() <-chan *sarama.ConsumerError { + return pc.errors +} + +// Messages implements the Messages method from the sarama.PartitionConsumer interface. +func (pc *MockAutoPartitionConsumer) Messages() <-chan *sarama.ConsumerMessage { + return pc.messages +} + +func (pc *MockAutoPartitionConsumer) HighWaterMarkOffset() int64 { + return atomic.LoadInt64(&pc.highWaterMarkOffset) +} + +/////////////////////////////////////////////////// +// Expectation API +/////////////////////////////////////////////////// + +// YieldMessage will yield a messages Messages channel of this partition consumer +// when it is consumed. By default, the mock consumer will not verify whether this +// message was consumed from the Messages channel, because there are legitimate +// reasons forthis not to happen. ou can call ExpectMessagesDrainedOnClose so it will +// verify that the channel is empty on close. +func (pc *MockAutoPartitionConsumer) YieldMessage(msg *sarama.ConsumerMessage) { + pc.l.Lock() + defer pc.l.Unlock() + + msg.Topic = pc.topic + msg.Partition = pc.partition + msg.Offset = atomic.LoadInt64(&pc.highWaterMarkOffset) + atomic.AddInt64(&pc.highWaterMarkOffset, 1) + + pc.messages <- msg +} + +// YieldError will yield an error on the Errors channel of this partition consumer +// when it is consumed. By default, the mock consumer will not verify whether this error was +// consumed from the Errors channel, because there are legitimate reasons for this +// not to happen. You can call ExpectErrorsDrainedOnClose so it will verify that +// the channel is empty on close. +func (pc *MockAutoPartitionConsumer) YieldError(err error) { + pc.errors <- &sarama.ConsumerError{ + Topic: pc.topic, + Partition: pc.partition, + Err: err, + } +} + +// ExpectMessagesDrainedOnClose sets an expectation on the partition consumer +// that the messages channel will be fully drained when Close is called. If this +// expectation is not met, an error is reported to the error reporter. +func (pc *MockAutoPartitionConsumer) ExpectMessagesDrainedOnClose() { + pc.messagesShouldBeDrained = true +} + +// ExpectErrorsDrainedOnClose sets an expectation on the partition consumer +// that the errors channel will be fully drained when Close is called. If this +// expectation is not met, an error is reported to the error reporter. +func (pc *MockAutoPartitionConsumer) ExpectErrorsDrainedOnClose() { + pc.errorsShouldBeDrained = true +} + +// MockConsumerGroupSession mocks the consumer group session used for testing +type MockConsumerGroupSession struct { + ctx context.Context + generation int32 + topics []string + claims map[string]*MockConsumerGroupClaim + + consumerGroup *MockConsumerGroup +} + +// ConsumerGroupClaim mocks the claim... +type MockConsumerGroupClaim struct { + topic string + partition int32 + msgs chan *sarama.ConsumerMessage +} + +// NewConsumerGroupClaim creates a new mock +func NewMockConsumerGroupClaim(topic string, partition int32) *MockConsumerGroupClaim { + return &MockConsumerGroupClaim{ + topic: topic, + partition: partition, + msgs: make(chan *sarama.ConsumerMessage), + } +} + +// Topic returns the current topic of the claim +func (cgc *MockConsumerGroupClaim) Topic() string { + return cgc.topic +} + +// Partition returns the partition +func (cgc *MockConsumerGroupClaim) Partition() int32 { + return cgc.partition +} + +// InitialOffset returns the initial offset +func (cgc *MockConsumerGroupClaim) InitialOffset() int64 { + return 0 +} + +// HighWaterMarkOffset returns the hwm offset +func (cgc *MockConsumerGroupClaim) HighWaterMarkOffset() int64 { + return 0 +} + +// Messages returns the message channel that must be +func (cgc *MockConsumerGroupClaim) Messages() <-chan *sarama.ConsumerMessage { + return cgc.msgs +} + +func newConsumerGroupSession(ctx context.Context, generation int32, cg *MockConsumerGroup, topics []string) *MockConsumerGroupSession { + return &MockConsumerGroupSession{ + ctx: ctx, + generation: generation, + consumerGroup: cg, + topics: topics, + claims: make(map[string]*MockConsumerGroupClaim), + } +} + +// Claims returns the number of partitions assigned in the group session for each topic +func (cgs *MockConsumerGroupSession) Claims() map[string][]int32 { + claims := make(map[string][]int32) + for _, topic := range cgs.topics { + claims[topic] = []int32{0} + } + return claims +} + +func (cgs *MockConsumerGroupSession) createGroupClaim(topic string, partition int32) *MockConsumerGroupClaim { + cgs.claims[topic] = NewMockConsumerGroupClaim(topic, 0) + + return cgs.claims[topic] +} + +// SendMessage sends a message to the consumer +func (cgs *MockConsumerGroupSession) SendMessage(msg *sarama.ConsumerMessage) { + + for topic, claim := range cgs.claims { + if topic == msg.Topic { + claim.msgs <- msg + } + } +} + +// MemberID returns the member ID +// TOOD: clarify what that actually means and whether we need to mock taht somehow +func (cgs *MockConsumerGroupSession) MemberID() string { + panic("MemberID not provided by mock") +} + +// GenerationID returns the generation ID of the group consumer +func (cgs *MockConsumerGroupSession) GenerationID() int32 { + return cgs.generation +} + +// MarkOffset marks the passed offset consumed in topic/partition +func (cgs *MockConsumerGroupSession) MarkOffset(topic string, partition int32, offset int64, metadata string) { + panic("not implemented") +} + +// ResetOffset resets the offset to be consumed from +func (cgs *MockConsumerGroupSession) ResetOffset(topic string, partition int32, offset int64, metadata string) { + panic("reset offset is not implemented by the mock") +} + +// MarkMessage marks the passed message as consumed +func (cgs *MockConsumerGroupSession) MarkMessage(msg *sarama.ConsumerMessage, metadata string) { + cgs.consumerGroup.markMessage(msg) +} + +// Context returns the consumer group's context +func (cgs *MockConsumerGroupSession) Context() context.Context { + return cgs.ctx +} + +// MockConsumerGroup mocks the consumergroup +type MockConsumerGroup struct { + errs chan error + + // setting this makes the consume call fail with this error for testing + failOnConsume error + + // use the same offset counter for all topics + offset int64 + currentGeneration int32 + + // messages we sent to the consumergroup and need to wait for + mMessages sync.Mutex + messages map[int64]int64 + wgMessages sync.WaitGroup + + sessions map[string]*MockConsumerGroupSession +} + +// NewMockConsumerGroup creates a new consumer group +func NewMockConsumerGroup(t *testing.T) *MockConsumerGroup { + return &MockConsumerGroup{ + errs: make(chan error, 1), + sessions: make(map[string]*MockConsumerGroupSession), + messages: make(map[int64]int64), + } +} + +func (cg *MockConsumerGroup) FailOnConsume(err error) { + cg.failOnConsume = err +} + +func (cg *MockConsumerGroup) nextOffset() int64 { + return atomic.AddInt64(&cg.offset, 1) +} + +func (cg *MockConsumerGroup) topicKey(topics []string) string { + return strings.Join(topics, ",") +} + +func (cg *MockConsumerGroup) markMessage(msg *sarama.ConsumerMessage) { + cg.mMessages.Lock() + defer cg.mMessages.Unlock() + + cnt := cg.messages[msg.Offset] + + if cnt == 0 { + panic(fmt.Errorf("Cannot mark message with offest %d, it's not a valid offset or was already marked", msg.Offset)) + } + + cg.messages[msg.Offset] = cnt - 1 + + cg.wgMessages.Done() +} + +// Consume starts consuming from the consumergroup +func (cg *MockConsumerGroup) Consume(ctx context.Context, topics []string, handler sarama.ConsumerGroupHandler) error { + if cg.failOnConsume != nil { + return cg.failOnConsume + } + + key := cg.topicKey(topics) + for { + cg.currentGeneration++ + session := newConsumerGroupSession(ctx, cg.currentGeneration, cg, topics) + + cg.sessions[key] = session + + err := handler.Setup(session) + if err != nil { + return fmt.Errorf("Error setting up: %v", err) + } + errg, _ := multierr.NewErrGroup(ctx) + for _, topic := range topics { + claim := session.createGroupClaim(topic, 0) + errg.Go(func() error { + <-ctx.Done() + close(claim.msgs) + return nil + }) + errg.Go(func() error { + err := handler.ConsumeClaim(session, claim) + return err + }) + } + + errs := new(multierr.Errors) + + // wait for runner errors and collect error + errs.Collect(errg.Wait().NilOrError()) + + // cleanup and collect errors + errs.Collect(handler.Cleanup(session)) + + // remove current sessions + delete(cg.sessions, key) + + err = errs.NilOrError() + if err != nil { + return fmt.Errorf("Error running or cleaning: %v", err) + } + + select { + // if the session was terminated because of a cancelled context, + // stop the loop + case <-ctx.Done(): + return nil + + // otherwise just continue with the next generation + default: + } + } +} + +// SendError sends an error the consumergroup +func (cg *MockConsumerGroup) SendError(err error) { + cg.errs <- err +} + +// SendMessage sends a message to the consumergroup +// returns a channel that will be closed when the message has been committed +// by the group +func (cg *MockConsumerGroup) SendMessage(message *sarama.ConsumerMessage) <-chan struct{} { + cg.mMessages.Lock() + defer cg.mMessages.Unlock() + + message.Offset = cg.nextOffset() + + var messages int + for _, session := range cg.sessions { + session.SendMessage(message) + messages++ + } + + cg.messages[message.Offset] += int64(messages) + cg.wgMessages.Add(messages) + + done := make(chan struct{}) + go func() { + defer close(done) + cg.wgMessages.Wait() + }() + + return done +} + +// SendMessageWait sends a message to the consumergroup waiting for the message for being committed +func (cg *MockConsumerGroup) SendMessageWait(message *sarama.ConsumerMessage) { + <-cg.SendMessage(message) +} + +// Errors returns the errors channel +func (cg *MockConsumerGroup) Errors() <-chan error { + return cg.errs +} + +// Close closes the consumergroup +func (cg *MockConsumerGroup) Close() error { + cg.messages = make(map[int64]int64) + + // close old errs chan and create new one + close(cg.errs) + cg.errs = make(chan error) + + cg.offset = 0 + cg.currentGeneration = 0 + cg.sessions = make(map[string]*MockConsumerGroupSession) + cg.failOnConsume = nil + return nil +} diff --git a/mockbuilder.go b/mockbuilder.go new file mode 100644 index 00000000..0fe02cac --- /dev/null +++ b/mockbuilder.go @@ -0,0 +1,89 @@ +package goka + +import ( + "errors" + "fmt" + "hash" + "testing" + + "github.com/golang/mock/gomock" + + "github.com/lovoo/goka/storage" +) + +var ( + errProducerBuilder error = errors.New("building producer failed on purpose") +) + +type builderMock struct { + ctrl *gomock.Controller + st storage.Storage + mst *MockStorage + tmgr *MockTopicManager + consumerGroup *MockConsumerGroup + producer *MockProducer + client *MockClient + broker *MockBroker +} + +func newBuilderMock(ctrl *gomock.Controller) *builderMock { + return &builderMock{ + ctrl: ctrl, + mst: NewMockStorage(ctrl), + tmgr: NewMockTopicManager(ctrl), + producer: NewMockProducer(ctrl), + client: NewMockClient(ctrl), + broker: NewMockBroker(ctrl), + } +} + +func (bm *builderMock) createProcessorOptions(consBuilder SaramaConsumerBuilder, groupBuilder ConsumerGroupBuilder) []ProcessorOption { + return []ProcessorOption{ + WithStorageBuilder(bm.getStorageBuilder()), + WithTopicManagerBuilder(bm.getTopicManagerBuilder()), + WithProducerBuilder(bm.getProducerBuilder()), + WithConsumerGroupBuilder(groupBuilder), + WithConsumerSaramaBuilder(consBuilder), + } +} + +func (bm *builderMock) getStorageBuilder() storage.Builder { + return func(topic string, partition int32) (storage.Storage, error) { + if bm.st != nil { + return bm.st, nil + } + return bm.mst, nil + } +} + +func (bm *builderMock) getTopicManagerBuilder() TopicManagerBuilder { + return func([]string) (TopicManager, error) { + return bm.tmgr, nil + } +} + +func (bm *builderMock) getProducerBuilder() ProducerBuilder { + return func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { + return bm.producer, nil + } +} + +func (bm *builderMock) getErrorProducerBuilder() ProducerBuilder { + return func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { + return nil, errProducerBuilder + } +} + +func (bm *builderMock) useMemoryStorage() { + bm.st = storage.NewMemory() +} + +func errStorageBuilder() storage.Builder { + return func(topic string, partition int32) (storage.Storage, error) { + return nil, fmt.Errorf("error returned by errStorageBuilder") + } +} + +func defaultSaramaAutoConsumerMock(t *testing.T) *MockAutoConsumer { + return NewMockAutoConsumer(t, DefaultConfig()) +} diff --git a/mock/controller.go b/mockcontroller.go similarity index 98% rename from mock/controller.go rename to mockcontroller.go index 003116bf..9fc0e76f 100644 --- a/mock/controller.go +++ b/mockcontroller.go @@ -1,4 +1,4 @@ -package mock +package goka import ( "fmt" diff --git a/mock/kafka.go b/mocks.go similarity index 58% rename from mock/kafka.go rename to mocks.go index a8750fd8..27f9082f 100644 --- a/mock/kafka.go +++ b/mocks.go @@ -1,134 +1,15 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/lovoo/goka/kafka (interfaces: Consumer,TopicManager,Producer) +// Source: github.com/lovoo/goka (interfaces: TopicManager,Producer,Broker) -// Package mock is a generated GoMock package. -package mock +// Package goka is a generated GoMock package. +package goka import ( + sarama "github.com/Shopify/sarama" gomock "github.com/golang/mock/gomock" - kafka "github.com/lovoo/goka/kafka" reflect "reflect" ) -// MockConsumer is a mock of Consumer interface -type MockConsumer struct { - ctrl *gomock.Controller - recorder *MockConsumerMockRecorder -} - -// MockConsumerMockRecorder is the mock recorder for MockConsumer -type MockConsumerMockRecorder struct { - mock *MockConsumer -} - -// NewMockConsumer creates a new mock instance -func NewMockConsumer(ctrl *gomock.Controller) *MockConsumer { - mock := &MockConsumer{ctrl: ctrl} - mock.recorder = &MockConsumerMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockConsumer) EXPECT() *MockConsumerMockRecorder { - return m.recorder -} - -// AddGroupPartition mocks base method -func (m *MockConsumer) AddGroupPartition(arg0 int32) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "AddGroupPartition", arg0) -} - -// AddGroupPartition indicates an expected call of AddGroupPartition -func (mr *MockConsumerMockRecorder) AddGroupPartition(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddGroupPartition", reflect.TypeOf((*MockConsumer)(nil).AddGroupPartition), arg0) -} - -// AddPartition mocks base method -func (m *MockConsumer) AddPartition(arg0 string, arg1 int32, arg2 int64) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "AddPartition", arg0, arg1, arg2) - ret0, _ := ret[0].(error) - return ret0 -} - -// AddPartition indicates an expected call of AddPartition -func (mr *MockConsumerMockRecorder) AddPartition(arg0, arg1, arg2 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddPartition", reflect.TypeOf((*MockConsumer)(nil).AddPartition), arg0, arg1, arg2) -} - -// Close mocks base method -func (m *MockConsumer) 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 *MockConsumerMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockConsumer)(nil).Close)) -} - -// Commit mocks base method -func (m *MockConsumer) Commit(arg0 string, arg1 int32, arg2 int64) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Commit", arg0, arg1, arg2) - ret0, _ := ret[0].(error) - return ret0 -} - -// Commit indicates an expected call of Commit -func (mr *MockConsumerMockRecorder) Commit(arg0, arg1, arg2 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Commit", reflect.TypeOf((*MockConsumer)(nil).Commit), arg0, arg1, arg2) -} - -// Events mocks base method -func (m *MockConsumer) Events() <-chan kafka.Event { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Events") - ret0, _ := ret[0].(<-chan kafka.Event) - return ret0 -} - -// Events indicates an expected call of Events -func (mr *MockConsumerMockRecorder) Events() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Events", reflect.TypeOf((*MockConsumer)(nil).Events)) -} - -// RemovePartition mocks base method -func (m *MockConsumer) RemovePartition(arg0 string, arg1 int32) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "RemovePartition", arg0, arg1) - ret0, _ := ret[0].(error) - return ret0 -} - -// RemovePartition indicates an expected call of RemovePartition -func (mr *MockConsumerMockRecorder) RemovePartition(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemovePartition", reflect.TypeOf((*MockConsumer)(nil).RemovePartition), arg0, arg1) -} - -// Subscribe mocks base method -func (m *MockConsumer) Subscribe(arg0 map[string]int64) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Subscribe", arg0) - ret0, _ := ret[0].(error) - return ret0 -} - -// Subscribe indicates an expected call of Subscribe -func (mr *MockConsumerMockRecorder) Subscribe(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Subscribe", reflect.TypeOf((*MockConsumer)(nil).Subscribe), arg0) -} - // MockTopicManager is a mock of TopicManager interface type MockTopicManager struct { ctrl *gomock.Controller @@ -208,6 +89,21 @@ func (mr *MockTopicManagerMockRecorder) EnsureTopicExists(arg0, arg1, arg2, arg3 return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EnsureTopicExists", reflect.TypeOf((*MockTopicManager)(nil).EnsureTopicExists), arg0, arg1, arg2, arg3) } +// GetOffset mocks base method +func (m *MockTopicManager) GetOffset(arg0 string, arg1 int32, arg2 int64) (int64, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetOffset", arg0, arg1, arg2) + ret0, _ := ret[0].(int64) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetOffset indicates an expected call of GetOffset +func (mr *MockTopicManagerMockRecorder) GetOffset(arg0, arg1, arg2 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetOffset", reflect.TypeOf((*MockTopicManager)(nil).GetOffset), arg0, arg1, arg2) +} + // Partitions mocks base method func (m *MockTopicManager) Partitions(arg0 string) ([]int32, error) { m.ctrl.T.Helper() @@ -261,10 +157,10 @@ func (mr *MockProducerMockRecorder) Close() *gomock.Call { } // Emit mocks base method -func (m *MockProducer) Emit(arg0, arg1 string, arg2 []byte) *kafka.Promise { +func (m *MockProducer) Emit(arg0, arg1 string, arg2 []byte) *Promise { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Emit", arg0, arg1, arg2) - ret0, _ := ret[0].(*kafka.Promise) + ret0, _ := ret[0].(*Promise) return ret0 } @@ -273,3 +169,84 @@ func (mr *MockProducerMockRecorder) Emit(arg0, arg1, arg2 interface{}) *gomock.C mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Emit", reflect.TypeOf((*MockProducer)(nil).Emit), arg0, arg1, arg2) } + +// MockBroker is a mock of Broker interface +type MockBroker struct { + ctrl *gomock.Controller + recorder *MockBrokerMockRecorder +} + +// MockBrokerMockRecorder is the mock recorder for MockBroker +type MockBrokerMockRecorder struct { + mock *MockBroker +} + +// NewMockBroker creates a new mock instance +func NewMockBroker(ctrl *gomock.Controller) *MockBroker { + mock := &MockBroker{ctrl: ctrl} + mock.recorder = &MockBrokerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use +func (m *MockBroker) EXPECT() *MockBrokerMockRecorder { + return m.recorder +} + +// Addr mocks base method +func (m *MockBroker) Addr() string { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Addr") + ret0, _ := ret[0].(string) + return ret0 +} + +// Addr indicates an expected call of Addr +func (mr *MockBrokerMockRecorder) Addr() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Addr", reflect.TypeOf((*MockBroker)(nil).Addr)) +} + +// Connected mocks base method +func (m *MockBroker) Connected() (bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Connected") + ret0, _ := ret[0].(bool) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Connected indicates an expected call of Connected +func (mr *MockBrokerMockRecorder) Connected() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Connected", reflect.TypeOf((*MockBroker)(nil).Connected)) +} + +// CreateTopics mocks base method +func (m *MockBroker) CreateTopics(arg0 *sarama.CreateTopicsRequest) (*sarama.CreateTopicsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CreateTopics", arg0) + ret0, _ := ret[0].(*sarama.CreateTopicsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CreateTopics indicates an expected call of CreateTopics +func (mr *MockBrokerMockRecorder) CreateTopics(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateTopics", reflect.TypeOf((*MockBroker)(nil).CreateTopics), arg0) +} + +// Open mocks base method +func (m *MockBroker) Open(arg0 *sarama.Config) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Open", arg0) + ret0, _ := ret[0].(error) + return ret0 +} + +// Open indicates an expected call of Open +func (mr *MockBrokerMockRecorder) Open(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Open", reflect.TypeOf((*MockBroker)(nil).Open), arg0) +} diff --git a/kafka/mock/sarama.go b/mockssarama.go similarity index 60% rename from kafka/mock/sarama.go rename to mockssarama.go index 97c75465..cd186ae7 100644 --- a/kafka/mock/sarama.go +++ b/mockssarama.go @@ -1,8 +1,8 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/Shopify/sarama (interfaces: Client,Consumer,PartitionConsumer) +// Source: github.com/Shopify/sarama (interfaces: Client) -// Package mock is a generated GoMock package. -package mock +// Package goka is a generated GoMock package. +package goka import ( sarama "github.com/Shopify/sarama" @@ -209,6 +209,21 @@ func (mr *MockClientMockRecorder) Partitions(arg0 interface{}) *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Partitions", reflect.TypeOf((*MockClient)(nil).Partitions), arg0) } +// RefreshController mocks base method +func (m *MockClient) RefreshController() (*sarama.Broker, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RefreshController") + ret0, _ := ret[0].(*sarama.Broker) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RefreshController indicates an expected call of RefreshController +func (mr *MockClientMockRecorder) RefreshController() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RefreshController", reflect.TypeOf((*MockClient)(nil).RefreshController)) +} + // RefreshCoordinator mocks base method func (m *MockClient) RefreshCoordinator(arg0 string) error { m.ctrl.T.Helper() @@ -285,190 +300,3 @@ func (mr *MockClientMockRecorder) WritablePartitions(arg0 interface{}) *gomock.C mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WritablePartitions", reflect.TypeOf((*MockClient)(nil).WritablePartitions), arg0) } - -// MockConsumer is a mock of Consumer interface -type MockConsumer struct { - ctrl *gomock.Controller - recorder *MockConsumerMockRecorder -} - -// MockConsumerMockRecorder is the mock recorder for MockConsumer -type MockConsumerMockRecorder struct { - mock *MockConsumer -} - -// NewMockConsumer creates a new mock instance -func NewMockConsumer(ctrl *gomock.Controller) *MockConsumer { - mock := &MockConsumer{ctrl: ctrl} - mock.recorder = &MockConsumerMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockConsumer) EXPECT() *MockConsumerMockRecorder { - return m.recorder -} - -// Close mocks base method -func (m *MockConsumer) 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 *MockConsumerMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockConsumer)(nil).Close)) -} - -// ConsumePartition mocks base method -func (m *MockConsumer) ConsumePartition(arg0 string, arg1 int32, arg2 int64) (sarama.PartitionConsumer, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ConsumePartition", arg0, arg1, arg2) - ret0, _ := ret[0].(sarama.PartitionConsumer) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ConsumePartition indicates an expected call of ConsumePartition -func (mr *MockConsumerMockRecorder) ConsumePartition(arg0, arg1, arg2 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ConsumePartition", reflect.TypeOf((*MockConsumer)(nil).ConsumePartition), arg0, arg1, arg2) -} - -// HighWaterMarks mocks base method -func (m *MockConsumer) HighWaterMarks() map[string]map[int32]int64 { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "HighWaterMarks") - ret0, _ := ret[0].(map[string]map[int32]int64) - return ret0 -} - -// HighWaterMarks indicates an expected call of HighWaterMarks -func (mr *MockConsumerMockRecorder) HighWaterMarks() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HighWaterMarks", reflect.TypeOf((*MockConsumer)(nil).HighWaterMarks)) -} - -// Partitions mocks base method -func (m *MockConsumer) Partitions(arg0 string) ([]int32, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Partitions", arg0) - ret0, _ := ret[0].([]int32) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// Partitions indicates an expected call of Partitions -func (mr *MockConsumerMockRecorder) Partitions(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Partitions", reflect.TypeOf((*MockConsumer)(nil).Partitions), arg0) -} - -// Topics mocks base method -func (m *MockConsumer) Topics() ([]string, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Topics") - ret0, _ := ret[0].([]string) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// Topics indicates an expected call of Topics -func (mr *MockConsumerMockRecorder) Topics() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Topics", reflect.TypeOf((*MockConsumer)(nil).Topics)) -} - -// MockPartitionConsumer is a mock of PartitionConsumer interface -type MockPartitionConsumer struct { - ctrl *gomock.Controller - recorder *MockPartitionConsumerMockRecorder -} - -// MockPartitionConsumerMockRecorder is the mock recorder for MockPartitionConsumer -type MockPartitionConsumerMockRecorder struct { - mock *MockPartitionConsumer -} - -// NewMockPartitionConsumer creates a new mock instance -func NewMockPartitionConsumer(ctrl *gomock.Controller) *MockPartitionConsumer { - mock := &MockPartitionConsumer{ctrl: ctrl} - mock.recorder = &MockPartitionConsumerMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockPartitionConsumer) EXPECT() *MockPartitionConsumerMockRecorder { - return m.recorder -} - -// AsyncClose mocks base method -func (m *MockPartitionConsumer) AsyncClose() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "AsyncClose") -} - -// AsyncClose indicates an expected call of AsyncClose -func (mr *MockPartitionConsumerMockRecorder) AsyncClose() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AsyncClose", reflect.TypeOf((*MockPartitionConsumer)(nil).AsyncClose)) -} - -// Close mocks base method -func (m *MockPartitionConsumer) 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 *MockPartitionConsumerMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockPartitionConsumer)(nil).Close)) -} - -// Errors mocks base method -func (m *MockPartitionConsumer) Errors() <-chan *sarama.ConsumerError { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Errors") - ret0, _ := ret[0].(<-chan *sarama.ConsumerError) - return ret0 -} - -// Errors indicates an expected call of Errors -func (mr *MockPartitionConsumerMockRecorder) Errors() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Errors", reflect.TypeOf((*MockPartitionConsumer)(nil).Errors)) -} - -// HighWaterMarkOffset mocks base method -func (m *MockPartitionConsumer) HighWaterMarkOffset() int64 { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "HighWaterMarkOffset") - ret0, _ := ret[0].(int64) - return ret0 -} - -// HighWaterMarkOffset indicates an expected call of HighWaterMarkOffset -func (mr *MockPartitionConsumerMockRecorder) HighWaterMarkOffset() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HighWaterMarkOffset", reflect.TypeOf((*MockPartitionConsumer)(nil).HighWaterMarkOffset)) -} - -// Messages mocks base method -func (m *MockPartitionConsumer) Messages() <-chan *sarama.ConsumerMessage { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Messages") - ret0, _ := ret[0].(<-chan *sarama.ConsumerMessage) - return ret0 -} - -// Messages indicates an expected call of Messages -func (mr *MockPartitionConsumerMockRecorder) Messages() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Messages", reflect.TypeOf((*MockPartitionConsumer)(nil).Messages)) -} diff --git a/mock/storage.go b/mockstorage.go similarity index 99% rename from mock/storage.go rename to mockstorage.go index 9b0f1c3f..43e1e655 100644 --- a/mock/storage.go +++ b/mockstorage.go @@ -1,8 +1,8 @@ // Code generated by MockGen. DO NOT EDIT. // Source: github.com/lovoo/goka/storage (interfaces: Storage) -// Package mock is a generated GoMock package. -package mock +// Package goka is a generated GoMock package. +package goka import ( gomock "github.com/golang/mock/gomock" diff --git a/multierr/errgroup_test.go b/multierr/errgroup_test.go index 7abaf7f8..109b09ac 100644 --- a/multierr/errgroup_test.go +++ b/multierr/errgroup_test.go @@ -5,7 +5,7 @@ import ( "fmt" "testing" - "github.com/facebookgo/ensure" + "github.com/lovoo/goka/internal/test" ) func TestErrGroup_Go(t *testing.T) { @@ -16,19 +16,19 @@ func TestErrGroup_Go(t *testing.T) { g.Go(func() error { return nil }) errs := g.Wait() err := errs.NilOrError() - ensure.Nil(t, err) - ensure.NotNil(t, ctx.Err()) - ensure.StringContains(t, ctx.Err().Error(), "context canceled") + test.AssertNil(t, err) + test.AssertNotNil(t, ctx.Err()) + test.AssertStringContains(t, ctx.Err().Error(), "context canceled") // with one error g, ctx = NewErrGroup(bctx) g.Go(func() error { return fmt.Errorf("some error") }) errs = g.Wait() err = errs.NilOrError() - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "some error") - ensure.NotNil(t, ctx.Err()) - ensure.StringContains(t, ctx.Err().Error(), "context canceled") + test.AssertNotNil(t, err) + test.AssertStringContains(t, err.Error(), "some error") + test.AssertNotNil(t, ctx.Err()) + test.AssertStringContains(t, ctx.Err().Error(), "context canceled") // with one error g, ctx = NewErrGroup(bctx) @@ -36,9 +36,9 @@ func TestErrGroup_Go(t *testing.T) { g.Go(func() error { return fmt.Errorf("some error2") }) errs = g.Wait() err = errs.NilOrError() - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "some error") - ensure.StringContains(t, err.Error(), "some error2") - ensure.NotNil(t, ctx.Err()) - ensure.StringContains(t, ctx.Err().Error(), "context canceled") + test.AssertNotNil(t, err) + test.AssertStringContains(t, err.Error(), "some error") + test.AssertStringContains(t, err.Error(), "some error2") + test.AssertNotNil(t, ctx.Err()) + test.AssertStringContains(t, ctx.Err().Error(), "context canceled") } diff --git a/multierr/multiwait.go b/multierr/multiwait.go new file mode 100644 index 00000000..57eafe38 --- /dev/null +++ b/multierr/multiwait.go @@ -0,0 +1,48 @@ +package multierr + +import ( + "context" + "sync" +) + +type MultiWait struct { + ctx context.Context + wg sync.WaitGroup +} + +func NewMultiWait(ctx context.Context, num int) *MultiWait { + mw := &MultiWait{ + ctx: ctx, + } + mw.wg.Add(num) + + return mw +} + +func (mw *MultiWait) Add(done <-chan struct{}) { + go func() { + select { + case <-mw.ctx.Done(): + case <-done: + mw.wg.Done() + } + }() +} + +func (mw *MultiWait) Done() <-chan struct{} { + done := make(chan struct{}) + go func() { + defer close(done) + mw.wg.Wait() + }() + return done +} + +func (mw *MultiWait) Wait() bool { + select { + case <-mw.Done(): + return true + case <-mw.ctx.Done(): + return false + } +} diff --git a/once_test.go b/once_test.go index cb20e192..afeb378e 100644 --- a/once_test.go +++ b/once_test.go @@ -4,16 +4,16 @@ import ( "errors" "testing" - "github.com/facebookgo/ensure" + "github.com/lovoo/goka/internal/test" ) func TestOnce_Do(t *testing.T) { var o once err := o.Do(func() error { return errors.New("some error") }) - ensure.NotNil(t, err) + test.AssertNotNil(t, err) err2 := o.Do(func() error { return nil }) - ensure.NotNil(t, err2) - ensure.DeepEqual(t, err, err2) + test.AssertNotNil(t, err2) + test.AssertEqual(t, err, err2) } diff --git a/options.go b/options.go index 13663587..a016cb28 100644 --- a/options.go +++ b/options.go @@ -6,7 +6,7 @@ import ( "hash/fnv" "path/filepath" - "github.com/lovoo/goka/kafka" + "github.com/Shopify/sarama" "github.com/lovoo/goka/logger" "github.com/lovoo/goka/storage" ) @@ -16,7 +16,7 @@ import ( type UpdateCallback func(s storage.Storage, partition int32, key string, value []byte) error // RebalanceCallback is invoked when the processor receives a new partition assignment. -type RebalanceCallback func(a kafka.Assignment) +type RebalanceCallback func(a Assignment) /////////////////////////////////////////////////////////////////////////////// // default values @@ -51,10 +51,9 @@ func DefaultUpdate(s storage.Storage, partition int32, key string, value []byte) return s.Set(key, value) } - // DefaultRebalance is the default callback when a new partition assignment is received. // DefaultRebalance can be used in the function passed to WithRebalanceCallback. -func DefaultRebalance(a kafka.Assignment) {} +func DefaultRebalance(a Assignment) {} // DefaultHasher returns an FNV hasher builder to assign keys to partitions. func DefaultHasher() func() hash.Hash32 { @@ -83,10 +82,11 @@ type poptions struct { nilHandling NilHandling builders struct { - storage storage.Builder - consumer kafka.ConsumerBuilder - producer kafka.ProducerBuilder - topicmgr kafka.TopicManagerBuilder + storage storage.Builder + consumerSarama SaramaConsumerBuilder + consumerGroup ConsumerGroupBuilder + producer ProducerBuilder + topicmgr TopicManagerBuilder } } @@ -113,21 +113,28 @@ func WithStorageBuilder(sb storage.Builder) ProcessorOption { } // WithTopicManagerBuilder replaces the default topic manager builder. -func WithTopicManagerBuilder(tmb kafka.TopicManagerBuilder) ProcessorOption { +func WithTopicManagerBuilder(tmb TopicManagerBuilder) ProcessorOption { return func(o *poptions, gg *GroupGraph) { o.builders.topicmgr = tmb } } -// WithConsumerBuilder replaces the default consumer builder. -func WithConsumerBuilder(cb kafka.ConsumerBuilder) ProcessorOption { +// WithConsumerGroupBuilder replaces the default consumer group builder +func WithConsumerGroupBuilder(cgb ConsumerGroupBuilder) ProcessorOption { return func(o *poptions, gg *GroupGraph) { - o.builders.consumer = cb + o.builders.consumerGroup = cgb + } +} + +// WithConsumerSaramaBuilder replaces the default consumer group builder +func WithConsumerSaramaBuilder(cgb SaramaConsumerBuilder) ProcessorOption { + return func(o *poptions, gg *GroupGraph) { + o.builders.consumerSarama = cgb } } // WithProducerBuilder replaces the default producer builder. -func WithProducerBuilder(pb kafka.ProducerBuilder) ProcessorOption { +func WithProducerBuilder(pb ProducerBuilder) ProcessorOption { return func(o *poptions, gg *GroupGraph) { o.builders.producer = pb } @@ -188,13 +195,14 @@ func WithNilHandling(nh NilHandling) ProcessorOption { // the tester. type Tester interface { StorageBuilder() storage.Builder - ConsumerBuilder() kafka.ConsumerBuilder - ProducerBuilder() kafka.ProducerBuilder - EmitterProducerBuilder() kafka.ProducerBuilder - TopicManagerBuilder() kafka.TopicManagerBuilder - RegisterGroupGraph(*GroupGraph) + ProducerBuilder() ProducerBuilder + ConsumerGroupBuilder() ConsumerGroupBuilder + ConsumerBuilder() SaramaConsumerBuilder + EmitterProducerBuilder() ProducerBuilder + TopicManagerBuilder() TopicManagerBuilder + RegisterGroupGraph(*GroupGraph) string RegisterEmitter(Stream, Codec) - RegisterView(Table, Codec) + RegisterView(Table, Codec) string } // WithTester configures all external connections of a processor, ie, storage, @@ -202,11 +210,12 @@ type Tester interface { func WithTester(t Tester) ProcessorOption { return func(o *poptions, gg *GroupGraph) { o.builders.storage = t.StorageBuilder() - o.builders.consumer = t.ConsumerBuilder() o.builders.producer = t.ProducerBuilder() o.builders.topicmgr = t.TopicManagerBuilder() + o.builders.consumerGroup = t.ConsumerGroupBuilder() + o.builders.consumerSarama = t.ConsumerBuilder() o.partitionChannelSize = 0 - t.RegisterGroupGraph(gg) + o.clientID = t.RegisterGroupGraph(gg) } } @@ -223,16 +232,23 @@ func (opt *poptions) applyOptions(gg *GroupGraph, opts ...ProcessorOption) error if opt.builders.storage == nil { return fmt.Errorf("StorageBuilder not set") } - if opt.builders.consumer == nil { - opt.builders.consumer = kafka.DefaultConsumerBuilder + + if globalConfig.Producer.RequiredAcks == sarama.NoResponse { + return fmt.Errorf("Processors do not work with `Config.Producer.RequiredAcks==sarama.NoResponse`, as it uses the response's offset to store the value") } + if opt.builders.producer == nil { - opt.builders.producer = kafka.DefaultProducerBuilder + opt.builders.producer = DefaultProducerBuilder } if opt.builders.topicmgr == nil { - opt.builders.topicmgr = kafka.DefaultTopicManagerBuilder + opt.builders.topicmgr = DefaultTopicManagerBuilder + } + if opt.builders.consumerGroup == nil { + opt.builders.consumerGroup = DefaultConsumerGroupBuilder + } + if opt.builders.consumerSarama == nil { + opt.builders.consumerSarama = DefaultSaramaConsumerBuilder } - return nil } @@ -252,18 +268,17 @@ func WithRebalanceCallback(cb RebalanceCallback) ProcessorOption { type ViewOption func(*voptions, Table, Codec) type voptions struct { - log logger.Logger - clientID string - tableCodec Codec - updateCallback UpdateCallback - partitionChannelSize int - hasher func() hash.Hash32 - restartable bool + log logger.Logger + clientID string + tableCodec Codec + updateCallback UpdateCallback + hasher func() hash.Hash32 + restartable bool builders struct { - storage storage.Builder - consumer kafka.ConsumerBuilder - topicmgr kafka.TopicManagerBuilder + storage storage.Builder + consumerSarama SaramaConsumerBuilder + topicmgr TopicManagerBuilder } } @@ -290,29 +305,20 @@ func WithViewStorageBuilder(sb storage.Builder) ViewOption { } } -// WithViewConsumerBuilder replaces default view consumer. -func WithViewConsumerBuilder(cb kafka.ConsumerBuilder) ViewOption { +// WithViewConsumerSaramaBuilder replaces the default sarama consumer builder +func WithViewConsumerSaramaBuilder(cgb SaramaConsumerBuilder) ViewOption { return func(o *voptions, table Table, codec Codec) { - o.builders.consumer = cb + o.builders.consumerSarama = cgb } } // WithViewTopicManagerBuilder replaces the default topic manager. -func WithViewTopicManagerBuilder(tmb kafka.TopicManagerBuilder) ViewOption { +func WithViewTopicManagerBuilder(tmb TopicManagerBuilder) ViewOption { return func(o *voptions, table Table, codec Codec) { o.builders.topicmgr = tmb } } -// WithViewPartitionChannelSize replaces the default partition channel size. -// This is mostly used for testing by setting it to 0 to have synchronous behavior -// of goka. -func WithViewPartitionChannelSize(size int) ViewOption { - return func(o *voptions, table Table, codec Codec) { - o.partitionChannelSize = size - } -} - // WithViewHasher sets the hash function that assigns keys to partitions. func WithViewHasher(hasher func() hash.Hash32) ViewOption { return func(o *voptions, table Table, codec Codec) { @@ -341,10 +347,9 @@ func WithViewRestartable() ViewOption { func WithViewTester(t Tester) ViewOption { return func(o *voptions, table Table, codec Codec) { o.builders.storage = t.StorageBuilder() - o.builders.consumer = t.ConsumerBuilder() o.builders.topicmgr = t.TopicManagerBuilder() - o.partitionChannelSize = 0 - t.RegisterView(table, codec) + o.builders.consumerSarama = t.ConsumerBuilder() + o.clientID = t.RegisterView(table, codec) } } @@ -361,11 +366,13 @@ func (opt *voptions) applyOptions(topic Table, codec Codec, opts ...ViewOption) if opt.builders.storage == nil { return fmt.Errorf("StorageBuilder not set") } - if opt.builders.consumer == nil { - opt.builders.consumer = kafka.DefaultConsumerBuilder + + if opt.builders.consumerSarama == nil { + opt.builders.consumerSarama = DefaultSaramaConsumerBuilder } + if opt.builders.topicmgr == nil { - opt.builders.topicmgr = kafka.DefaultTopicManagerBuilder + opt.builders.topicmgr = DefaultTopicManagerBuilder } return nil @@ -387,8 +394,8 @@ type eoptions struct { hasher func() hash.Hash32 builders struct { - topicmgr kafka.TopicManagerBuilder - producer kafka.ProducerBuilder + topicmgr TopicManagerBuilder + producer ProducerBuilder } } @@ -408,14 +415,14 @@ func WithEmitterClientID(clientID string) EmitterOption { } // WithEmitterTopicManagerBuilder replaces the default topic manager builder. -func WithEmitterTopicManagerBuilder(tmb kafka.TopicManagerBuilder) EmitterOption { +func WithEmitterTopicManagerBuilder(tmb TopicManagerBuilder) EmitterOption { return func(o *eoptions, topic Stream, codec Codec) { o.builders.topicmgr = tmb } } // WithEmitterProducerBuilder replaces the default producer builder. -func WithEmitterProducerBuilder(pb kafka.ProducerBuilder) EmitterOption { +func WithEmitterProducerBuilder(pb ProducerBuilder) EmitterOption { return func(o *eoptions, topic Stream, codec Codec) { o.builders.producer = pb } @@ -435,7 +442,7 @@ func WithEmitterTester(t Tester) EmitterOption { t.RegisterEmitter(topic, codec) } } -func (opt *eoptions) applyOptions(topic Stream, codec Codec, opts ...EmitterOption) error { +func (opt *eoptions) applyOptions(topic Stream, codec Codec, opts ...EmitterOption) { opt.clientID = defaultClientID opt.log = logger.Default() opt.hasher = DefaultHasher() @@ -446,11 +453,9 @@ func (opt *eoptions) applyOptions(topic Stream, codec Codec, opts ...EmitterOpti // config not set, use default one if opt.builders.producer == nil { - opt.builders.producer = kafka.DefaultProducerBuilder + opt.builders.producer = DefaultProducerBuilder } if opt.builders.topicmgr == nil { - opt.builders.topicmgr = kafka.DefaultTopicManagerBuilder + opt.builders.topicmgr = DefaultTopicManagerBuilder } - - return nil } diff --git a/options_test.go b/options_test.go index 04faf401..f4e8bab9 100644 --- a/options_test.go +++ b/options_test.go @@ -5,17 +5,24 @@ import ( "regexp" "testing" - "github.com/facebookgo/ensure" + "github.com/lovoo/goka/internal/test" + "github.com/lovoo/goka/storage" ) +func nullStorageBuilder() storage.Builder { + return func(topic string, partition int32) (storage.Storage, error) { + return &storage.Null{}, nil + } +} + func newMockOptions(t *testing.T) *poptions { opts := new(poptions) err := opts.applyOptions(new(GroupGraph)) - ensure.Err(t, err, regexp.MustCompile("StorageBuilder not set$")) + test.AssertError(t, err, regexp.MustCompile("StorageBuilder not set$")) opts.builders.storage = nullStorageBuilder() err = opts.applyOptions(new(GroupGraph)) - ensure.Nil(t, err) + test.AssertNil(t, err) fmt.Printf("%+v\n", opts) return opts diff --git a/partition.go b/partition.go deleted file mode 100644 index 7462c4b6..00000000 --- a/partition.go +++ /dev/null @@ -1,448 +0,0 @@ -package goka - -import ( - "context" - "fmt" - "log" - "sync" - "sync/atomic" - "time" - - "github.com/lovoo/goka/kafka" - "github.com/lovoo/goka/logger" - "github.com/lovoo/goka/multierr" - "github.com/lovoo/goka/storage" - - "github.com/Shopify/sarama" -) - -const ( - defaultPartitionChannelSize = 10 - stallPeriod = 30 * time.Second - stalledTimeout = 2 * time.Minute -) - -// partition represents one partition of a group table and handles the updates to -// this table via UpdateCallback and ProcessCallback. -// -// partition can be started in two modes: -// - catchup-mode: used by views, starts with startCatchup(), only UpdateCallback called -// - processing-mode: used by processors, starts with start(), -// recovers table with UpdateCallback -// processes input streams with ProcessCallback -// -// The partition should never be called with a closed storage proxy. -// - Before starting the partition in either way, the client must open the storage proxy. -// - A partition may be restarted even if it returned errors. Before restarting -// it, the client must call reinit(). -// - To release all resources, after stopping the partition, the client must -// close the storage proxy. -// -type partition struct { - log logger.Logger - topic string - - ch chan kafka.Event - st *storageProxy - proxy kafkaProxy - process processCallback - - recoveredFlag int32 - hwm int64 - offset int64 - - recoveredOnce sync.Once - - stats *PartitionStats - lastStats *PartitionStats - requestStats chan bool - responseStats chan *PartitionStats - - droppedEvents []kafka.Event -} - -type kafkaProxy interface { - Add(string, int64) error - Remove(string) error - AddGroup() - Stop() -} - -type processCallback func(msg *message, st storage.Storage, wg *sync.WaitGroup, pstats *PartitionStats) (int, error) - -func newPartition(log logger.Logger, topic string, cb processCallback, st *storageProxy, proxy kafkaProxy, channelSize int) *partition { - return &partition{ - log: log, - topic: topic, - - ch: make(chan kafka.Event, channelSize), - st: st, - proxy: proxy, - process: cb, - - stats: newPartitionStats(), - lastStats: newPartitionStats(), - requestStats: make(chan bool), - responseStats: make(chan *PartitionStats, 1), - } -} - -// reinit reinitialzes the partition to connect to Kafka and start its goroutine -func (p *partition) reinit(proxy kafkaProxy) { - if proxy != nil { - p.proxy = proxy - } -} - -// start loads the table partition up to HWM and then consumes streams -func (p *partition) start(ctx context.Context) error { - defer p.proxy.Stop() - p.stats.Table.StartTime = time.Now() - - // init events - p.droppedEvents = make([]kafka.Event, 0) - - if p.st.Stateless() { - if err := p.markRecovered(false); err != nil { - return fmt.Errorf("error marking stateless partition as recovered: %v", err) - } - } else if err := p.recover(ctx); err != nil { - return err - } - - // if stopped, just return - select { - case <-ctx.Done(): - return nil - default: - } - - return p.run(ctx) -} - -// startCatchup continually loads the table partition -func (p *partition) startCatchup(ctx context.Context) error { - defer p.proxy.Stop() - p.stats.Table.StartTime = time.Now() - - return p.catchup(ctx) -} - -/////////////////////////////////////////////////////////////////////////////// -// processing -/////////////////////////////////////////////////////////////////////////////// -func newMessage(ev *kafka.Message) *message { - return &message{ - Topic: ev.Topic, - Partition: ev.Partition, - Offset: ev.Offset, - Timestamp: ev.Timestamp, - Data: ev.Value, - Key: ev.Key, - Header: ev.Header, - } -} - -func (p *partition) run(ctx context.Context) error { - var wg sync.WaitGroup - p.proxy.AddGroup() - - defer func() { - done := make(chan struct{}) - go func() { - wg.Wait() - close(done) - }() - - select { - case <-done: - case <-time.NewTimer(10 * time.Second).C: - log.Printf("partition shutdown timed out. Will stop waiting.") - } - }() - - // recover the dropped events - for _, ev := range p.droppedEvents { - select { - case p.ch <- ev: - case <-ctx.Done(): - return nil - } - } - - for { - select { - case ev, isOpen := <-p.ch: - // channel already closed, ev will be nil - if !isOpen { - return nil - } - switch ev := ev.(type) { - case *kafka.Message: - if ev.Topic == p.topic { - return fmt.Errorf("received message from group table topic after recovery: %s", p.topic) - } - - updates, err := p.process(newMessage(ev), p.st, &wg, p.stats) - if err != nil { - return fmt.Errorf("error processing message: %v", err) - } - p.offset += int64(updates) - p.hwm = p.offset + 1 - - // metrics - s := p.stats.Input[ev.Topic] - s.Count++ - s.Bytes += len(ev.Value) - if !ev.Timestamp.IsZero() { - s.Delay = time.Since(ev.Timestamp) - } - p.stats.Input[ev.Topic] = s - - case *kafka.NOP: - // don't do anything but also don't log. - case *kafka.EOF: - // if ev.Topic != p.topic { - // return fmt.Errorf("received EOF of topic that is not ours. This should not happend (ours=%s, received=%s)", p.topic, ev.Topic) - // } - default: - return fmt.Errorf("load: cannot handle %T = %v", ev, ev) - } - - case <-p.requestStats: - p.lastStats = newPartitionStats().init(p.stats, p.offset, p.hwm) - select { - case p.responseStats <- p.lastStats: - case <-ctx.Done(): - p.log.Printf("Partitioning exiting, context is cancelled") - return nil - } - - case <-ctx.Done(): - p.log.Printf("Partitioning exiting, context is cancelled (outer)") - return nil - } - - } -} - -/////////////////////////////////////////////////////////////////////////////// -// loading storage -/////////////////////////////////////////////////////////////////////////////// - -func (p *partition) catchup(ctx context.Context) error { - return p.load(ctx, true) -} - -func (p *partition) recover(ctx context.Context) error { - return p.load(ctx, false) -} - -func (p *partition) recovered() bool { - return atomic.LoadInt32(&p.recoveredFlag) == 1 -} - -func (p *partition) load(ctx context.Context, catchup bool) (rerr error) { - // fetch local offset - if local, err := p.st.GetOffset(sarama.OffsetOldest); err != nil { - return fmt.Errorf("error reading local offset: %v", err) - } else if err = p.proxy.Add(p.topic, local); err != nil { - return err - } - - defer func() { - var derr multierr.Errors - _ = derr.Collect(rerr) - if e := p.proxy.Remove(p.topic); e != nil { - _ = derr.Collect(e) - } - rerr = derr.NilOrError() - }() - - stallTicker := time.NewTicker(stallPeriod) - defer stallTicker.Stop() - - // reset stats after load - defer p.stats.reset() - - var lastMessage time.Time - for { - select { - case ev, isOpen := <-p.ch: - - // channel already closed, ev will be nil - if !isOpen { - return nil - } - - switch ev := ev.(type) { - case *kafka.BOF: - p.hwm = ev.Hwm - - if ev.Offset == ev.Hwm { - // nothing to recover - if err := p.markRecovered(false); err != nil { - return fmt.Errorf("error setting recovered: %v", err) - } - } - - case *kafka.EOF: - p.offset = ev.Hwm - 1 - p.hwm = ev.Hwm - - if err := p.markRecovered(catchup); err != nil { - return fmt.Errorf("error setting recovered: %v", err) - } - - if catchup { - continue - } - return nil - - case *kafka.Message: - lastMessage = time.Now() - if ev.Topic != p.topic { - p.log.Printf("dropping message from topic = %s while loading", ev.Topic) - // saving the dropped messages from input stream - p.droppedEvents = append(p.droppedEvents, ev) - continue - } - if err := p.storeEvent(ev); err != nil { - return fmt.Errorf("load: error updating storage: %v", err) - } - p.offset = ev.Offset - if p.offset >= p.hwm-1 { - if err := p.markRecovered(catchup); err != nil { - return fmt.Errorf("error setting recovered: %v", err) - } - } - - // update metrics - s := p.stats.Input[ev.Topic] - s.Count++ - s.Bytes += len(ev.Value) - if !ev.Timestamp.IsZero() { - s.Delay = time.Since(ev.Timestamp) - } - p.stats.Input[ev.Topic] = s - p.stats.Table.Stalled = false - - case *kafka.NOP: - // don't do anything - - default: - return fmt.Errorf("load: cannot handle %T = %v", ev, ev) - } - - case now := <-stallTicker.C: - // only set to stalled, if the last message was earlier - // than the stalled timeout - if now.Sub(lastMessage) > stalledTimeout { - p.stats.Table.Stalled = true - } - - case <-p.requestStats: - p.lastStats = newPartitionStats().init(p.stats, p.offset, p.hwm) - select { - case p.responseStats <- p.lastStats: - case <-ctx.Done(): - return nil - } - - case <-ctx.Done(): - return nil - } - } -} - -func (p *partition) storeEvent(msg *kafka.Message) error { - err := p.st.Update(msg.Key, msg.Value) - if err != nil { - return fmt.Errorf("Error from the update callback while recovering from the log: %v", err) - } - err = p.st.SetOffset(msg.Offset) - if err != nil { - return fmt.Errorf("Error updating offset in local storage while recovering from the log: %v", err) - } - return nil -} - -// mark storage as recovered -func (p *partition) markRecovered(catchup bool) (err error) { - p.recoveredOnce.Do(func() { - p.lastStats = newPartitionStats().init(p.stats, p.offset, p.hwm) - p.lastStats.Table.Status = PartitionPreparing - - var ( - done = make(chan bool) - wg sync.WaitGroup - ) - if catchup { - // if catching up (views), stop reading from topic before marking - // partition as recovered to avoid blocking other partitions when - // p.ch gets full - if err = p.proxy.Remove(p.topic); err != nil { - return - } - - // drain events channel -- we'll fetch them again later - wg.Add(1) - go func() { - defer wg.Done() - for { - select { - case <-p.ch: - case <-done: - return - } - } - }() - } - - // mark storage as recovered -- this may take long - if err = p.st.MarkRecovered(); err != nil { - close(done) - return - } - - if catchup { - close(done) - wg.Wait() - // start reading from topic again if in catchup mode - if err = p.proxy.Add(p.topic, p.hwm); err != nil { - return - } - } - - // update stats - p.stats.Table.Status = PartitionRunning - p.stats.Table.RecoveryTime = time.Now() - - atomic.StoreInt32(&p.recoveredFlag, 1) - }) - - // Be sure to mark partition as not stalled after EOF arrives, as - // this will not be written in the run-method - p.stats.Table.Stalled = false - return -} - -func (p *partition) fetchStats(ctx context.Context) *PartitionStats { - timer := time.NewTimer(100 * time.Millisecond) - defer timer.Stop() - - select { - case p.requestStats <- true: - case <-ctx.Done(): - return newPartitionStats().init(p.lastStats, p.offset, p.hwm) - case <-timer.C: - return p.lastStats - } - - select { - case s := <-p.responseStats: - return s - case <-ctx.Done(): - return newPartitionStats().init(p.lastStats, p.offset, p.hwm) - } -} diff --git a/partition_processor.go b/partition_processor.go new file mode 100644 index 00000000..2aa52863 --- /dev/null +++ b/partition_processor.go @@ -0,0 +1,473 @@ +package goka + +import ( + "context" + "fmt" + "runtime/debug" + "sync" + "time" + + "github.com/Shopify/sarama" + "github.com/lovoo/goka/logger" + "github.com/lovoo/goka/multierr" +) + +const ( + PPStateIdle State = iota + PPStateRecovering + PPStateRunning + PPStateStopping +) + +// PartitionProcessor handles message processing of one partition by serializing +// messages from different input topics. +// It also handles joined tables as well as lookup views (managed by `Processor`). +type PartitionProcessor struct { + callbacks map[string]ProcessCallback + + log logger.Logger + + table *PartitionTable + joins map[string]*PartitionTable + lookups map[string]*View + graph *GroupGraph + + state *Signal + + partition int32 + + input chan *sarama.ConsumerMessage + inputTopics []string + + runnerGroup *multierr.ErrGroup + cancelRunnerGroup func() + + consumer sarama.Consumer + tmgr TopicManager + stats *PartitionProcStats + + requestStats chan bool + responseStats chan *PartitionProcStats + + session sarama.ConsumerGroupSession + producer Producer + + opts *poptions +} + +func newPartitionProcessor(partition int32, + graph *GroupGraph, + session sarama.ConsumerGroupSession, + logger logger.Logger, + opts *poptions, + lookupTables map[string]*View, + consumer sarama.Consumer, + producer Producer, + tmgr TopicManager) *PartitionProcessor { + + // collect all topics I am responsible for + topicMap := make(map[string]bool) + for _, stream := range graph.InputStreams() { + topicMap[stream.Topic()] = true + } + if loop := graph.LoopStream(); loop != nil { + topicMap[loop.Topic()] = true + } + + var ( + topicList []string + outputList []string + callbacks = make(map[string]ProcessCallback) + ) + for t := range topicMap { + topicList = append(topicList, t) + callbacks[t] = graph.callback(t) + } + for _, output := range graph.OutputStreams() { + outputList = append(outputList, output.Topic()) + } + if graph.LoopStream() != nil { + outputList = append(outputList, graph.LoopStream().Topic()) + } + + if graph.GroupTable() != nil { + outputList = append(outputList, graph.GroupTable().Topic()) + } + + log := logger.Prefix(fmt.Sprintf("PartitionProcessor (%d)", partition)) + + partProc := &PartitionProcessor{ + log: log, + opts: opts, + partition: partition, + state: NewSignal(PPStateIdle, PPStateRecovering, PPStateRunning, PPStateStopping).SetState(PPStateIdle), + callbacks: callbacks, + lookups: lookupTables, + consumer: consumer, + producer: producer, + tmgr: tmgr, + joins: make(map[string]*PartitionTable), + input: make(chan *sarama.ConsumerMessage, opts.partitionChannelSize), + inputTopics: topicList, + graph: graph, + stats: newPartitionProcStats(topicList, outputList), + requestStats: make(chan bool), + responseStats: make(chan *PartitionProcStats, 1), + session: session, + } + + if graph.GroupTable() != nil { + partProc.table = newPartitionTable(graph.GroupTable().Topic(), + partition, + consumer, + tmgr, + opts.updateCallback, + opts.builders.storage, + log.Prefix("PartTable"), + ) + } + return partProc +} + +func (pp *PartitionProcessor) EnqueueMessage(msg *sarama.ConsumerMessage) { + pp.input <- msg +} + +func (pp *PartitionProcessor) Recovered() bool { + return pp.state.IsState(PPStateRunning) +} + +func (pp *PartitionProcessor) Errors() <-chan error { + errs := make(chan error) + + go func() { + defer close(errs) + err := pp.runnerGroup.Wait().NilOrError() + if err != nil { + errs <- err + } + }() + return errs +} + +func (pp *PartitionProcessor) Setup(ctx context.Context) error { + ctx, pp.cancelRunnerGroup = context.WithCancel(ctx) + + var runnerCtx context.Context + pp.runnerGroup, runnerCtx = multierr.NewErrGroup(ctx) + + setupErrg, setupCtx := multierr.NewErrGroup(ctx) + + pp.state.SetState(PPStateRecovering) + defer pp.state.SetState(PPStateRunning) + + if pp.table != nil { + setupErrg.Go(func() error { + pp.log.Debugf("catching up table") + defer pp.log.Debugf("catching up table done") + return pp.table.SetupAndRecover(setupCtx) + }) + } + + for _, join := range pp.graph.JointTables() { + table := newPartitionTable(join.Topic(), + pp.partition, + pp.consumer, + pp.tmgr, + pp.opts.updateCallback, + pp.opts.builders.storage, + pp.log.Prefix(fmt.Sprintf("Join %s", join.Topic())), + ) + pp.joins[join.Topic()] = table + + setupErrg.Go(func() error { + table.SetupAndRecover(setupCtx) + return nil + }) + } + + // here we wait for our table and the joins to recover + err := setupErrg.Wait().NilOrError() + if err != nil { + return fmt.Errorf("Setup failed. Cannot start processor for partition %d: %v", pp.partition, err) + } + + // as the table is now recovered, we have to start handling stats requests + // separately during running + if pp.table != nil { + pp.runnerGroup.Go(func() error { + pp.table.handleStatsRequests(runnerCtx) + return nil + }) + } + + for _, join := range pp.joins { + join := join + pp.runnerGroup.Go(func() error { + return join.CatchupForever(runnerCtx, false) + }) + } + + // now run the processor in a runner-group + pp.runnerGroup.Go(func() error { + err := pp.run(runnerCtx) + if err != nil { + pp.log.Printf("Run failed with error: %v", err) + } + return err + }) + return nil +} + +// Stop stops the partition processor +func (pp *PartitionProcessor) Stop() error { + pp.log.Debugf("Stopping") + defer pp.log.Debugf("... Stopping done") + pp.state.SetState(PPStateStopping) + defer pp.state.SetState(PPStateIdle) + errs := new(multierr.Errors) + + if pp.cancelRunnerGroup != nil { + pp.cancelRunnerGroup() + } + if pp.runnerGroup != nil { + errs.Collect(pp.runnerGroup.Wait().NilOrError()) + } + if pp.table != nil { + errs.Collect(pp.table.Close()) + } + + return errs.NilOrError() +} + +func (pp *PartitionProcessor) run(ctx context.Context) (rerr error) { + pp.log.Debugf("starting") + defer pp.log.Debugf("stopped") + + errs := new(multierr.Errors) + defer func() { + errs.Collect(rerr) + rerr = errs.NilOrError() + }() + + var ( + // syncFailer is called synchronously from the callback within *this* + // goroutine + syncFailer = func(err error) { + // only fail processor if context not already Done + select { + case <-ctx.Done(): + rerr = err + return + default: + } + panic(err) + } + + closeOnce = new(sync.Once) + asyncErrs = make(chan struct{}) + + // asyncFailer is called asynchronously from other goroutines, e.g. + // when the promise of a Emit (using a producer internally) fails + asyncFailer = func(err error) { + errs.Collect(err) + closeOnce.Do(func() { close(asyncErrs) }) + } + + wg sync.WaitGroup + ) + + defer func() { + if r := recover(); r != nil { + rerr = fmt.Errorf("%v\n%v", r, string(debug.Stack())) + return + } + + done := make(chan struct{}) + go func() { + wg.Wait() + close(done) + }() + + select { + case <-done: + case <-time.NewTimer(60 * time.Second).C: + pp.log.Printf("partition processor did not shutdown in time. Will stop waiting") + } + }() + + updateHwmStatsTicker := time.NewTicker(statsHwmUpdateInterval) + defer updateHwmStatsTicker.Stop() + + for { + select { + case ev, isOpen := <-pp.input: + // channel already closed, ev will be nil + if !isOpen { + return nil + } + err := pp.processMessage(ctx, &wg, ev, syncFailer, asyncFailer) + if err != nil { + return fmt.Errorf("error processing message: from %s %v", ev.Value, err) + } + + pp.updateStats(ev) + + case <-pp.requestStats: + stats := pp.collectStats(ctx) + select { + case pp.responseStats <- stats: + case <-ctx.Done(): + pp.log.Debugf("exiting, context is cancelled") + return + } + case <-updateHwmStatsTicker.C: + pp.updateHwmStats() + + case <-ctx.Done(): + pp.log.Debugf("exiting, context is cancelled") + return + + case <-asyncErrs: + pp.log.Debugf("Errors occurred asynchronously. Will exit partition processor") + return + } + } +} + +func (pp *PartitionProcessor) updateStats(ev *sarama.ConsumerMessage) { + ip := pp.stats.Input[ev.Topic] + ip.Bytes += len(ev.Value) + ip.LastOffset = ev.Offset + if !ev.Timestamp.IsZero() { + ip.Delay = time.Since(ev.Timestamp) + } + ip.Count++ +} + +func (pp *PartitionProcessor) updateHwmStats() { + hwms := pp.consumer.HighWaterMarks() + for input, inputStats := range pp.stats.Input { + hwm := hwms[input][pp.partition] + if hwm != 0 && inputStats.LastOffset != 0 { + inputStats.OffsetLag = hwm - inputStats.LastOffset + } + } +} + +func (pp *PartitionProcessor) collectStats(ctx context.Context) *PartitionProcStats { + var ( + stats = pp.stats.clone() + m sync.Mutex + ) + + errg, ctx := multierr.NewErrGroup(ctx) + + for topic, join := range pp.joins { + topic, join := topic, join + errg.Go(func() error { + joinStats := join.fetchStats(ctx) + m.Lock() + defer m.Unlock() + stats.Joined[topic] = joinStats + return nil + }) + } + + if pp.table != nil { + errg.Go(func() error { + stats.TableStats = pp.table.fetchStats(ctx) + return nil + }) + } + + err := errg.Wait().NilOrError() + if err != nil { + pp.log.Printf("Error retrieving stats: %v", err) + } + + return stats +} + +func (pp *PartitionProcessor) fetchStats(ctx context.Context) *PartitionProcStats { + select { + case <-ctx.Done(): + return nil + case <-time.After(fetchStatsTimeout): + pp.log.Printf("requesting stats timed out") + return nil + case pp.requestStats <- true: + } + + // retrieve from response-channel + select { + case <-ctx.Done(): + return nil + case <-time.After(fetchStatsTimeout): + pp.log.Printf("Fetching stats timed out") + return nil + case stats := <-pp.responseStats: + return stats + } +} + +func (pp *PartitionProcessor) processMessage(ctx context.Context, wg *sync.WaitGroup, msg *sarama.ConsumerMessage, syncFailer func(err error), asyncFailer func(err error)) error { + msgContext := &cbContext{ + ctx: ctx, + graph: pp.graph, + + partProcStats: pp.stats, + pviews: pp.joins, + views: pp.lookups, + commit: func() { pp.session.MarkMessage(msg, "") }, + wg: wg, + msg: msg, + syncFailer: syncFailer, + asyncFailer: asyncFailer, + emitter: pp.producer.Emit, + table: pp.table, + } + + var ( + m interface{} + err error + ) + + // decide whether to decode or ignore message + switch { + case msg.Value == nil && pp.opts.nilHandling == NilIgnore: + // mark the message upstream so we don't receive it again. + // this is usually only an edge case in unit tests, as kafka probably never sends us nil messages + pp.session.MarkMessage(msg, "") + // otherwise drop it. + return nil + case msg.Value == nil && pp.opts.nilHandling == NilProcess: + // process nil messages without decoding them + m = nil + default: + // get stream subcription + codec := pp.graph.codec(msg.Topic) + if codec == nil { + return fmt.Errorf("cannot handle topic %s", msg.Topic) + } + + // decode message + m, err = codec.Decode(msg.Value) + if err != nil { + return fmt.Errorf("error decoding message for key %s from %s/%d: %v", msg.Key, msg.Topic, msg.Partition, err) + } + } + + cb := pp.callbacks[msg.Topic] + if cb == nil { + return fmt.Errorf("error processing message for key %s from %s/%d: %v", string(msg.Key), msg.Topic, msg.Partition, err) + } + + // start context and call the ProcessorCallback cb + msgContext.start() + + // now call cb + cb(msgContext, m) + msgContext.finish(nil) + return nil +} diff --git a/partition_table.go b/partition_table.go new file mode 100644 index 00000000..54b2064f --- /dev/null +++ b/partition_table.go @@ -0,0 +1,612 @@ +package goka + +import ( + "context" + "fmt" + "sync" + "time" + + "github.com/Shopify/sarama" + "github.com/lovoo/goka/logger" + "github.com/lovoo/goka/multierr" + "github.com/lovoo/goka/storage" +) + +const ( + defaultPartitionChannelSize = 10 + defaultStallPeriod = 30 * time.Second + defaultStalledTimeout = 2 * time.Minute + + // internal offset we use to detect if the offset has never been stored locally + offsetNotStored int64 = -3 +) + +// PartitionTable manages the usage of a table for one partition. +// It allows to setup and recover/catchup the table contents from kafka, +// allow updates via Get/Set/Delete accessors +type PartitionTable struct { + log logger.Logger + topic string + partition int32 + state *Signal + builder storage.Builder + st *storageProxy + consumer sarama.Consumer + tmgr TopicManager + updateCallback UpdateCallback + + stats *TableStats + requestStats chan bool + responseStats chan *TableStats + + offsetM sync.Mutex + // current offset + offset int64 + hwm int64 + + // stall config + stallPeriod time.Duration + stalledTimeout time.Duration +} + +func newPartitionTable(topic string, + partition int32, + consumer sarama.Consumer, + tmgr TopicManager, + updateCallback UpdateCallback, + builder storage.Builder, + log logger.Logger) *PartitionTable { + return &PartitionTable{ + partition: partition, + state: NewSignal( + State(PartitionStopped), + State(PartitionInitializing), + State(PartitionRecovering), + State(PartitionPreparing), + State(PartitionRunning), + ).SetState(State(PartitionStopped)), + consumer: consumer, + tmgr: tmgr, + topic: topic, + updateCallback: updateCallback, + builder: builder, + log: log, + stallPeriod: defaultStallPeriod, + stalledTimeout: defaultStalledTimeout, + + stats: newTableStats(), + requestStats: make(chan bool), + responseStats: make(chan *TableStats, 1), + } +} + +// SetupAndRecover sets up the partition storage and recovers to HWM +func (p *PartitionTable) SetupAndRecover(ctx context.Context) error { + err := p.setup(ctx) + if err != nil { + return err + } + // do not continue if the context is already cancelled. + // this can happen if the context was closed during opening the storage. + // Since this is no error we have to check it here, otherwise it'll nil-panic later. + select { + case <-ctx.Done(): + return nil + default: + } + + return p.load(ctx, true) +} + +// CatchupForever starts catching the partition table forever (until the context is cancelled). +// Option restartOnError allows the view to stay open/intact even in case of consumer errors +func (p *PartitionTable) CatchupForever(ctx context.Context, restartOnError bool) error { + if restartOnError { + for { + err := p.load(ctx, false) + if err != nil { + p.log.Printf("Error while catching up, but we'll try to keep it running: %v", err) + } + + select { + case <-ctx.Done(): + return nil + + case <-time.After(10 * time.Second): + // retry after some time + // TODO (frairon) add exponential backoff + } + } + } + return p.load(ctx, false) +} + +// Setup creates the storage for the partition table +func (p *PartitionTable) setup(ctx context.Context) error { + p.state.SetState(State(PartitionInitializing)) + storage, err := p.createStorage(ctx) + if err != nil { + p.state.SetState(State(PartitionStopped)) + return fmt.Errorf("error setting up partition table: %v", err) + } + + p.st = storage + return nil +} + +// Close closes the partition table +func (p *PartitionTable) Close() error { + if p.st != nil { + return p.st.Close() + } + return nil +} + +func (p *PartitionTable) createStorage(ctx context.Context) (*storageProxy, error) { + var ( + err error + st storage.Storage + done = make(chan struct{}) + ) + start := time.Now() + ticker := time.NewTicker(5 * time.Minute) + defer ticker.Stop() + go func() { + defer close(done) + st, err = p.builder(p.topic, p.partition) + }() + +WaitLoop: + for { + select { + case <-ctx.Done(): + return nil, nil + case <-ticker.C: + p.log.Printf("creating storage for topic %s/%d for %.1f minutes ...", p.topic, p.partition, time.Since(start).Minutes()) + case <-done: + p.log.Printf("finished building storage for topic %s/%d in %.1f minutes", p.topic, p.partition, time.Since(start).Minutes()) + if err != nil { + return nil, fmt.Errorf("error building storage: %v", err) + } + break WaitLoop + } + } + + return &storageProxy{ + Storage: st, + partition: p.partition, + update: p.updateCallback, + }, nil + +} + +// TODO(jb): refactor comment +// findOffsetToLoad returns the first and the last offset (hwm) to load. +// If storedOffset is sarama.OffsetOldest the oldest offset known to kafka is returned as first offset. +// If storedOffset is sarama.OffsetNewest the hwm is returned as first offset. +// If storedOffset is higher than the hwm, the hwm is returned as first offset. +// If storedOffset is lower than the oldest offset, the oldest offset is returned as first offset. +func (p *PartitionTable) findOffsetToLoad(storedOffset int64) (int64, int64, error) { + oldest, err := p.tmgr.GetOffset(p.topic, p.partition, sarama.OffsetOldest) + if err != nil { + return 0, 0, fmt.Errorf("Error getting oldest offset for topic/partition %s/%d: %v", p.topic, p.partition, err) + } + hwm, err := p.tmgr.GetOffset(p.topic, p.partition, sarama.OffsetNewest) + if err != nil { + return 0, 0, fmt.Errorf("Error getting newest offset for topic/partition %s/%d: %v", p.topic, p.partition, err) + } + p.log.Debugf("topic manager gives us oldest: %d, hwm: %d", oldest, hwm) + + var start int64 + + if storedOffset == offsetNotStored { + start = oldest + } else { + start = storedOffset + 1 + } + + // if kafka does not have the offset we're looking for, use the oldest kafka has + // This can happen when the log compaction removes offsets that we stored. + if start < oldest { + start = oldest + } + return start, hwm, nil +} + +func (p *PartitionTable) load(ctx context.Context, stopAfterCatchup bool) (rerr error) { + var ( + storedOffset int64 + partConsumer sarama.PartitionConsumer + err error + errs = new(multierr.Errors) + ) + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + // deferred error handling + defer func() { + errs.Collect(rerr) + + rerr = errs.NilOrError() + return + }() + + // fetch local offset + storedOffset, err = p.st.GetOffset(offsetNotStored) + if err != nil { + errs.Collect(fmt.Errorf("error reading local offset: %v", err)) + return + } + + loadOffset, hwm, err := p.findOffsetToLoad(storedOffset) + if err != nil { + errs.Collect(err) + return + } + + if storedOffset > 0 && hwm == 0 { + errs.Collect(fmt.Errorf("kafka tells us there's no message in the topic, but our cache has one. The table might be gone. Try to delete your local cache! Topic %s, partition %d, hwm %d, local offset %d", p.topic, p.partition, hwm, storedOffset)) + return + } + + if storedOffset >= hwm { + p.log.Printf("Error: local offset is higher than partition offset. topic %s, partition %d, hwm %d, local offset %d. This can have several reasons: \n(1) The kafka topic storing the table is gone --> delete the local cache and restart! \n(2) the processor crashed last time while writing to disk. \n(3) You found a bug!", p.topic, p.partition, hwm, storedOffset) + + // we'll just pretend we were done so the partition looks recovered + loadOffset = hwm + } + + // initialize recovery stats here, in case we don't do the recovery because + // we're up to date already + if stopAfterCatchup { + p.stats.Recovery.StartTime = time.Now() + p.stats.Recovery.Hwm = hwm + p.stats.Recovery.Offset = loadOffset + } + + // we are exactly where we're supposed to be + // AND we're here for catchup, so let's stop here + // and do not attempt to load anything + if stopAfterCatchup && loadOffset >= hwm { + errs.Collect(p.markRecovered(ctx)) + return + } + + if stopAfterCatchup { + p.log.Debugf("Recovering from %d to hwm=%d; (local offset is %d)", loadOffset, hwm, storedOffset) + } else { + p.log.Debugf("Catching up from %d to hwm=%d; (local offset is %d)", loadOffset, hwm, storedOffset) + } + + defer p.log.Debugf("... Loading done") + + if stopAfterCatchup { + p.state.SetState(State(PartitionRecovering)) + } + + partConsumer, err = p.consumer.ConsumePartition(p.topic, p.partition, loadOffset) + if err != nil { + errs.Collect(fmt.Errorf("Error creating partition consumer for topic %s, partition %d, offset %d: %v", p.topic, p.partition, storedOffset, err)) + return + } + + // consume errors asynchronously + go p.handleConsumerErrors(ctx, errs, partConsumer) + + // close the consumer + defer func() { + partConsumer.AsyncClose() + p.drainConsumer(partConsumer, errs) + }() + + // load messages and stop when you're at HWM + loadErr := p.loadMessages(ctx, partConsumer, hwm, stopAfterCatchup) + + if loadErr != nil { + errs.Collect(loadErr) + return + } + + if stopAfterCatchup { + errs.Collect(p.markRecovered(ctx)) + p.stats.Recovery.RecoveryTime = time.Now() + } + return +} + +func (p *PartitionTable) markRecovered(ctx context.Context) error { + var ( + start = time.Now() + ticker = time.NewTicker(10 * time.Second) + done = make(chan error, 1) + ) + defer ticker.Stop() + + p.state.SetState(State(PartitionPreparing)) + p.stats.Recovery.RecoveryTime = time.Now() + + go func() { + defer close(done) + err := p.st.MarkRecovered() + if err != nil { + done <- err + } + }() + + for { + select { + case <-ticker.C: + p.log.Printf("Committing storage after recovery for topic/partition %s/%d since %0.f seconds", p.topic, p.partition, time.Since(start).Seconds()) + case <-ctx.Done(): + return nil + case err := <-done: + if err != nil { + return err + } + p.state.SetState(State(PartitionRunning)) + return nil + } + } +} + +func (p *PartitionTable) handleConsumerErrors(ctx context.Context, errs *multierr.Errors, cons sarama.PartitionConsumer) { + for { + select { + case consError, ok := <-cons.Errors(): + if !ok { + return + } + err := fmt.Errorf("Consumer error: %v", consError) + p.log.Printf("%v", err) + errs.Collect(err) + // if there's an error, close the consumer + cons.AsyncClose() + case <-ctx.Done(): + return + } + } +} + +func (p *PartitionTable) drainConsumer(cons sarama.PartitionConsumer, errs *multierr.Errors) { + + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + errg, ctx := multierr.NewErrGroup(ctx) + + // drain errors channel + errg.Go(func() error { + for { + select { + case <-ctx.Done(): + p.log.Printf("draining errors channel timed out") + return nil + case err, ok := <-cons.Errors(): + if !ok { + return nil + } + errs.Collect(err) + } + return nil + } + }) + + // drain message channel + errg.Go(func() error { + for { + select { + case <-ctx.Done(): + p.log.Printf("draining messages channel timed out") + return nil + case _, ok := <-cons.Messages(): + if !ok { + return nil + } + } + return nil + } + }) + + errg.Wait() +} + +func (p *PartitionTable) loadMessages(ctx context.Context, cons sarama.PartitionConsumer, partitionHwm int64, stopAfterCatchup bool) (rerr error) { + errs := new(multierr.Errors) + + // deferred error handling + defer func() { + errs.Collect(rerr) + + rerr = errs.NilOrError() + return + }() + + stallTicker := time.NewTicker(p.stallPeriod) + defer stallTicker.Stop() + + updateHwmStatsTicker := time.NewTicker(statsHwmUpdateInterval) + defer updateHwmStatsTicker.Stop() + + lastMessage := time.Now() + + for { + select { + case msg, ok := <-cons.Messages(): + if !ok { + return + } + + // This case is for the Tester to achieve synchronity. + // Nil messages are never generated by the Sarama Consumer + if msg == nil { + continue + } + + if p.state.IsState(State(PartitionRunning)) && stopAfterCatchup { + // TODO: should we really ignore the message? + // Shouldn't we instead break here to avoid losing messages or fail or just consume it? + p.log.Printf("received message in topic %s, partition %s after catchup. Another processor is still producing messages. Ignoring message.", p.topic, p.partition) + continue + } + + lastMessage = time.Now() + if err := p.storeEvent(string(msg.Key), msg.Value, msg.Offset); err != nil { + errs.Collect(fmt.Errorf("load: error updating storage: %v", err)) + return + } + + if stopAfterCatchup { + p.stats.Recovery.Offset = msg.Offset + } + + p.trackIncomingMessageStats(msg) + + if stopAfterCatchup && msg.Offset >= partitionHwm-1 { + return + } + + case now := <-stallTicker.C: + // only set to stalled, if the last message was earlier + // than the stalled timeout + if now.Sub(lastMessage) > p.stalledTimeout { + p.stats.Stalled = true + } + + case <-p.requestStats: + p.handleStatsRequest(ctx) + + case <-updateHwmStatsTicker.C: + p.updateHwmStats() + + case <-ctx.Done(): + return + } + } +} + +func (p *PartitionTable) handleStatsRequests(ctx context.Context) { + for { + select { + case <-p.requestStats: + p.handleStatsRequest(ctx) + case <-ctx.Done(): + return + } + } +} + +func (p *PartitionTable) handleStatsRequest(ctx context.Context) { + stats := p.stats.clone() + stats.Status = PartitionStatus(p.state.State()) + select { + case p.responseStats <- stats: + case <-ctx.Done(): + p.log.Debugf("exiting, context is cancelled") + } +} + +func (p *PartitionTable) fetchStats(ctx context.Context) *TableStats { + select { + case <-ctx.Done(): + return nil + case <-time.After(fetchStatsTimeout): + p.log.Printf("requesting stats timed out") + return nil + case p.requestStats <- true: + } + + // retrieve from response-channel + select { + case <-ctx.Done(): + return nil + case <-time.After(fetchStatsTimeout): + p.log.Printf("fetching stats timed out") + return nil + case stats := <-p.responseStats: + return stats + } +} + +func (p *PartitionTable) trackIncomingMessageStats(msg *sarama.ConsumerMessage) { + ip := p.stats.Input + ip.Bytes += len(msg.Value) + ip.LastOffset = msg.Offset + if !msg.Timestamp.IsZero() { + ip.Delay = time.Since(msg.Timestamp) + } + ip.Count++ + p.stats.Stalled = false +} + +func (p *PartitionTable) trackMessageWrite(length int) { + p.stats.Writes.Bytes += length + p.stats.Writes.Count++ +} + +func (p *PartitionTable) updateHwmStats() { + hwms := p.consumer.HighWaterMarks() + hwm := hwms[p.topic][p.partition] + if hwm != 0 { + p.stats.Input.OffsetLag = hwm - p.stats.Input.LastOffset + } +} + +func (p *PartitionTable) storeEvent(key string, value []byte, offset int64) error { + err := p.st.Update(key, value) + if err != nil { + return fmt.Errorf("Error from the update callback while recovering from the log: %v", err) + } + err = p.st.SetOffset(offset) + if err != nil { + return fmt.Errorf("Error updating offset in local storage while recovering from the log: %v", err) + } + return nil +} + +// IsRecovered returns whether the partition table is recovered +func (p *PartitionTable) IsRecovered() bool { + return p.state.IsState(State(PartitionRunning)) +} + +// WaitRecovered returns a channel that closes when the partition table enters state `PartitionRunning` +func (p *PartitionTable) WaitRecovered() chan struct{} { + return p.state.WaitForState(State(PartitionRunning)) +} + +// Get returns the value for passed key +func (p *PartitionTable) Get(key string) ([]byte, error) { + return p.st.Get(key) +} + +// Set sets a key value key in the partition table by modifying the underlying storage +func (p *PartitionTable) Set(key string, value []byte) error { + return p.st.Set(key, value) +} + +// Delete removes the passed key from the partition table by deleting from the underlying storage +func (p *PartitionTable) Delete(key string) error { + return p.st.Delete(key) +} + +func (p *PartitionTable) storeNewestOffset(newOffset int64) error { + p.offsetM.Lock() + defer p.offsetM.Unlock() + + oldOffset, err := p.GetOffset(offsetNotStored) + if err != nil { + return err + } + + if offsetNotStored != oldOffset && oldOffset <= newOffset { + return p.SetOffset(newOffset) + } + return nil +} + +func (p *PartitionTable) SetOffset(value int64) error { + return p.st.SetOffset(value) +} + +func (p *PartitionTable) GetOffset(defValue int64) (int64, error) { + return p.st.GetOffset(defValue) +} diff --git a/partition_table_test.go b/partition_table_test.go new file mode 100644 index 00000000..131d5bdc --- /dev/null +++ b/partition_table_test.go @@ -0,0 +1,982 @@ +package goka + +import ( + "context" + "fmt" + "sync" + "testing" + "time" + + "github.com/Shopify/sarama" + "github.com/golang/mock/gomock" + "github.com/lovoo/goka/internal/test" + "github.com/lovoo/goka/logger" + "github.com/lovoo/goka/storage" +) + +func defaultPT( + t *testing.T, + topic string, + partition int32, + consumer sarama.Consumer, + updateCallback UpdateCallback, +) (*PartitionTable, *builderMock, *gomock.Controller) { + + ctrl := gomock.NewController(t) + bm := newBuilderMock(ctrl) + return newPartitionTable( + topic, + partition, + consumer, + bm.tmgr, + updateCallback, + bm.getStorageBuilder(), + logger.Default(), + ), bm, ctrl +} + +func TestPT_createStorage(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + var ( + partition int32 = 101 + callback UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + return nil + } + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + partition, + nil, + callback, + ) + defer ctrl.Finish() + + equalSP := &storageProxy{ + Storage: bm.mst, + partition: partition, + update: callback, + } + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + sp, err := pt.createStorage(ctx) + test.AssertNil(t, err) + test.AssertEqual(t, sp.Storage, equalSP.Storage) + test.AssertEqual(t, sp.partition, equalSP.partition) + test.AssertFuncEqual(t, sp.Update, equalSP.Update) + }) + t.Run("fail_ctx_cancel", func(t *testing.T) { + pt, _, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + pt.builder = errStorageBuilder() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*1) + cancel() + sp, err := pt.createStorage(ctx) + test.AssertNil(t, err) + test.AssertNil(t, sp) + }) + t.Run("fail_storage", func(t *testing.T) { + pt, _, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + pt.builder = errStorageBuilder() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*1) + defer cancel() + sp, err := pt.createStorage(ctx) + test.AssertNotNil(t, err) + test.AssertNil(t, sp) + }) +} + +func TestPT_setup(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + pt, _, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + }) + t.Run("fail", func(t *testing.T) { + pt, _, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + pt.builder = errStorageBuilder() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*1) + defer cancel() + err := pt.setup(ctx) + test.AssertNotNil(t, err) + }) +} + +func TestPT_close(t *testing.T) { + t.Run("on_storage", func(t *testing.T) { + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + bm.mst.EXPECT().Close().AnyTimes() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.Close() + test.AssertNil(t, err) + }) + t.Run("on_nil_storage", func(t *testing.T) { + pt, _, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + + err := pt.Close() + test.AssertNil(t, err) + }) +} + +func TestPT_findOffsetToLoad(t *testing.T) { + t.Run("old_local", func(t *testing.T) { + var ( + oldest int64 = 161 + newest int64 = 1312 + local int64 = 15 + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil) + + actualOldest, actualNewest, err := pt.findOffsetToLoad(local) + test.AssertNil(t, err) + test.AssertEqual(t, actualOldest, oldest) + test.AssertEqual(t, actualNewest, newest) + }) + t.Run("new_local", func(t *testing.T) { + var ( + oldest int64 = 161 + newest int64 = 1312 + local int64 = 175 + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil) + + offsetToLoad, actualNewest, err := pt.findOffsetToLoad(local) + test.AssertNil(t, err) + test.AssertEqual(t, offsetToLoad, local+1) + test.AssertEqual(t, actualNewest, newest) + }) + t.Run("too_new_local", func(t *testing.T) { + var ( + oldest int64 = 161 + newest int64 = 1312 + local int64 = 161111 + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil) + + offsetToLoad, actualNewest, err := pt.findOffsetToLoad(local) + test.AssertNil(t, err) + test.AssertEqual(t, offsetToLoad, local+1) + test.AssertEqual(t, actualNewest, newest) + }) + t.Run("sarama_oldest", func(t *testing.T) { + var ( + oldest int64 = 161 + newest int64 = 1312 + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil) + + actualOldest, actualNewest, err := pt.findOffsetToLoad(sarama.OffsetOldest) + test.AssertNil(t, err) + test.AssertEqual(t, actualOldest, oldest) + test.AssertEqual(t, actualNewest, newest) + }) + t.Run("fail_getoffset", func(t *testing.T) { + var ( + expectedErr error = fmt.Errorf("some error") + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(int64(0), expectedErr) + + _, _, err := pt.findOffsetToLoad(sarama.OffsetOldest) + test.AssertNotNil(t, err) + }) + t.Run("fail_getoffset2", func(t *testing.T) { + var ( + oldest int64 = 161 + expectedErr error = fmt.Errorf("some error") + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(int64(0), expectedErr) + + _, _, err := pt.findOffsetToLoad(sarama.OffsetOldest) + test.AssertNotNil(t, err) + }) +} + +func TestPT_load(t *testing.T) { + t.Run("succeed_no_load_stopAfterCatchup", func(t *testing.T) { + var ( + oldest int64 = 161 + newest int64 = 1312 + local int64 = 1311 + stopAfterCatchup = true + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + bm.mst.EXPECT().GetOffset(offsetNotStored).Return(local, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil) + bm.mst.EXPECT().MarkRecovered().Return(nil) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.load(ctx, stopAfterCatchup) + test.AssertNil(t, err) + test.AssertTrue(t, pt.state.IsState(State(PartitionRunning))) + }) + t.Run("local_offset_too_high_stopAfterCatchup_no_error", func(t *testing.T) { + var ( + oldest int64 = 161 + newest int64 = 1312 + local int64 = 1314 + stopAfterCatchup = true + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + nil, + ) + defer ctrl.Finish() + bm.mst.EXPECT().GetOffset(offsetNotStored).Return(local, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil) + bm.mst.EXPECT().MarkRecovered().Return(nil) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.load(ctx, stopAfterCatchup) + test.AssertNil(t, err) + }) + t.Run("consume", func(t *testing.T) { + var ( + oldest int64 = 161 + newest int64 = 1312 + local int64 = sarama.OffsetOldest + stopAfterCatchup = false + consumer = defaultSaramaAutoConsumerMock(t) + topic = "some-topic" + partition int32 + count int32 + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + count++ + return nil + } + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + updateCB, + ) + defer ctrl.Finish() + pt.consumer = consumer + bm.mst.EXPECT().GetOffset(gomock.Any()).Return(local, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil) + partConsumer := consumer.ExpectConsumePartition(topic, partition, AnyOffset) + partConsumer.ExpectMessagesDrainedOnClose() + for i := 0; i < 10; i++ { + partConsumer.YieldMessage(&sarama.ConsumerMessage{}) + bm.mst.EXPECT().SetOffset(gomock.Any()).Return(nil) + } + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + go func() { + for { + select { + case <-ctx.Done(): + return + default: + } + if count == 10 { + cancel() + return + } + } + }() + + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.load(ctx, stopAfterCatchup) + test.AssertNil(t, err) + test.AssertTrue(t, count == 10) + }) +} + +func TestPT_loadMessages(t *testing.T) { + t.Run("consume_till_hwm", func(t *testing.T) { + var ( + localOffset int64 = sarama.OffsetOldest + partitionHwm int64 = 1 + stopAfterCatchup bool = true + topic string = "some-topic" + partition int32 = 0 + consumer *MockAutoConsumer = defaultSaramaAutoConsumerMock(t) + recKey string + recVal []byte + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + recKey = key + recVal = value + return nil + } + key string = "some-key" + value []byte = []byte("some-vale") + ) + pt, bm, ctrl := defaultPT( + t, + "some-topic", + 0, + nil, + updateCB, + ) + defer ctrl.Finish() + partConsumer := consumer.ExpectConsumePartition(topic, partition, localOffset) + partConsumer.YieldMessage(&sarama.ConsumerMessage{ + Key: []byte(key), + Value: value, + Topic: topic, + Partition: partition, + Offset: partitionHwm, + }) + partConsumer.ExpectMessagesDrainedOnClose() + bm.mst.EXPECT().SetOffset(int64(0)).Return(nil) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.loadMessages(ctx, partConsumer, partitionHwm, stopAfterCatchup) + test.AssertNil(t, err) + test.AssertEqual(t, recKey, key) + test.AssertEqual(t, recVal, value) + }) + t.Run("consume_till_hwm_more_msgs", func(t *testing.T) { + var ( + localOffset int64 = 0 + partitionHwm int64 = 2 + stopAfterCatchup bool = true + topic string = "some-topic" + partition int32 = 0 + consumer *MockAutoConsumer = defaultSaramaAutoConsumerMock(t) + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + return nil + } + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + updateCB, + ) + defer ctrl.Finish() + partConsumer := consumer.ExpectConsumePartition(topic, partition, localOffset) + partConsumer.YieldMessage(&sarama.ConsumerMessage{ + Topic: topic, + Partition: partition, + Offset: 1, + }) + partConsumer.YieldMessage(&sarama.ConsumerMessage{ + Topic: topic, + Partition: partition, + Offset: 1, + }) + partConsumer.ExpectMessagesDrainedOnClose() + bm.mst.EXPECT().SetOffset(int64(0)).Return(nil) + bm.mst.EXPECT().SetOffset(int64(1)).Return(nil) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.loadMessages(ctx, partConsumer, partitionHwm, stopAfterCatchup) + test.AssertNil(t, err) + }) + t.Run("consume_till_cancel", func(t *testing.T) { + var ( + localOffset int64 = 0 + partitionHwm int64 = 2 + stopAfterCatchup bool = false + topic string = "some-topic" + partition int32 = 0 + consumer *MockAutoConsumer = defaultSaramaAutoConsumerMock(t) + count int32 = 0 + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + count++ + return nil + } + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + updateCB, + ) + defer ctrl.Finish() + partConsumer := consumer.ExpectConsumePartition(topic, partition, localOffset) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + go func(ctx context.Context) { + for i := 0; i < 100; i++ { + bm.mst.EXPECT().SetOffset(gomock.Any()).Return(nil) + partConsumer.YieldMessage(&sarama.ConsumerMessage{}) + } + for { + select { + case <-ctx.Done(): + return + default: + } + if count == 100 { + break + } + } + cancel() + }(ctx) + err = pt.loadMessages(ctx, partConsumer, partitionHwm, stopAfterCatchup) + test.AssertNil(t, err) + test.AssertTrue(t, count == 100) + }) + t.Run("close_msg_chan", func(t *testing.T) { + var ( + localOffset int64 = 0 + partitionHwm int64 = 2 + stopAfterCatchup bool = false + topic string = "some-topic" + partition int32 = 0 + consumer *MockAutoConsumer = defaultSaramaAutoConsumerMock(t) + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + return nil + } + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + updateCB, + ) + defer ctrl.Finish() + partConsumer := consumer.ExpectConsumePartition(topic, partition, localOffset) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + err := pt.setup(ctx) + test.AssertNil(t, err) + go func() { + defer cancel() + err = pt.loadMessages(ctx, partConsumer, partitionHwm, stopAfterCatchup) + test.AssertNil(t, err) + }() + go func(ctx context.Context) { + lock := sync.Mutex{} + open := true + go func() { + lock.Lock() + defer lock.Unlock() + partConsumer.AsyncClose() + open = false + }() + for i := 0; i < 100; i++ { + select { + case <-ctx.Done(): + return + default: + } + lock.Lock() + if open { + bm.mst.EXPECT().SetOffset(gomock.Any()).Return(nil) + partConsumer.YieldMessage(&sarama.ConsumerMessage{}) + } + lock.Unlock() + } + }(ctx) + <-ctx.Done() + }) + t.Run("stalled", func(t *testing.T) { + var ( + localOffset int64 = 0 + partitionHwm int64 = 2 + stopAfterCatchup bool = false + topic string = "some-topic" + partition int32 = 0 + consumer *MockAutoConsumer = defaultSaramaAutoConsumerMock(t) + ) + pt, _, ctrl := defaultPT( + t, + topic, + partition, + nil, + nil, + ) + defer ctrl.Finish() + pt.stalledTimeout = time.Duration(0) + pt.stallPeriod = time.Nanosecond + + partConsumer := consumer.ExpectConsumePartition(topic, partition, localOffset) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + err := pt.setup(ctx) + test.AssertNil(t, err) + go func() { + defer cancel() + for { + select { + case <-ctx.Done(): + return + default: + } + if pt.stats.Stalled { + return + } + } + }() + err = pt.loadMessages(ctx, partConsumer, partitionHwm, stopAfterCatchup) + test.AssertNil(t, err) + }) + t.Run("fail", func(t *testing.T) { + var ( + localOffset int64 = 0 + partitionHwm int64 = 2 + stopAfterCatchup bool = true + topic string = "some-topic" + partition int32 = 0 + consumer *MockAutoConsumer = defaultSaramaAutoConsumerMock(t) + retErr error = fmt.Errorf("update error") + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + return retErr + } + ) + pt, _, ctrl := defaultPT( + t, + topic, + partition, + nil, + updateCB, + ) + defer ctrl.Finish() + partConsumer := consumer.ExpectConsumePartition(topic, partition, localOffset) + partConsumer.YieldMessage(&sarama.ConsumerMessage{ + Topic: topic, + Partition: partition, + Offset: 1, + }) + partConsumer.ExpectMessagesDrainedOnClose() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.loadMessages(ctx, partConsumer, partitionHwm, stopAfterCatchup) + test.AssertNotNil(t, err) + }) +} + +func TestPT_storeEvent(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + var ( + localOffset int64 = 0 + partition int32 = 0 + topic string = "some-topic" + key string = "some-key" + value []byte = []byte("some-vale") + actualKey string + actualValue []byte + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + actualKey = key + actualValue = value + return nil + } + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + updateCB, + ) + defer ctrl.Finish() + bm.mst.EXPECT().SetOffset(localOffset).Return(nil) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.storeEvent(key, value, localOffset) + test.AssertEqual(t, actualKey, key) + test.AssertEqual(t, actualValue, value) + test.AssertNil(t, err) + }) + t.Run("fail", func(t *testing.T) { + var ( + localOffset int64 = 0 + partition int32 = 0 + topic string = "some-topic" + key string = "some-key" + value []byte = []byte("some-vale") + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + return nil + } + retErr error = fmt.Errorf("storage err") + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + updateCB, + ) + defer ctrl.Finish() + bm.mst.EXPECT().SetOffset(localOffset).Return(retErr) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.storeEvent(key, value, localOffset) + test.AssertNotNil(t, err) + }) +} + +func TestPT_Close(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + var ( + partition int32 = 0 + topic string = "some-topic" + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + nil, + ) + defer ctrl.Finish() + bm.mst.EXPECT().Close().Return(nil) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.Close() + test.AssertNil(t, err) + }) + t.Run("succeed2", func(t *testing.T) { + var ( + partition int32 = 0 + topic string = "some-topic" + ) + pt, _, ctrl := defaultPT( + t, + topic, + partition, + nil, + nil, + ) + defer ctrl.Finish() + err := pt.Close() + test.AssertNil(t, err) + }) +} + +func TestPT_markRecovered(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + var ( + partition int32 = 0 + topic string = "some-topic" + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + nil, + ) + defer ctrl.Finish() + bm.mst.EXPECT().MarkRecovered().Return(nil) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + test.AssertTrue(t, !pt.state.IsState(State(PartitionRunning))) + err = pt.markRecovered(ctx) + test.AssertNil(t, err) + test.AssertTrue(t, pt.state.IsState(State(PartitionRunning))) + }) + t.Run("fail", func(t *testing.T) { + var ( + partition int32 = 0 + topic string = "some-topic" + retErr error = fmt.Errorf("store error") + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + nil, + ) + defer ctrl.Finish() + bm.mst.EXPECT().MarkRecovered().Return(retErr) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + err := pt.setup(ctx) + test.AssertNil(t, err) + err = pt.markRecovered(ctx) + test.AssertNotNil(t, err) + test.AssertTrue(t, pt.state.IsState(State(PartitionPreparing))) + }) +} + +func TestPT_SetupAndCatchupToHwm(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + var ( + oldest int64 = 0 + newest int64 = 5 + local = oldest + consumer = defaultSaramaAutoConsumerMock(t) + topic = "some-topic" + partition int32 + count int64 + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + count++ + return nil + } + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + updateCB, + ) + logger.Debug(true, false) + defer ctrl.Finish() + pt.consumer = consumer + bm.mst.EXPECT().GetOffset(gomock.Any()).Return(local, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil) + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil) + bm.mst.EXPECT().MarkRecovered().Return(nil) + partConsumer := consumer.ExpectConsumePartition(topic, partition, local+1) + partConsumer.ExpectMessagesDrainedOnClose() + + msgsToRecover := newest - local + for i := int64(0); i < msgsToRecover; i++ { + partConsumer.YieldMessage(&sarama.ConsumerMessage{}) + bm.mst.EXPECT().SetOffset(gomock.Any()).Return(nil) + } + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + defer cancel() + + err := pt.SetupAndRecover(ctx) + test.AssertNil(t, err) + test.AssertTrue(t, count == msgsToRecover) + }) + t.Run("fail", func(t *testing.T) { + var ( + consumer = defaultSaramaAutoConsumerMock(t) + topic = "some-topic" + partition int32 + retErr = fmt.Errorf("offset-error") + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + nil, + ) + defer ctrl.Finish() + pt.consumer = consumer + bm.mst.EXPECT().GetOffset(gomock.Any()).Return(int64(0), retErr) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + + err := pt.SetupAndRecover(ctx) + test.AssertNotNil(t, err) + }) +} + +func TestPT_SetupAndCatchupForever(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + var ( + oldest int64 = 0 + newest int64 = 10 + // local int64 = oldest + consumer = defaultSaramaAutoConsumerMock(t) + topic = "some-topic" + partition int32 = 0 + count int64 = 0 + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + count++ + return nil + } + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + updateCB, + ) + defer ctrl.Finish() + bm.useMemoryStorage() + pt.consumer = consumer + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil).AnyTimes() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil).AnyTimes() + partConsumer := consumer.ExpectConsumePartition(topic, partition, AnyOffset) + for i := 0; i < 10; i++ { + partConsumer.YieldMessage(&sarama.ConsumerMessage{}) + } + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + go func() { + for { + select { + case <-ctx.Done(): + return + default: + } + if count == 10 { + time.Sleep(time.Millisecond * 10) + cancel() + return + } + } + }() + + err := pt.SetupAndRecover(ctx) + test.AssertNil(t, err) + cancel() + }) + t.Run("fail", func(t *testing.T) { + var ( + consumer = defaultSaramaAutoConsumerMock(t) + topic = "some-topic" + partition int32 = 0 + retErr = fmt.Errorf("offset-error") + ) + pt, bm, ctrl := defaultPT( + t, + topic, + partition, + nil, + nil, + ) + defer ctrl.Finish() + pt.consumer = consumer + bm.mst.EXPECT().GetOffset(gomock.Any()).Return(int64(0), retErr) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + + err := pt.SetupAndRecover(ctx) + test.AssertNotNil(t, err) + cancel() + }) +} diff --git a/partition_test.go b/partition_test.go deleted file mode 100644 index d97a0161..00000000 --- a/partition_test.go +++ /dev/null @@ -1,916 +0,0 @@ -package goka - -import ( - "context" - "errors" - "log" - "sync" - "sync/atomic" - "testing" - "time" - - "github.com/lovoo/goka/kafka" - "github.com/lovoo/goka/logger" - "github.com/lovoo/goka/mock" - "github.com/lovoo/goka/storage" - - "github.com/facebookgo/ensure" - "github.com/golang/mock/gomock" -) - -const ( - group = "group" - topic = "topic" -) - -func newStorageProxy(st storage.Storage, id int32, update UpdateCallback) *storageProxy { - return &storageProxy{ - Storage: st, - partition: id, - update: update, - } -} - -func newNullStorageProxy(id int32) *storageProxy { - return &storageProxy{ - Storage: storage.NewMemory(), - partition: id, - stateless: true, - } -} - -func TestNewPartition(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - p := newPartition(logger.Default(), topic, nil, nil, nil, defaultPartitionChannelSize) - ensure.True(t, p != nil) -} - -func TestPartition_startStateless(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - wait = make(chan bool) - final = make(chan bool) - ctx, cancel = context.WithCancel(context.Background()) - ) - - p := newPartition(logger.Default(), topic, nil, newNullStorageProxy(0), proxy, defaultPartitionChannelSize) - proxy.EXPECT().AddGroup().Do(func() { close(wait) }) - proxy.EXPECT().Stop() - - go func() { - err := p.start(ctx) - ensure.Nil(t, err) - close(final) - }() - - err := doTimed(t, func() { - <-wait - cancel() - <-final - }) - ensure.Nil(t, err) -} - -func TestPartition_startStateful(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - st = mock.NewMockStorage(ctrl) - offset = int64(123) - wait = make(chan bool) - ctx, cancel = context.WithCancel(context.Background()) - ) - - p := newPartition(logger.Default(), topic, nil, newStorageProxy(st, 0, nil), proxy, defaultPartitionChannelSize) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(offset, nil), - proxy.EXPECT().Add(topic, int64(offset)), - proxy.EXPECT().Remove(topic), - proxy.EXPECT().Stop(), - ) - go func() { - err := p.start(ctx) - ensure.Nil(t, err) - close(wait) - }() - - err := doTimed(t, func() { - time.Sleep(100 * time.Millisecond) - cancel() - <-wait - }) - ensure.Nil(t, err) -} - -func TestPartition_runStateless(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait = make(chan bool) - step = make(chan bool) - ctx, cancel = context.WithCancel(context.Background()) - count int64 - ) - - consume := func(msg *message, st storage.Storage, wg *sync.WaitGroup, pstats *PartitionStats) (int, error) { - atomic.AddInt64(&count, 1) - ensure.DeepEqual(t, msg.Key, string(key)) - ensure.DeepEqual(t, msg.Data, value) - step <- true - return 0, nil - } - - p := newPartition(logger.Default(), topic, consume, newNullStorageProxy(0), proxy, defaultPartitionChannelSize) - - proxy.EXPECT().AddGroup() - proxy.EXPECT().Stop() - go func() { - err := p.start(ctx) - ensure.Nil(t, err) - close(wait) - }() - - // message will be processed - p.ch <- &kafka.Message{ - Key: key, - Offset: offset, - Partition: par, - Topic: "some-other-topic", - Value: value, - } - - // garbage will be dropped - p.ch <- new(kafka.NOP) - - err := doTimed(t, func() { - <-step - cancel() - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(1)) - <-wait - }) - ensure.Nil(t, err) -} - -func TestPartition_runStatelessWithError(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait = make(chan bool) - count int64 - ) - - consume := func(msg *message, st storage.Storage, wg *sync.WaitGroup, pstats *PartitionStats) (int, error) { - atomic.AddInt64(&count, 1) - return 0, nil - } - - p := newPartition(logger.Default(), topic, consume, newNullStorageProxy(0), proxy, defaultPartitionChannelSize) - - proxy.EXPECT().AddGroup() - proxy.EXPECT().Stop() - go func() { - err := p.start(context.Background()) - ensure.NotNil(t, err) - close(wait) - }() - - // message causes error dropped because topic is the same as group table topic - p.ch <- &kafka.Message{ - Key: key + "something", - Offset: offset - 1, - Partition: par, - Topic: topic, - Value: value, - } - - err := doTimed(t, func() { - <-wait - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(0)) - }) - ensure.Nil(t, err) - - // test sending error into the channel - p = newPartition(logger.Default(), topic, consume, newNullStorageProxy(0), proxy, defaultPartitionChannelSize) - wait = make(chan bool) - - proxy.EXPECT().AddGroup() - proxy.EXPECT().Stop() - go func() { - err = p.start(context.Background()) - ensure.NotNil(t, err) - close(wait) - }() - - p.ch <- &kafka.Error{} - - err = doTimed(t, func() { - <-wait - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(0)) - }) - ensure.Nil(t, err) - -} - -func TestPartition_runStateful(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - st = mock.NewMockStorage(ctrl) - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait = make(chan bool) - step = make(chan bool) - ctx, cancel = context.WithCancel(context.Background()) - count int64 - ) - - consume := func(msg *message, st storage.Storage, wg *sync.WaitGroup, pstats *PartitionStats) (int, error) { - atomic.AddInt64(&count, 1) - ensure.DeepEqual(t, msg.Key, string(key)) - ensure.DeepEqual(t, msg.Data, value) - step <- true - return 0, nil - } - - p := newPartition(logger.Default(), topic, consume, newStorageProxy(st, 0, nil), proxy, 0) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(offset), nil), - proxy.EXPECT().Add(topic, offset), - st.EXPECT().MarkRecovered(), - proxy.EXPECT().Remove(topic), - proxy.EXPECT().AddGroup(), - proxy.EXPECT().Stop(), - ) - - go func() { - err := p.start(ctx) - log.Printf("%v", err) - ensure.Nil(t, err) - close(wait) - }() - - // partition should be marked recovered after the HWM or EOF message - ensure.False(t, p.recovered()) - p.ch <- &kafka.BOF{ - Partition: par, - Topic: topic, - Offset: offset, - Hwm: offset, - } - - // message will terminate load - p.ch <- &kafka.EOF{ - Partition: par, - Topic: topic, - Hwm: offset, - } - p.ch <- new(kafka.NOP) - ensure.True(t, p.recovered()) - - // message will be processed - p.ch <- &kafka.Message{ - Key: key, - Offset: offset, - Partition: par, - Topic: "some-other-topic", - Value: value, - } - - err := doTimed(t, func() { - <-step - cancel() - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(1)) - <-wait - }) - ensure.Nil(t, err) -} - -func TestPartition_runStatefulWithError(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - st = mock.NewMockStorage(ctrl) - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait = make(chan bool) - step = make(chan bool) - count int64 - ) - - consume := func(msg *message, st storage.Storage, wg *sync.WaitGroup, pstats *PartitionStats) (int, error) { - if msg.Topic == "error" { - return 0, errors.New("some error") - } - atomic.AddInt64(&count, 1) - ensure.DeepEqual(t, msg.Key, string(key)) - ensure.DeepEqual(t, msg.Data, value) - step <- true - return 0, nil - } - - p := newPartition(logger.Default(), topic, consume, newStorageProxy(st, 0, nil), proxy, defaultPartitionChannelSize) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(offset), nil), - proxy.EXPECT().Add(topic, offset), - st.EXPECT().MarkRecovered(), - proxy.EXPECT().Remove(topic), - proxy.EXPECT().AddGroup(), - proxy.EXPECT().Stop(), - ) - - go func() { - err := p.start(context.Background()) - ensure.NotNil(t, err) - close(wait) - }() - - // message will terminate load - p.ch <- &kafka.EOF{ - Partition: par, - Topic: topic, - Hwm: offset, - } - - // message will be processed - p.ch <- &kafka.Message{ - Key: key, - Offset: offset, - Partition: par, - Topic: "some-other-topic", - Value: value, - } - - // message will generate an error and will return the goroutine - p.ch <- &kafka.Message{ - Key: key, - Offset: offset, - Partition: par, - Topic: "error", - Value: value, - } - - err := doTimed(t, func() { - <-step - <-wait - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(1)) - }) - ensure.Nil(t, err) -} - -func TestPartition_loadStateful(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - st = mock.NewMockStorage(ctrl) - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait = make(chan bool) - step = make(chan bool) - ctx, cancel = context.WithCancel(context.Background()) - count int64 - ) - - consume := func(msg *message, st storage.Storage, wg *sync.WaitGroup, pstats *PartitionStats) (int, error) { - atomic.AddInt64(&count, 1) - ensure.DeepEqual(t, msg.Key, string(key)) - ensure.DeepEqual(t, msg.Topic, "some-other-topic") - ensure.DeepEqual(t, msg.Data, value) - step <- true - return 0, nil - } - - p := newPartition(logger.Default(), topic, consume, newStorageProxy(st, 0, DefaultUpdate), proxy, defaultPartitionChannelSize) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(offset), nil), - proxy.EXPECT().Add(topic, offset), - st.EXPECT().Set(key, value), - st.EXPECT().SetOffset(int64(offset)).Return(nil), - st.EXPECT().MarkRecovered(), - proxy.EXPECT().Remove(topic), - proxy.EXPECT().AddGroup(), - proxy.EXPECT().Stop(), - ) - - go func() { - err := p.start(ctx) - ensure.Nil(t, err) - close(wait) - }() - - // message will be loaded (Topic is tableTopic) - p.ch <- &kafka.Message{ - Key: key, - Offset: offset, - Partition: par, - Topic: topic, - Value: value, - } - - // kafka.NOP will be dropped - p.ch <- new(kafka.NOP) - - // message will terminate load - p.ch <- &kafka.EOF{ - Partition: par, - Topic: topic, - Hwm: offset, - } - - // message will be processed (Topic is != tableTopic) - p.ch <- &kafka.Message{ - Key: key, - Offset: offset + 1, - Partition: par, - Topic: "some-other-topic", - Value: value, - } - - // kafka.NOP will be dropped - p.ch <- new(kafka.NOP) - - err := doTimed(t, func() { - <-step - cancel() - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(1)) - <-wait - }) - ensure.Nil(t, err) -} - -func TestPartition_loadStatefulWithError(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - st = mock.NewMockStorage(ctrl) - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait = make(chan bool) - count int64 - ) - - // error in update - update := func(st storage.Storage, p int32, k string, v []byte) error { - atomic.AddInt64(&count, 1) - return errors.New("some error") - } - - p := newPartition(logger.Default(), topic, nil, newStorageProxy(st, 0, update), proxy, 0) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(offset), nil), - proxy.EXPECT().Add(topic, offset), - proxy.EXPECT().Remove(topic), - proxy.EXPECT().Stop(), - ) - - go func() { - err := p.start(context.Background()) - ensure.NotNil(t, err) - close(wait) - }() - - p.ch <- &kafka.Message{ - Key: key, - Offset: offset, - Partition: par, - Topic: topic, - Value: value, - } - - err := doTimed(t, func() { - <-wait - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(1)) - }) - ensure.Nil(t, err) - - // error in SetOffset - wait = make(chan bool) - p = newPartition(logger.Default(), topic, nil, newStorageProxy(st, 0, DefaultUpdate), proxy, 0) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(offset), nil), - proxy.EXPECT().Add(topic, offset), - st.EXPECT().Set(key, value), - st.EXPECT().SetOffset(int64(offset)).Return(errors.New("some error")), - proxy.EXPECT().Remove(topic), - proxy.EXPECT().Stop(), - ) - - go func() { - err = p.start(context.Background()) - ensure.NotNil(t, err) - close(wait) - }() - - p.ch <- &kafka.Message{ - Key: key, - Offset: offset, - Partition: par, - Topic: topic, - Value: value, - } - - err = doTimed(t, func() { - <-wait - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(1)) - }) - ensure.Nil(t, err) - - // error in GetOffset - wait = make(chan bool) - p = newPartition(logger.Default(), topic, nil, newStorageProxy(st, 0, nil), proxy, 0) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(0), errors.New("some error")), - proxy.EXPECT().Stop(), - ) - - go func() { - err = p.start(context.Background()) - ensure.NotNil(t, err) - close(wait) - }() - - err = doTimed(t, func() { - <-wait - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(1)) - }) - ensure.Nil(t, err) -} - -func TestPartition_loadStatefulWithErrorAddRemovePartition(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - st = mock.NewMockStorage(ctrl) - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait chan bool - count int64 - ) - - // error in AddPartitionError - wait = make(chan bool) - p := newPartition(logger.Default(), topic, nil, newStorageProxy(st, 0, DefaultUpdate), proxy, 0) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(offset), nil), - proxy.EXPECT().Add(topic, offset).Return(errors.New("some error adding partition")), - proxy.EXPECT().Stop(), - ) - - go func() { - err := p.start(context.Background()) - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "some error") - close(wait) - }() - ensure.Nil(t, doTimed(t, func() { <-wait })) - - // error in RemovePartition - update := func(st storage.Storage, p int32, k string, v []byte) error { - atomic.AddInt64(&count, 1) - return errors.New("some error") - } - - wait = make(chan bool) - p = newPartition(logger.Default(), topic, nil, newStorageProxy(st, 0, update), proxy, 0) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(offset), nil), - proxy.EXPECT().Add(topic, offset).Return(nil), - proxy.EXPECT().Remove(topic).Return(errors.New("error while removing partition")), - proxy.EXPECT().Stop(), - ) - - go func() { - err := p.start(context.Background()) - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "some error") - ensure.StringContains(t, err.Error(), "error while removing partition") - log.Printf("%v", err) - close(wait) - }() - - p.ch <- &kafka.Message{ - Key: key, - Offset: offset, - Partition: par, - Topic: topic, - Value: value, - } - - err := doTimed(t, func() { - <-wait - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(1)) - }) - ensure.Nil(t, err) -} - -func TestPartition_catchupStateful(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - proxy = mock.NewMockkafkaProxy(ctrl) - st = mock.NewMockStorage(ctrl) - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait = make(chan bool) - step = make(chan bool) - sync = func() error { return doTimed(t, func() { <-step }) } - ctx, cancel = context.WithCancel(context.Background()) - count int64 - ) - - update := func(st storage.Storage, p int32, k string, v []byte) error { - atomic.AddInt64(&count, 1) - step <- true - return DefaultUpdate(st, p, k, v) - } - p := newPartition(logger.Default(), topic, nil, newStorageProxy(st, 0, update), proxy, 0) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(offset), nil), - proxy.EXPECT().Add(topic, offset), - st.EXPECT().Set(key, value), - st.EXPECT().SetOffset(offset).Return(nil), - st.EXPECT().Set(key, value), - st.EXPECT().SetOffset(offset+1).Return(nil), - proxy.EXPECT().Remove(topic), - st.EXPECT().MarkRecovered(), - proxy.EXPECT().Add(topic, offset+2), - st.EXPECT().Set(key, value), - st.EXPECT().SetOffset(offset+2).Return(nil), - proxy.EXPECT().Remove(topic), - proxy.EXPECT().Stop(), - ) - - go func() { - err := p.startCatchup(ctx) - ensure.Nil(t, err) - close(wait) - }() - - // beginning of file marks the beginning of topic - p.ch <- &kafka.BOF{ - Topic: topic, - Partition: par, - Offset: offset, // first offset that will arrive - Hwm: offset + 2, // highwatermark is one offset after the last one that will arrive - } - - // message will be loaded (Topic is tableTopic) - p.ch <- &kafka.Message{ - Topic: topic, - Partition: par, - Offset: offset, - Key: key, - Value: value, - } - err := sync() - ensure.Nil(t, err) - offset++ - - // message will be loaded (Topic is tableTopic) - p.ch <- &kafka.Message{ - Topic: topic, - Partition: par, - Offset: offset, - Key: key, - Value: value, - } - err = sync() - ensure.Nil(t, err) - offset++ - - // message will not terminate load (catchup modus) but will mark as - // recovered - p.ch <- &kafka.EOF{ - Topic: topic, - Partition: par, - Hwm: offset, - } - p.ch <- new(kafka.NOP) - ensure.True(t, p.recovered()) - - // message will not terminate load (catchup modus) - p.ch <- &kafka.EOF{ - Topic: topic, - Partition: par, - Hwm: offset, - } - p.ch <- new(kafka.NOP) - ensure.True(t, p.recovered()) - - // message will be loaded (Topic is tableTopic) - p.ch <- &kafka.Message{ - Topic: topic, - Partition: par, - Offset: offset, - Key: key, - Value: value, - } - err = sync() - ensure.Nil(t, err) - p.ch <- new(kafka.NOP) - - err = doTimed(t, func() { - cancel() - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(3)) - <-wait - }) - ensure.Nil(t, err) -} - -func TestPartition_catchupStatefulWithError(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - ctx, cancel = context.WithCancel(context.Background()) - proxy = mock.NewMockkafkaProxy(ctrl) - st = mock.NewMockStorage(ctrl) - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait = make(chan bool) - step = make(chan bool) - sync = func() error { return doTimed(t, func() { <-step }) } - count int64 - ) - - update := func(st storage.Storage, p int32, k string, v []byte) error { - atomic.AddInt64(&count, 1) - step <- true - return DefaultUpdate(st, p, k, v) - } - p := newPartition(logger.Default(), topic, nil, newStorageProxy(st, 0, update), proxy, 0) - - gomock.InOrder( - st.EXPECT().GetOffset(int64(-2)).Return(int64(offset), nil), - proxy.EXPECT().Add(topic, offset), - st.EXPECT().Set(key, value), - st.EXPECT().SetOffset(offset).Return(nil), - st.EXPECT().Set(key, value), - st.EXPECT().SetOffset(offset+1).Return(nil), - proxy.EXPECT().Remove(topic), - st.EXPECT().MarkRecovered(), - proxy.EXPECT().Add(topic, offset+2), - proxy.EXPECT().Remove(topic), - proxy.EXPECT().Stop(), - ) - - go func() { - err := p.startCatchup(ctx) - ensure.Nil(t, err) - close(wait) - }() - - // beginning of file marks the beginning of topic - p.ch <- &kafka.BOF{ - Topic: topic, - Partition: par, - Offset: offset, // first offset that will arrive - Hwm: offset + 2, // highwatermark is one offset after the last one that will arrive - } - - // message will be loaded (Topic is tableTopic) - p.ch <- &kafka.Message{ - Topic: topic, - Partition: par, - Offset: offset, - Key: key, - Value: value, - } - err := sync() - ensure.Nil(t, err) - offset++ - - // message will be loaded (Topic is tableTopic) - p.ch <- &kafka.Message{ - Topic: topic, - Partition: par, - Offset: offset, - Key: key, - Value: value, - } - err = sync() - ensure.Nil(t, err) - offset++ - - // message will not terminate load (catchup modus) but will mark as - // recovered - p.ch <- &kafka.EOF{ - Topic: topic, - Partition: par, - Hwm: offset, - } - p.ch <- new(kafka.NOP) - ensure.True(t, p.recovered()) - - // message will not terminate load (catchup modus) - p.ch <- &kafka.EOF{ - Topic: topic, - Partition: par, - Hwm: offset, - } - p.ch <- new(kafka.NOP) - ensure.True(t, p.recovered()) - - cancel() - err = doTimed(t, func() { - <-wait - ensure.DeepEqual(t, atomic.LoadInt64(&count), int64(2)) - }) - ensure.Nil(t, err) -} - -func BenchmarkPartition_load(b *testing.B) { - var ( - key = "key" - par int32 = 1 - offset int64 = 4 - value = []byte("value") - wait = make(chan bool) - st = storage.NewNull() - ) - - update := func(st storage.Storage, p int32, k string, v []byte) error { - return nil - } - p := newPartition(logger.Default(), topic, nil, newStorageProxy(st, 0, update), new(nullProxy), 0) - ctx, cancel := context.WithCancel(context.Background()) - go func() { - err := p.start(ctx) - if err != nil { - panic(err) - } - close(wait) - }() - - // beginning of file marks the beginning of topic - p.ch <- &kafka.BOF{ - Topic: topic, - Partition: par, - Offset: offset, - Hwm: int64(b.N + 1), - } - // run the Fib function b.N times - for n := 0; n < b.N; n++ { - p.ch <- &kafka.Message{ - Topic: topic, - Partition: par, - Offset: offset, - Key: key, - Value: value, - } - offset++ - } - cancel() - <-wait -} diff --git a/processor.go b/processor.go index b3f7dd66..5444dc0a 100644 --- a/processor.go +++ b/processor.go @@ -4,56 +4,58 @@ import ( "context" "errors" "fmt" - "runtime/debug" + "strings" "sync" "time" - "github.com/lovoo/goka/kafka" + "github.com/Shopify/sarama" "github.com/lovoo/goka/logger" "github.com/lovoo/goka/multierr" "github.com/lovoo/goka/storage" ) +const ( + ProcStateIdle State = iota + ProcStateStarting + ProcStateSetup + ProcStateRunning + ProcStateStopping +) + +// ProcessCallback function is called for every message received by the +// processor. +type ProcessCallback func(ctx Context, msg interface{}) + // Processor is a set of stateful callback functions that, on the arrival of // messages, modify the content of a table (the group table) and emit messages into other // topics. Messages as well as rows in the group table are key-value pairs. // A group is composed by multiple processor instances. type Processor struct { opts *poptions + log logger.Logger brokers []string - partitions map[int32]*partition - partitionViews map[int32]map[string]*partition + rebalanceCallback RebalanceCallback + + // Partition processors + partitions map[int32]*PartitionProcessor + // lookup tables + lookupTables map[string]*View + partitionCount int - views map[string]*View graph *GroupGraph - m sync.RWMutex - consumer kafka.Consumer - producer kafka.Producer - asCh chan kafka.Assignment + saramaConsumer sarama.Consumer + producer Producer + tmgr TopicManager - errors *multierr.Errors - cancel func() - ctx context.Context -} + state *Signal -// message to be consumed -type message struct { - Key string - Data []byte - Topic string - Partition int32 - Offset int64 - Timestamp time.Time - Header map[string][]byte + ctx context.Context + cancel context.CancelFunc } -// ProcessCallback function is called for every message received by the -// processor. -type ProcessCallback func(ctx Context, msg interface{}) - // NewProcessor creates a processor instance in a group given the address of // Kafka brokers, the consumer group name, a list of subscriptions (topics, // codecs, and callbacks), and series of options. @@ -61,6 +63,7 @@ func NewProcessor(brokers []string, gg *GroupGraph, options ...ProcessorOption) options = append( // default options comes first []ProcessorOption{ + WithClientID(fmt.Sprintf("goka-processor-%s", gg.Group())), WithLogger(logger.Default()), WithUpdateCallback(DefaultUpdate), WithPartitionChannelSize(defaultPartitionChannelSize), @@ -88,104 +91,44 @@ func NewProcessor(brokers []string, gg *GroupGraph, options ...ProcessorOption) } // create views - views := make(map[string]*View) + lookupTables := make(map[string]*View) for _, t := range gg.LookupTables() { view, err := NewView(brokers, Table(t.Topic()), t.Codec(), WithViewLogger(opts.log), WithViewHasher(opts.hasher), - WithViewPartitionChannelSize(opts.partitionChannelSize), WithViewClientID(opts.clientID), WithViewTopicManagerBuilder(opts.builders.topicmgr), WithViewStorageBuilder(opts.builders.storage), - WithViewConsumerBuilder(opts.builders.consumer), + WithViewConsumerSaramaBuilder(opts.builders.consumerSarama), ) if err != nil { return nil, fmt.Errorf("error creating view: %v", err) } - views[t.Topic()] = view + lookupTables[t.Topic()] = view } // combine things together processor := &Processor{ opts: opts, + log: opts.log.Prefix(fmt.Sprintf("Processor %s", gg.Group())), brokers: brokers, - partitions: make(map[int32]*partition), - partitionViews: make(map[int32]map[string]*partition), + rebalanceCallback: opts.rebalanceCallback, + + partitions: make(map[int32]*PartitionProcessor), partitionCount: npar, - views: views, + lookupTables: lookupTables, graph: gg, - asCh: make(chan kafka.Assignment, 1), + state: NewSignal(ProcStateIdle, ProcStateStarting, ProcStateSetup, ProcStateRunning, ProcStateStopping).SetState(ProcStateIdle), } return processor, nil } -func prepareTopics(brokers []string, gg *GroupGraph, opts *poptions) (npar int, err error) { - // create topic manager - tm, err := opts.builders.topicmgr(brokers) - if err != nil { - return 0, fmt.Errorf("Error creating topic manager: %v", err) - } - defer func() { - e := tm.Close() - if e != nil && err == nil { - err = fmt.Errorf("Error closing topic manager: %v", e) - } - }() - - // check co-partitioned (external) topics have the same number of partitions - npar, err = ensureCopartitioned(tm, gg.copartitioned().Topics()) - if err != nil { - return 0, err - } - - // TODO(diogo): add output topics - if ls := gg.LoopStream(); ls != nil { - ensureStreams := []string{ls.Topic()} - for _, t := range ensureStreams { - if err = tm.EnsureStreamExists(t, npar); err != nil { - return 0, err - } - } - } - - if gt := gg.GroupTable(); gt != nil { - if err = tm.EnsureTableExists(gt.Topic(), npar); err != nil { - return 0, err - } - } - - return -} - -// returns the number of partitions the topics have, and an error if topics are -// not copartitionea. -func ensureCopartitioned(tm kafka.TopicManager, topics []string) (int, error) { - var npar int - for _, topic := range topics { - partitions, err := tm.Partitions(topic) - if err != nil { - return 0, fmt.Errorf("Error fetching partitions for topic %s: %v", topic, err) - } - - // check assumption that partitions are gap-less - for i, p := range partitions { - if i != int(p) { - return 0, fmt.Errorf("Topic %s has partition gap: %v", topic, partitions) - } - } - - if npar == 0 { - npar = len(partitions) - } - if len(partitions) != npar { - return 0, fmt.Errorf("Topic %s does not have %d partitions", topic, npar) - } - } - return npar, nil +func (g *Processor) Graph() *GroupGraph { + return g.graph } // isStateless returns whether the processor is a stateless one. @@ -242,7 +185,7 @@ func (g *Processor) find(key string) (storage.Storage, error) { return nil, fmt.Errorf("this processor does not contain partition %v", p) } - return g.partitions[p].st, nil + return g.partitions[p].table.st, nil } func (g *Processor) hash(key string) (int32, error) { @@ -265,16 +208,9 @@ func (g *Processor) hash(key string) (int32, error) { return hash % int32(g.partitionCount), nil } -/////////////////////////////////////////////////////////////////////////////// -// lifecyle -/////////////////////////////////////////////////////////////////////////////// - -// Run starts receiving messages from Kafka for the subscribed topics. For each -// partition, a recovery will be attempted. Cancel the context to stop the -// processor. func (g *Processor) Run(ctx context.Context) (rerr error) { - g.opts.log.Printf("Processor [%s]: starting", g.graph.Group()) - defer g.opts.log.Printf("Processor [%s]: stopped", g.graph.Group()) + g.log.Debugf("starting") + defer g.log.Debugf("stopped") // create errorgroup ctx, g.cancel = context.WithCancel(ctx) @@ -282,623 +218,440 @@ func (g *Processor) Run(ctx context.Context) (rerr error) { g.ctx = ctx defer g.cancel() + // set a starting state. From this point on we know that there's a cancel and a valid context set + // in the processor which we can use for waiting + g.state.SetState(ProcStateStarting) + // collect all errors before leaving - g.errors = new(multierr.Errors) + errors := new(multierr.Errors) defer func() { - rerr = g.errors.Collect(rerr).NilOrError() + _ = errors.Collect(rerr) + rerr = errors.NilOrError() }() // create kafka consumer - g.opts.log.Printf("Processor [%s]: creating consumer ", g.graph.Group()) - consumer, err := g.opts.builders.consumer(g.brokers, string(g.graph.Group()), g.opts.clientID) + consumerGroup, err := g.opts.builders.consumerGroup(g.brokers, string(g.graph.Group()), g.opts.clientID) if err != nil { return fmt.Errorf(errBuildConsumer, err) } - g.consumer = consumer - defer func() { - g.opts.log.Printf("Processor [%s]: closing consumer", g.graph.Group()) - if err = g.consumer.Close(); err != nil { - g.errors.Collect(fmt.Errorf("error closing consumer: %v", err)) + + go func() { + for err := range consumerGroup.Errors() { + g.log.Printf("Error executing group consumer: %v", err) } - g.opts.log.Printf("Processor [%s]: closing consumer done", g.graph.Group()) }() + g.saramaConsumer, err = g.opts.builders.consumerSarama(g.brokers, g.opts.clientID) + if err != nil { + return fmt.Errorf("Error creating consumer for brokers [%s]: %v", strings.Join(g.brokers, ","), err) + } + + g.tmgr, err = g.opts.builders.topicmgr(g.brokers) + if err != nil { + return fmt.Errorf("Error creating topic manager for brokers [%s]: %v", strings.Join(g.brokers, ","), err) + } + // create kafka producer - g.opts.log.Printf("Processor [%s]: creating producer", g.graph.Group()) + g.log.Debugf("creating producer") producer, err := g.opts.builders.producer(g.brokers, g.opts.clientID, g.opts.hasher) if err != nil { return fmt.Errorf(errBuildProducer, err) } g.producer = producer defer func() { - g.opts.log.Printf("Processor [%s]: closing producer", g.graph.Group()) + g.log.Debugf("closing producer") + defer g.log.Debugf("producer ... closed") if err := g.producer.Close(); err != nil { - g.errors.Collect(fmt.Errorf("error closing producer: %v", err)) + errors.Collect(fmt.Errorf("error closing producer: %v", err)) } - g.opts.log.Printf("Processor [%s]: closing producer done.", g.graph.Group()) }() - // start all views - for t, v := range g.views { - t, v := t, v + // start all lookup tables + for topic, view := range g.lookupTables { + g.log.Debugf("Starting lookup table for %s", topic) + // make local copies + topic, view := topic, view errg.Go(func() error { - if err := v.Run(ctx); err != nil { - return fmt.Errorf("error starting lookup table %s: %v", t, err) + if err := view.Run(ctx); err != nil { + return fmt.Errorf("error running lookup table %s: %v", topic, err) } return nil }) - defer func() { g.errors.Collect(v.Terminate()) }() } - // subscribe for streams - topics := make(map[string]int64) - for _, e := range g.graph.InputStreams() { - topics[e.Topic()] = -1 - } - if lt := g.graph.LoopStream(); lt != nil { - topics[lt.Topic()] = -1 - } - if err := g.consumer.Subscribe(topics); err != nil { - g.cancel() - g.errors.Merge(errg.Wait()) - return fmt.Errorf("error subscribing topics: %v", err) - } + g.waitForLookupTables() - // start processor dispatcher + // run the main rebalance-consume-loop errg.Go(func() error { - g.asCh <- kafka.Assignment{} - return g.waitAssignment(ctx) + return g.rebalanceLoop(ctx, consumerGroup) }) - // wait for goroutines to return - g.errors.Merge(errg.Wait()) - - // remove all partitions first - g.opts.log.Printf("Processor [%s]: removing partitions", g.graph.Group()) - g.errors.Merge(g.removePartitions()) - - return + return errg.Wait().NilOrError() } -func (g *Processor) pushToPartition(ctx context.Context, part int32, ev kafka.Event) error { - p, ok := g.partitions[part] - if !ok { - return fmt.Errorf("dropping message, no partition yet: %v", ev) +func (g *Processor) rebalanceLoop(ctx context.Context, consumerGroup sarama.ConsumerGroup) (rerr error) { + var topics []string + for _, e := range g.graph.InputStreams() { + topics = append(topics, e.Topic()) } - select { - case p.ch <- ev: - case <-ctx.Done(): + if g.graph.LoopStream() != nil { + topics = append(topics, g.graph.LoopStream().Topic()) } - return nil -} -func (g *Processor) pushToPartitionView(ctx context.Context, topic string, part int32, ev kafka.Event) error { - views, ok := g.partitionViews[part] - if !ok { - return fmt.Errorf("dropping message, no partition yet: %v", ev) - } - p, ok := views[topic] - if !ok { - return fmt.Errorf("dropping message, no view yet: %v", ev) - } - select { - case p.ch <- ev: - case <-ctx.Done(): - } - return nil -} + var errs = new(multierr.Errors) + + defer func() { + errs.Collect(rerr) + rerr = errs.NilOrError() + }() + + defer func() { + g.log.Debugf("closing consumer group") + defer g.log.Debugf("closing consumer group ... done") + errs.Collect(consumerGroup.Close()) + }() -func (g *Processor) waitAssignment(ctx context.Context) error { for { + var ( + consumeErr = make(chan error) + ) + go func() { + g.log.Debugf("consuming from consumer loop") + defer g.log.Debugf("consuming from consumer loop done") + defer close(consumeErr) + err := consumerGroup.Consume(ctx, topics, g) + if err != nil { + consumeErr <- err + } + }() select { - case <-ctx.Done(): - g.opts.log.Printf("Processor [%s]: context cancelled, will stop the assignment loop", g.graph.Group()) - return nil - case a := <-g.asCh: - if err := g.runAssignment(ctx, a); err != nil { - return err + case err := <-consumeErr: + g.log.Debugf("Consumer group loop done, will stop here") + + if err != nil { + errs.Collect(err) + return } + case <-ctx.Done(): + g.log.Debugf("context closed, waiting for processor to finish up") + err := <-consumeErr + errs.Collect(err) + g.log.Debugf("context closed, waiting for processor to finish up") + return } + // let's wait some time before we retry to consume + <-time.After(5 * time.Second) } } -func (g *Processor) runAssignment(ctx context.Context, a kafka.Assignment) error { - errs := new(multierr.Errors) - ctx, cancel := context.WithCancel(ctx) - errg, ctx := multierr.NewErrGroup(ctx) - defer cancel() +func (g *Processor) waitForLookupTables() { - // create partitions based on assignmend - if err := g.rebalance(errg, ctx, a); err.HasErrors() { - return errs.Collect(err).NilOrError() + // no tables to wait for + if len(g.lookupTables) == 0 { + return } + multiWait := multierr.NewMultiWait(g.ctx, len(g.lookupTables)) - // start dispatcher - errg.Go(func() error { - err := g.dispatcher(ctx) - // cancel context even if dispatcher returned nil -- can only be a rebalance - cancel() - return err - }) - - // wait until dispatcher or partitions have returned - errs.Merge(errg.Wait()) - - // all partitions should have returned at this point, so clean up - errs.Merge(g.removePartitions()) - - return errs.NilOrError() -} + // init the multiwait with all + for _, view := range g.lookupTables { + multiWait.Add(view.WaitRunning()) + } -func (g *Processor) dispatcher(ctx context.Context) error { - g.opts.log.Printf("Processor: dispatcher started") - defer g.opts.log.Printf("Processor: dispatcher stopped") + var ( + start = time.Now() + logTicker = time.NewTicker(1 * time.Minute) + ) + defer logTicker.Stop() for { select { - case ev := <-g.consumer.Events(): - switch ev := ev.(type) { - case *kafka.Assignment: - g.asCh <- *ev - return nil - - case *kafka.Message: - var err error - if g.graph.joint(ev.Topic) { - err = g.pushToPartitionView(ctx, ev.Topic, ev.Partition, ev) - } else { - err = g.pushToPartition(ctx, ev.Partition, ev) + case <-g.ctx.Done(): + g.log.Debugf("Stopping to wait for views to get up, context closed") + return + case <-multiWait.Done(): + g.log.Debugf("View catchup finished") + return + case <-logTicker.C: + var tablesWaiting []string + for topic, view := range g.lookupTables { + if !view.Recovered() { + tablesWaiting = append(tablesWaiting, topic) } - if err != nil { - return fmt.Errorf("error consuming message: %v", err) - } - - case *kafka.BOF: - var err error - if g.graph.joint(ev.Topic) { - err = g.pushToPartitionView(ctx, ev.Topic, ev.Partition, ev) - } else { - err = g.pushToPartition(ctx, ev.Partition, ev) - } - if err != nil { - return fmt.Errorf("error consuming BOF: %v", err) - } - - case *kafka.EOF: - var err error - if g.graph.joint(ev.Topic) { - err = g.pushToPartitionView(ctx, ev.Topic, ev.Partition, ev) - } else { - err = g.pushToPartition(ctx, ev.Partition, ev) - } - if err != nil { - return fmt.Errorf("error consuming EOF: %v", err) - } - - case *kafka.NOP: - if g.graph.joint(ev.Topic) { - _ = g.pushToPartitionView(ctx, ev.Topic, ev.Partition, ev) - } else { - _ = g.pushToPartition(ctx, ev.Partition, ev) - } - - case *kafka.Error: - return fmt.Errorf("kafka error: %v", ev.Err) - - default: - return fmt.Errorf("processor: cannot handle %T = %v", ev, ev) } - case <-ctx.Done(): - return nil + g.log.Printf("Waiting for views [%s] to catchup since %.2f minutes", + strings.Join(tablesWaiting, ", "), + time.Since(start).Minutes()) } } } -func (g *Processor) fail(err error) { - g.opts.log.Printf("failing: %v", err) - g.errors.Collect(err) - g.cancel() +// Recovered returns whether the processor is running, i.e. if the processor +// has recovered all lookups/joins/tables and is running +func (g *Processor) Recovered() bool { + return g.state.IsState(ProcStateRunning) } -/////////////////////////////////////////////////////////////////////////////// -// partition management (rebalance) -/////////////////////////////////////////////////////////////////////////////// +func (g *Processor) assignmentFromSession(session sarama.ConsumerGroupSession) Assignment { + assignment := Assignment{} -func (g *Processor) newJoinStorage(topic string, id int32, update UpdateCallback) (*storageProxy, error) { - st, err := g.opts.builders.storage(topic, id) - if err != nil { - return nil, err + // get the partitions this processor is assigned to. + // Since we're copartitioned, we can stop after the first + for _, claim := range session.Claims() { + for _, part := range claim { + assignment[part] = sarama.OffsetNewest + } + break } - return &storageProxy{ - Storage: st, - partition: id, - update: update, - }, nil + + return assignment } -func (g *Processor) newStorage(topic string, id int32, update UpdateCallback) (*storageProxy, error) { - if g.isStateless() { - return &storageProxy{ - Storage: storage.NewMemory(), - partition: id, - stateless: true, - }, nil +// Setup is run at the beginning of a new session, before ConsumeClaim. +func (g *Processor) Setup(session sarama.ConsumerGroupSession) error { + g.state.SetState(ProcStateSetup) + defer g.state.SetState(ProcStateRunning) + g.log.Printf("setup generation %d", session.GenerationID()) + defer g.log.Printf("setup generation %d ... done", session.GenerationID()) + + assignment := g.assignmentFromSession(session) + + if g.rebalanceCallback != nil { + g.rebalanceCallback(assignment) } - var ( - err error - st storage.Storage - wg sync.WaitGroup - ) - start := time.Now() - ticker := time.NewTicker(5 * time.Minute) - defer ticker.Stop() - wg.Add(1) - go func() { - defer wg.Done() - st, err = g.opts.builders.storage(topic, id) - g.opts.log.Printf("finished building storage for topic %s", topic) - }() - go func() { - for range ticker.C { - g.opts.log.Printf("building storage for topic %s for %s ...", topic, time.Since(start).String()) + // no partitions configured, we cannot setup anything + if len(assignment) == 0 { + g.log.Printf("No partitions assigned. Claims were: %#v. Will probably sleep this generation", session.Claims()) + return nil + } + // create partition views for all partitions + for partition := range assignment { + // create partition processor for our partition + err := g.createPartitionProcessor(session.Context(), partition, session) + if err != nil { + return fmt.Errorf("Error creating partition processor for %s/%d: %v", g.Graph().Group(), partition, err) } - }() - wg.Wait() - if err != nil { - return nil, err } - return &storageProxy{ - Storage: st, - partition: id, - update: update, - }, nil -} - -func (g *Processor) createPartitionViews(errg *multierr.ErrGroup, ctx context.Context, id int32) error { - if _, has := g.partitionViews[id]; !has { - g.partitionViews[id] = make(map[string]*partition) + // setup all processors + errg, _ := multierr.NewErrGroup(session.Context()) + for _, partition := range g.partitions { + pproc := partition + errg.Go(func() error { + return pproc.Setup(session.Context()) + }) } - for _, t := range g.graph.JointTables() { - if _, has := g.partitions[id]; has { - continue - } - st, err := g.newJoinStorage(t.Topic(), id, DefaultUpdate) - if err != nil { - return fmt.Errorf("processor: error creating storage: %v", err) - } - p := newPartition( - g.opts.log, - t.Topic(), - nil, st, &proxy{id, g.consumer}, - g.opts.partitionChannelSize, - ) - g.partitionViews[id][t.Topic()] = p - - errg.Go(func() (err error) { - defer func() { - if rerr := recover(); rerr != nil { - g.opts.log.Printf("partition view %s/%d: panic", p.topic, id) - err = fmt.Errorf("panic partition view %s/%d: %v\nstack:%v", - p.topic, id, rerr, string(debug.Stack())) - } - }() + return errg.Wait().NilOrError() +} - if err = p.st.Open(); err != nil { - return fmt.Errorf("error opening storage %s/%d: %v", p.topic, id, err) - } - if err = p.startCatchup(ctx); err != nil { - return fmt.Errorf("error in partition view %s/%d: %v", p.topic, id, err) +// Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited +// but before the offsets are committed for the very last time. +func (g *Processor) Cleanup(session sarama.ConsumerGroupSession) error { + g.log.Printf("Cleaning up for %d", session.GenerationID()) + defer g.log.Printf("Cleaning up for %d ... done", session.GenerationID()) + + g.state.SetState(ProcStateStopping) + defer g.state.SetState(ProcStateIdle) + errg, _ := multierr.NewErrGroup(session.Context()) + for part, partition := range g.partitions { + partID, pproc := part, partition + errg.Go(func() error { + err := pproc.Stop() + if err != nil { + return fmt.Errorf("error stopping partition processor %d: %v", partID, err) } - g.opts.log.Printf("partition view %s/%d: exit", p.topic, id) return nil }) } - return nil + err := errg.Wait().NilOrError() + g.partitions = make(map[int32]*PartitionProcessor) + return err } -func (g *Processor) createPartition(errg *multierr.ErrGroup, ctx context.Context, id int32) error { - if _, has := g.partitions[id]; has { - return nil - } - // TODO(diogo) what name to use for stateless processors? - var groupTable string - if gt := g.graph.GroupTable(); gt != nil { - groupTable = gt.Topic() - } - st, err := g.newStorage(groupTable, id, g.opts.updateCallback) - if err != nil { - return fmt.Errorf("processor: error creating storage: %v", err) - } +// WaitForReady waits until the processor is ready to consume messages (or is actually consuming messages) +// i.e., it is done catching up all partition tables, joins and lookup tables +func (g *Processor) WaitForReady() { + // wait for the processor to be started + <-g.state.WaitForStateMin(ProcStateStarting) - // collect dependencies - var wait []func() bool - if pviews, has := g.partitionViews[id]; has { - for _, p := range pviews { - wait = append(wait, p.recovered) - } - } - for _, v := range g.views { - wait = append(wait, v.Recovered) + // wait that the processor is actually running + select { + case <-g.state.WaitForState(ProcStateRunning): + case <-g.ctx.Done(): } - g.partitions[id] = newPartition( - g.opts.log, - groupTable, - g.process, st, &delayProxy{proxy: proxy{partition: id, consumer: g.consumer}, wait: wait}, - g.opts.partitionChannelSize, - ) - par := g.partitions[id] - errg.Go(func() (err error) { - defer func() { - if rerr := recover(); rerr != nil { - g.opts.log.Printf("partition %s/%d: panic", par.topic, id) - err = fmt.Errorf("partition %s/%d: panic: %v\nstack:%v", - par.topic, id, rerr, string(debug.Stack())) - } - }() - if err = par.st.Open(); err != nil { - return fmt.Errorf("error opening storage partition %d: %v", id, err) - } - if err = par.start(ctx); err != nil { - return fmt.Errorf("error in partition %d: %v", id, err) + // wait for all partitionprocessors to be running + for _, part := range g.partitions { + select { + case <-part.state.WaitForState(PPStateRunning): + case <-g.ctx.Done(): } - g.opts.log.Printf("partition %s/%d: exit", par.topic, id) - return nil - }) - - return nil + } } -func (g *Processor) rebalance(errg *multierr.ErrGroup, ctx context.Context, partitions kafka.Assignment) *multierr.Errors { - errs := new(multierr.Errors) - g.opts.log.Printf("Processor: rebalancing: %+v", partitions) +// ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages(). +// Once the Messages() channel is closed, the Handler must finish its processing +// loop and exit. +func (g *Processor) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error { + g.log.Printf("ConsumeClaim for topic/partition %s/%d, initialOffset=%d", claim.Topic(), claim.Partition(), claim.InitialOffset()) + defer g.log.Printf("ConsumeClaim done for topic/partition %s/%d", claim.Topic(), claim.Partition()) + part, has := g.partitions[claim.Partition()] + if !has { + return fmt.Errorf("No partition (%d) to handle input in topic %s", claim.Partition(), claim.Topic()) + } - // callback the new partition assignment - g.opts.rebalanceCallback(partitions) + messages := claim.Messages() + errors := part.Errors() - g.m.Lock() - defer g.m.Unlock() + for { + select { + case msg, ok := <-messages: + if !ok { + return nil + } - for id := range partitions { - // create partition views - if err := g.createPartitionViews(errg, ctx, id); err != nil { - errs.Collect(err) - } - // create partition processor - if err := g.createPartition(errg, ctx, id); err != nil { - errs.Collect(err) + select { + case part.input <- msg: + case err := <-errors: + return err + } + + case err := <-errors: + return err } } - return errs } -func (g *Processor) removePartitions() *multierr.Errors { - g.m.Lock() - defer g.m.Unlock() - errs := new(multierr.Errors) - for partition := range g.partitions { - errs.Merge(g.removePartition(partition)) - } - return errs +// Stats returns the aggregated stats for the processor including all partitions, tables, lookups and joins +func (g *Processor) Stats() *ProcessorStats { + return g.StatsWithContext(context.Background()) } -func (g *Processor) removePartition(partition int32) *multierr.Errors { - errs := new(multierr.Errors) - g.opts.log.Printf("Removing partition %d", partition) +// StatsWithContext returns stats for the processor, see #Processor.Stats() +func (g *Processor) StatsWithContext(ctx context.Context) *ProcessorStats { + var ( + m sync.Mutex + stats = newProcessorStats(len(g.partitions)) + ) - // remove partition processor - if err := g.partitions[partition].st.Close(); err != nil { - errs.Collect(fmt.Errorf("error closing storage partition %d: %v", partition, err)) - } - delete(g.partitions, partition) + errg, ctx := multierr.NewErrGroup(ctx) - // remove partition views - pv, has := g.partitionViews[partition] - if !has { - return errs - } + // get partition-processor stats + for partID, proc := range g.partitions { + partID, proc := partID, proc + errg.Go(func() error { + partStats := proc.fetchStats(ctx) - for topic, p := range pv { - if err := p.st.Close(); err != nil { - errs.Collect(fmt.Errorf("error closing storage %s/%d: %v", topic, partition, err)) - } + m.Lock() + defer m.Unlock() + stats.Group[partID] = partStats + return nil + }) } - delete(g.partitionViews, partition) - - return errs -} - -/////////////////////////////////////////////////////////////////////////////// -// context builder -/////////////////////////////////////////////////////////////////////////////// -func (g *Processor) process(msg *message, st storage.Storage, wg *sync.WaitGroup, pstats *PartitionStats) (int, error) { - g.m.RLock() - views := g.partitionViews[msg.Partition] - g.m.RUnlock() - - ctx := &cbContext{ - ctx: g.ctx, - graph: g.graph, - - pstats: pstats, - pviews: views, - views: g.views, - wg: wg, - msg: msg, - failer: func(err error) { - // only fail processor if context not already Done - select { - case <-g.ctx.Done(): - return - default: - } - g.fail(err) - }, - emitter: func(topic string, key string, value []byte) *kafka.Promise { - return g.producer.Emit(topic, key, value).Then(func(err error) { - if err != nil { - g.fail(err) - } - }) - }, + for topic, view := range g.lookupTables { + topic, view := topic, view + errg.Go(func() error { + m.Lock() + viewStats := view.Stats(ctx) + defer m.Unlock() + stats.Lookup[topic] = viewStats + return nil + }) } - ctx.commit = func() { - // write group table offset to local storage - if ctx.counters.stores > 0 { - if offset, err := ctx.storage.GetOffset(0); err != nil { - ctx.failer(fmt.Errorf("error getting storage offset for %s/%d: %v", - g.graph.GroupTable().Topic(), msg.Partition, err)) - return - } else if err = ctx.storage.SetOffset(offset + int64(ctx.counters.stores)); err != nil { - ctx.failer(fmt.Errorf("error writing storage offset for %s/%d: %v", - g.graph.GroupTable().Topic(), msg.Partition, err)) - return - } - } - // mark upstream offset - if err := g.consumer.Commit(msg.Topic, msg.Partition, msg.Offset); err != nil { - g.fail(fmt.Errorf("error committing offsets of %s/%d: %v", - g.graph.GroupTable().Topic(), msg.Partition, err)) - } + err := errg.Wait().NilOrError() + if err != nil { + g.log.Printf("Error retrieving stats: %v", err) } + return stats +} + +// creates the partition that is responsible for the group processor's table +func (g *Processor) createPartitionProcessor(ctx context.Context, partition int32, session sarama.ConsumerGroupSession) error { - // use the storage if the processor is not stateless. Ignore otherwise - if !g.isStateless() { - ctx.storage = st + g.log.Debugf("Creating partition processor for partition %d", partition) + if _, has := g.partitions[partition]; has { + return fmt.Errorf("processor [%s]: partition %d already exists", g.graph.Group(), partition) } - var ( - m interface{} - err error - ) + pproc := newPartitionProcessor(partition, g.graph, session, g.log, g.opts, g.lookupTables, g.saramaConsumer, g.producer, g.tmgr) - // decide whether to decode or ignore message - switch { - case msg.Data == nil && g.opts.nilHandling == NilIgnore: - // drop nil messages - return 0, nil - case msg.Data == nil && g.opts.nilHandling == NilProcess: - // process nil messages without decoding them - m = nil - default: - // get stream subcription - codec := g.graph.codec(msg.Topic) - if codec == nil { - return 0, fmt.Errorf("cannot handle topic %s", msg.Topic) - } + g.partitions[partition] = pproc + return nil +} - // decode message - m, err = codec.Decode(msg.Data) - if err != nil { - return 0, fmt.Errorf("error decoding message for key %s from %s/%d: %v", msg.Key, msg.Topic, msg.Partition, err) - } - } +// Stop stops the processor. +// This is semantically equivalent of closing the Context +// that was passed to Processor.Run(..). +// This method will return immediately, errors during running +// will be returned from teh Processor.Run(..) +func (g *Processor) Stop() { + g.cancel() +} - cb := g.graph.callback(msg.Topic) - if cb == nil { - return 0, fmt.Errorf("error processing message for key %s from %s/%d: %v", msg.Key, msg.Topic, msg.Partition, err) +func prepareTopics(brokers []string, gg *GroupGraph, opts *poptions) (npar int, err error) { + // create topic manager + tm, err := opts.builders.topicmgr(brokers) + if err != nil { + return 0, fmt.Errorf("Error creating topic manager: %v", err) } - - // start context and call the ProcessorCallback cb - ctx.start() - // call finish(err) if a panic occurs in cb defer func() { - if r := recover(); r != nil { - ctx.finish(fmt.Errorf("panic: %v", r)) - panic(r) // propagate panic up + e := tm.Close() + if e != nil && err == nil { + err = fmt.Errorf("Error closing topic manager: %v", e) } }() - // now call cb - cb(ctx, m) - // if everything went fine, call finish(nil) - ctx.finish(nil) - - return ctx.counters.stores, nil -} -// Recovered returns true when the processor has caught up with events from kafka. -func (g *Processor) Recovered() bool { - for _, v := range g.views { - if !v.Recovered() { - return false - } + // check co-partitioned (external) topics have the same number of partitions + npar, err = ensureCopartitioned(tm, gg.copartitioned().Topics()) + if err != nil { + return 0, err } - for _, part := range g.partitionViews { - for _, topicPart := range part { - if !topicPart.recovered() { - return false + // TODO(diogo): add output topics + if ls := gg.LoopStream(); ls != nil { + ensureStreams := []string{ls.Topic()} + for _, t := range ensureStreams { + if err = tm.EnsureStreamExists(t, npar); err != nil { + return 0, err } } } - for _, p := range g.partitions { - if !p.recovered() { - return false + if gt := gg.GroupTable(); gt != nil { + if err = tm.EnsureTableExists(gt.Topic(), npar); err != nil { + return 0, err } } - return true + return } -// Stats returns a set of performance metrics of the processor. -func (g *Processor) Stats() *ProcessorStats { - return g.statsWithContext(context.Background()) -} +// returns the number of partitions the topics have, and an error if topics are +// not copartitionea. +func ensureCopartitioned(tm TopicManager, topics []string) (int, error) { + var npar int + for _, topic := range topics { + partitions, err := tm.Partitions(topic) + if err != nil { + return 0, fmt.Errorf("Error fetching partitions for topic %s: %v", topic, err) + } -func (g *Processor) statsWithContext(ctx context.Context) *ProcessorStats { - var ( - m sync.Mutex - wg sync.WaitGroup - stats = newProcessorStats(len(g.partitions)) - ) + // check assumption that partitions are gap-less + for i, p := range partitions { + if i != int(p) { + return 0, fmt.Errorf("Topic %s has partition gap: %v", topic, partitions) + } + } - for i, p := range g.partitions { - wg.Add(1) - go func(pid int32, par *partition) { - s := par.fetchStats(ctx) - m.Lock() - stats.Group[pid] = s - m.Unlock() - wg.Done() - }(i, p) - } - for i, p := range g.partitionViews { - if _, ok := stats.Joined[i]; !ok { - stats.Joined[i] = make(map[string]*PartitionStats) + if npar == 0 { + npar = len(partitions) } - for t, tp := range p { - wg.Add(1) - go func(pid int32, topic string, par *partition) { - s := par.fetchStats(ctx) - m.Lock() - stats.Joined[pid][topic] = s - m.Unlock() - wg.Done() - }(i, t, tp) + if len(partitions) != npar { + return 0, fmt.Errorf("Topic %s does not have %d partitions", topic, npar) } } - for t, v := range g.views { - wg.Add(1) - go func(topic string, vi *View) { - s := vi.statsWithContext(ctx) - m.Lock() - stats.Lookup[topic] = s - m.Unlock() - wg.Done() - }(t, v) - } - - wg.Wait() - return stats -} - -// Graph returns the GroupGraph given at the creation of the processor. -func (g *Processor) Graph() *GroupGraph { - return g.graph + return npar, nil } diff --git a/processor_integration_test.go b/processor_integration_test.go index 38c7fbc0..ecfc4096 100644 --- a/processor_integration_test.go +++ b/processor_integration_test.go @@ -1,5 +1,6 @@ package goka_test +/* import ( "context" "errors" @@ -12,7 +13,6 @@ import ( "github.com/facebookgo/ensure" "github.com/lovoo/goka" "github.com/lovoo/goka/codec" - "github.com/lovoo/goka/kafka" "github.com/lovoo/goka/mock" "github.com/lovoo/goka/storage" "github.com/lovoo/goka/tester" @@ -36,7 +36,7 @@ func doTimed(t *testing.T, do func()) error { } func TestProcessor_StatelessContext(t *testing.T) { - ctrl := mock.NewMockController(t) + ctrl := NewMockController(t) defer ctrl.Finish() var ( tester = tester.New(t) @@ -75,8 +75,8 @@ func TestProcessor_ProducerError(t *testing.T) { t.Run("SetValue", func(t *testing.T) { tester := tester.New(t) - tester.ReplaceEmitHandler(func(topic, key string, value []byte) *kafka.Promise { - return kafka.NewPromise().Finish(errors.New("producer error")) + tester.ReplaceEmitHandler(func(topic, key string, value []byte) *goka.Promise { + return goka.NewPromise().Finish(errors.New("producer error")) }) consume := func(ctx goka.Context, msg interface{}) { @@ -110,8 +110,8 @@ func TestProcessor_ProducerError(t *testing.T) { t.Run("Emit", func(t *testing.T) { tester := tester.New(t) - tester.ReplaceEmitHandler(func(topic, key string, value []byte) *kafka.Promise { - return kafka.NewPromise().Finish(errors.New("producer error")) + tester.ReplaceEmitHandler(func(topic, key string, value []byte) *goka.Promise { + return goka.NewPromise().Finish(errors.New("producer error")) }) consume := func(ctx goka.Context, msg interface{}) { @@ -146,8 +146,8 @@ func TestProcessor_ProducerError(t *testing.T) { t.Run("Value-stateless", func(t *testing.T) { tester := tester.New(t) - tester.ReplaceEmitHandler(func(topic, key string, value []byte) *kafka.Promise { - return kafka.NewPromise().Finish(errors.New("producer error")) + tester.ReplaceEmitHandler(func(topic, key string, value []byte) *goka.Promise { + return goka.NewPromise().Finish(errors.New("producer error")) }) consume := func(ctx goka.Context, msg interface{}) { @@ -386,3 +386,4 @@ func TestProcessor_failOnRecover(t *testing.T) { // we can't test that anymore since there is no recovery-functionality in the tester implemented //ensure.True(t, recovered > 0 && recovered < msgToRecover) } +*/ diff --git a/processor_test.go b/processor_test.go index 9d53d6ff..7e920739 100644 --- a/processor_test.go +++ b/processor_test.go @@ -2,1528 +2,319 @@ package goka import ( "context" - "errors" "fmt" - "hash" - "log" - "os" - "os/signal" - "sync" - "syscall" + "strconv" + "strings" "testing" "time" + "github.com/Shopify/sarama" + "github.com/golang/mock/gomock" "github.com/lovoo/goka/codec" - "github.com/lovoo/goka/kafka" - "github.com/lovoo/goka/logger" - "github.com/lovoo/goka/mock" - "github.com/lovoo/goka/multierr" + "github.com/lovoo/goka/internal/test" "github.com/lovoo/goka/storage" - - "github.com/facebookgo/ensure" - "github.com/golang/mock/gomock" -) - -var ( - rawCodec = new(codec.Bytes) - emptyRebalanceCallback = func(a kafka.Assignment){} ) -func nullStorageBuilder() storage.Builder { - return func(topic string, partition int32) (storage.Storage, error) { - return &storage.Null{}, nil - } -} - -func syncWith(t *testing.T, ch chan kafka.Event, p ...int32) error { - return doTimed(t, func() { - for _, par := range p { - ch <- &kafka.NOP{Partition: par} - ch <- &kafka.NOP{Partition: -1} - ch <- &kafka.NOP{Partition: par} - ch <- &kafka.NOP{Partition: -1} - } - ch <- &kafka.NOP{Partition: -1} - }) -} - -func createStorageBuilder(st storage.Storage) storage.Builder { - return func(topic string, partition int32) (storage.Storage, error) { - return st, nil - } -} - -func createConsumerBuilder(c kafka.Consumer) kafka.ConsumerBuilder { - return func(b []string, g, id string) (kafka.Consumer, error) { - return c, nil - } -} - -func createFailedConsumerBuilder() kafka.ConsumerBuilder { - return func(b []string, g, id string) (kafka.Consumer, error) { - return nil, errors.New("failed creating consumer") - } +func createMockBuilder(t *testing.T) (*gomock.Controller, *builderMock) { + ctrl := NewMockController(t) + bm := newBuilderMock(ctrl) + bm.st = storage.NewMemory() + return ctrl, bm } -func createProducerBuilder(p kafka.Producer) kafka.ProducerBuilder { - return func(b []string, id string, hasher func() hash.Hash32) (kafka.Producer, error) { - return p, nil - } +func createTestConsumerGroupBuilder(t *testing.T) (ConsumerGroupBuilder, *MockConsumerGroup) { + mock := NewMockConsumerGroup(t) + return func(brokers []string, group, clientID string) (sarama.ConsumerGroup, error) { + return mock, nil + }, mock } -func createFailedProducerBuilder() kafka.ProducerBuilder { - return func(b []string, id string, hasher func() hash.Hash32) (kafka.Producer, error) { - return nil, errors.New("failed creating producer") - } -} +func createTestConsumerBuilder(t *testing.T) (SaramaConsumerBuilder, *MockAutoConsumer) { + cons := NewMockAutoConsumer(t, nil) -func createTopicManagerBuilder(tm kafka.TopicManager) kafka.TopicManagerBuilder { - return func(b []string) (kafka.TopicManager, error) { - return tm, nil - } + return func(brokers []string, clientID string) (sarama.Consumer, error) { + return cons, nil + }, cons } -func createFailedTopicManagerBuilder(tm kafka.TopicManager) kafka.TopicManagerBuilder { - return func(b []string) (kafka.TopicManager, error) { - return nil, errors.New("failed creating topic manager") - } -} - -func createProcessorStateless(ctrl *gomock.Controller, consumer kafka.Consumer, producer kafka.Producer, npar int, rcb func(a kafka.Assignment)) *Processor { - tm := mock.NewMockTopicManager(ctrl) - - var partitions []int32 - for i := 0; i < npar; i++ { - partitions = append(partitions, int32(i)) +func expectCGEmit(bm *builderMock, table string, msgs []*sarama.ConsumerMessage) { + for _, msg := range msgs { + bm.producer.EXPECT().Emit(table, string(msg.Key), msg.Value).Return(NewPromise().Finish(nil, nil)) } - - // successfully create processor - tm.EXPECT().Partitions(topic).Return(partitions, nil) - tm.EXPECT().Partitions(topic2).Return(partitions, nil) - tm.EXPECT().EnsureStreamExists(loopName(group), len(partitions)).Return(nil) - tm.EXPECT().Close().Return(nil) - p, _ := NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Input(topic2, rawCodec, cb), - Loop(rawCodec, cb), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - WithConsumerBuilder(createConsumerBuilder(consumer)), - WithProducerBuilder(createProducerBuilder(producer)), - WithPartitionChannelSize(0), - WithRebalanceCallback(rcb), - ) - return p } -func createProcessor(t *testing.T, ctrl *gomock.Controller, consumer kafka.Consumer, npar int, sb storage.Builder) *Processor { - tm := mock.NewMockTopicManager(ctrl) - producer := mock.NewMockProducer(ctrl) - - var partitions []int32 - for i := 0; i < npar; i++ { - partitions = append(partitions, int32(i)) +func expectCGLoop(bm *builderMock, loop string, msgs []*sarama.ConsumerMessage) { + bm.tmgr.EXPECT().EnsureStreamExists(loop, 1).AnyTimes() + for _, msg := range msgs { + bm.producer.EXPECT().Emit(loop, string(msg.Key), gomock.Any()).Return(NewPromise().Finish(nil, nil)) } - - // the prodcuer may be closed, but doesn't have to - producer.EXPECT().Close().Return(nil).AnyTimes() - - // successfully create processor - tm.EXPECT().Partitions(topic).Return(partitions, nil) - tm.EXPECT().Partitions(topic2).Return(partitions, nil) - tm.EXPECT().EnsureStreamExists(loopName(group), len(partitions)).Return(nil) - tm.EXPECT().EnsureTableExists(tableName(group), len(partitions)).Return(nil) - tm.EXPECT().Close().Return(nil) - p, err := NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Input(topic2, rawCodec, cb), - Loop(rawCodec, cb), - Persist(new(codec.String)), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - WithConsumerBuilder(createConsumerBuilder(consumer)), - WithProducerBuilder(createProducerBuilder(producer)), - WithStorageBuilder(sb), - WithPartitionChannelSize(0), - ) - ensure.Nil(t, err) - return p } -func createProcessorWithTable(t *testing.T, ctrl *gomock.Controller, consumer kafka.Consumer, producer kafka.Producer, npar int, sb storage.Builder) *Processor { - tm := mock.NewMockTopicManager(ctrl) - - var partitions []int32 - for i := 0; i < npar; i++ { - partitions = append(partitions, int32(i)) - } - - // successfully create processor - tm.EXPECT().Partitions(topic).Return(partitions, nil) - tm.EXPECT().Partitions(topic2).Return(partitions, nil) - tm.EXPECT().Partitions(table).Return(partitions, nil) - tm.EXPECT().EnsureStreamExists(loopName(group), len(partitions)).Return(nil) - tm.EXPECT().EnsureTableExists(tableName(group), len(partitions)).Return(nil) - tm.EXPECT().Close().Return(nil) - p, err := NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Input(topic2, rawCodec, cb), - Loop(rawCodec, cb), - Join(table, rawCodec), - Persist(rawCodec), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - WithConsumerBuilder(createConsumerBuilder(consumer)), - WithProducerBuilder(createProducerBuilder(producer)), - WithStorageBuilder(sb), - WithPartitionChannelSize(0), - ) - ensure.Nil(t, err) - return p -} - -func createProcessorWithLookupTable(t *testing.T, ctrl *gomock.Controller, consumer kafka.Consumer, npar int, sb storage.Builder) *Processor { - tm := mock.NewMockTopicManager(ctrl) - producer := mock.NewMockProducer(ctrl) - - var partitions []int32 - for i := 0; i < npar; i++ { - partitions = append(partitions, int32(i)) - } - - // successfully create processor - tm.EXPECT().Partitions(topic).Return(partitions, nil) - tm.EXPECT().Partitions(table).Return(partitions, nil) - tm.EXPECT().Close().Return(nil).Times(2) - p, err := NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Lookup(table, rawCodec), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - WithConsumerBuilder(createConsumerBuilder(consumer)), - WithProducerBuilder(createProducerBuilder(producer)), - WithStorageBuilder(sb), - WithPartitionChannelSize(0), - ) - ensure.Nil(t, err) - return p -} - -var ( - topOff = map[string]int64{ - topic: -1, - loopName(group): -1, - topic2: -1, - } - errSome = errors.New("some error") - cb = func(ctx Context, msg interface{}) {} -) - -const ( - topic2 = "topic2" - table = "table" - table2 = "table2" -) - -func TestProcessor_process(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - wg sync.WaitGroup - st = mock.NewMockStorage(ctrl) - consumer = mock.NewMockConsumer(ctrl) - producer = mock.NewMockProducer(ctrl) - pstats = newPartitionStats() - ) - - p := &Processor{ - graph: DefineGroup(group, - Persist(new(codec.String)), - Loop(c, cb), - Input("sometopic", rawCodec, cb), - Output("anothertopic", new(codec.String)), - ), - - consumer: consumer, - producer: producer, - - ctx: context.Background(), - } - - // no emits - consumer.EXPECT().Commit("sometopic", int32(1), int64(123)) - msg := &message{Topic: "sometopic", Partition: 1, Offset: 123, Data: []byte("something")} - updates, err := p.process(msg, st, &wg, pstats) - ensure.Nil(t, err) - ensure.DeepEqual(t, updates, 0) - - // emit something - promise := new(kafka.Promise) - gomock.InOrder( - producer.EXPECT().Emit("anothertopic", "key", []byte("message")).Return(promise), - consumer.EXPECT().Commit("sometopic", int32(1), int64(123)), - ) - msg = &message{Topic: "sometopic", Partition: 1, Offset: 123, Data: []byte("something")} - p.graph.callbacks["sometopic"] = func(ctx Context, msg interface{}) { - ctx.Emit("anothertopic", "key", "message") - } - updates, err = p.process(msg, st, &wg, pstats) - ensure.Nil(t, err) - ensure.DeepEqual(t, updates, 0) - promise.Finish(nil) - - // store something - promise = new(kafka.Promise) - gomock.InOrder( - st.EXPECT().Set("key", []byte("message")), - producer.EXPECT().Emit(tableName(group), "key", []byte("message")).Return(promise), - st.EXPECT().GetOffset(int64(0)).Return(int64(321), nil), - st.EXPECT().SetOffset(int64(322)), - consumer.EXPECT().Commit("sometopic", int32(1), int64(123)), - ) - msg = &message{Topic: "sometopic", Key: "key", Partition: 1, Offset: 123, Data: []byte("something")} - p.graph.callbacks["sometopic"] = func(ctx Context, msg interface{}) { - ctx.SetValue("message") - } - updates, err = p.process(msg, st, &wg, pstats) - ensure.Nil(t, err) - ensure.DeepEqual(t, updates, 1) - promise.Finish(nil) - - // store something twice - promise = new(kafka.Promise) - promise2 := new(kafka.Promise) - gomock.InOrder( - st.EXPECT().Set("key", []byte("message")), - producer.EXPECT().Emit(tableName(group), "key", []byte("message")).Return(promise), - st.EXPECT().Set("key", []byte("message2")), - producer.EXPECT().Emit(tableName(group), "key", []byte("message2")).Return(promise2), - st.EXPECT().GetOffset(int64(0)).Return(int64(321), nil), - st.EXPECT().SetOffset(int64(323)), - consumer.EXPECT().Commit("sometopic", int32(1), int64(123)), - ) - msg = &message{Topic: "sometopic", Key: "key", Partition: 1, Offset: 123, Data: []byte("something")} - p.graph.callbacks["sometopic"] = func(ctx Context, msg interface{}) { - ctx.SetValue("message") - ctx.SetValue("message2") - } - updates, err = p.process(msg, st, &wg, pstats) - ensure.Nil(t, err) - ensure.DeepEqual(t, updates, 2) - promise.Finish(nil) - promise2.Finish(nil) - -} - -func TestProcessor_processFail(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - +func expectCGConsume(bm *builderMock, table string, msgs []*sarama.ConsumerMessage) { var ( - wg sync.WaitGroup - st = mock.NewMockStorage(ctrl) - consumer = mock.NewMockConsumer(ctrl) - producer = mock.NewMockProducer(ctrl) - pstats = newPartitionStats() + current int64 ) - var canceled chan bool - newProcessor := func() *Processor { - canceled = make(chan bool) - p := &Processor{ - graph: DefineGroup(group, - Persist(new(codec.String)), - Loop(c, cb), - Input("sometopic", rawCodec, cb), - Output("anothertopic", new(codec.String)), - ), - consumer: consumer, - producer: producer, - opts: new(poptions), + bm.producer.EXPECT().Close().Return(nil).AnyTimes() - errors: new(multierr.Errors), - cancel: func() { close(canceled) }, - ctx: context.Background(), - } - - p.opts.log = logger.Default() - return p - } - // fail get offset - p := newProcessor() - promise := new(kafka.Promise) - gomock.InOrder( - st.EXPECT().Set("key", []byte("message")), - producer.EXPECT().Emit(tableName(group), "key", []byte("message")).Return(promise), - st.EXPECT().GetOffset(int64(0)).Return(int64(321), errors.New("getOffset failed")), - ) - msg := &message{Topic: "sometopic", Key: "key", Partition: 1, Offset: 123, Data: []byte("something")} - p.graph.callbacks["sometopic"] = func(ctx Context, msg interface{}) { - ctx.SetValue("message") - } - updates, err := p.process(msg, st, &wg, pstats) - ensure.Nil(t, err) - ensure.DeepEqual(t, updates, 1) - promise.Finish(nil) - err = doTimed(t, func() { - <-canceled - }) - ensure.Nil(t, err) - - // fail set offset - promise = new(kafka.Promise) - p = newProcessor() - gomock.InOrder( - st.EXPECT().Set("key", []byte("message")), - producer.EXPECT().Emit(tableName(group), "key", []byte("message")).Return(promise), - st.EXPECT().GetOffset(int64(0)).Return(int64(321), nil), - st.EXPECT().SetOffset(int64(322)).Return(errors.New("setOffset failed")), - ) - msg = &message{Topic: "sometopic", Key: "key", Partition: 1, Offset: 123, Data: []byte("something")} - p.graph.callbacks["sometopic"] = func(ctx Context, msg interface{}) { - ctx.SetValue("message") - } - updates, err = p.process(msg, st, &wg, pstats) - ensure.Nil(t, err) - ensure.DeepEqual(t, updates, 1) - promise.Finish(nil) - err = doTimed(t, func() { - <-canceled - }) - ensure.Nil(t, err) - - // fail commit - promise = new(kafka.Promise) - p = newProcessor() - gomock.InOrder( - st.EXPECT().Set("key", []byte("message")), - producer.EXPECT().Emit(tableName(group), "key", []byte("message")).Return(promise), - st.EXPECT().GetOffset(int64(0)).Return(int64(321), nil), - st.EXPECT().SetOffset(int64(322)), - consumer.EXPECT().Commit("sometopic", int32(1), int64(123)).Return(errors.New("commit error")), - ) - msg = &message{Topic: "sometopic", Key: "key", Partition: 1, Offset: 123, Data: []byte("something")} - p.graph.callbacks["sometopic"] = func(ctx Context, msg interface{}) { - ctx.SetValue("message") - } - updates, err = p.process(msg, st, &wg, pstats) - ensure.Nil(t, err) - ensure.DeepEqual(t, updates, 1) - promise.Finish(nil) - err = doTimed(t, func() { - <-canceled - }) - ensure.Nil(t, err) - - // fail with panic (ctx.Fatal) - p = newProcessor() - // we dont add expect consumer.EXPECT().Commit() here, so if the context - // would call it, the test would fail - p.graph.callbacks["sometopic"] = func(ctx Context, msg interface{}) { - ctx.Fail(errSome) - t.Errorf("should never reach this point") - t.Fail() - } - go func() { + bm.tmgr.EXPECT().Close().Return(nil).AnyTimes() + bm.tmgr.EXPECT().EnsureTableExists(table, gomock.Any()).Return(nil) + bm.tmgr.EXPECT().Partitions(gomock.Any()).Return([]int32{0}, nil).AnyTimes() + bm.tmgr.EXPECT().GetOffset(table, gomock.Any(), sarama.OffsetNewest).Return(func() int64 { defer func() { - if x := recover(); x != nil { - ensure.StringContains(t, fmt.Sprintf("%v", x), errSome.Error()) - } + current++ }() - updates, err = p.process(msg, st, &wg, pstats) - }() - ensure.Nil(t, err) - ensure.DeepEqual(t, updates, 1) - err = doTimed(t, func() { - <-canceled - }) - ensure.Nil(t, err) - -} - -func TestNewProcessor(t *testing.T) { - _, err := NewProcessor(nil, DefineGroup(group)) - ensure.NotNil(t, err) - - _, err = NewProcessor(nil, DefineGroup(group, Input("topic", rawCodec, nil))) - ensure.NotNil(t, err) - - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - consumer = mock.NewMockConsumer(ctrl) - producer = mock.NewMockProducer(ctrl) - tm = mock.NewMockTopicManager(ctrl) - ) - - // error in applyOptions - _, err = NewProcessor(nil, - DefineGroup(group, Input(topic, rawCodec, cb)), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - func(o *poptions, gg *GroupGraph) { - o.builders.storage = nil - }, - ) - ensure.NotNil(t, err) - - // error creating topic manager - _, err = NewProcessor(nil, - DefineGroup(group, Input(topic, rawCodec, cb)), - WithTopicManagerBuilder(createFailedTopicManagerBuilder(tm)), - ) - ensure.NotNil(t, err) - - // error closing topic manager - tm.EXPECT().Partitions(topic).Return([]int32{0, 1}, nil) - tm.EXPECT().Close().Return(errSome) - _, err = NewProcessor(nil, - DefineGroup(group, Input(topic, rawCodec, cb)), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - ) - ensure.NotNil(t, err) - - // error calling Partitions in copartitioned - tm.EXPECT().Partitions(topic).Return([]int32{0, 1}, errSome) - tm.EXPECT().Close().Return(nil) - _, err = NewProcessor(nil, - DefineGroup(group, Input(topic, rawCodec, cb)), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - ) - ensure.NotNil(t, err) - - // error with partition gap in copartitioned - tm.EXPECT().Partitions(topic).Return([]int32{0, 2}, nil) - tm.EXPECT().Close().Return(nil) - _, err = NewProcessor(nil, - DefineGroup(group, Input(topic, rawCodec, cb)), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - ) - ensure.NotNil(t, err) - - // error with non-copartitioned topics - tm.EXPECT().Partitions(topic).Return([]int32{0, 1}, nil) - tm.EXPECT().Partitions(topic2).Return([]int32{0, 1, 2}, nil) - tm.EXPECT().Close().Return(nil) - _, err = NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Input(topic2, rawCodec, cb), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - ) - ensure.NotNil(t, err) - - // error ensuring streams - tm.EXPECT().Partitions(topic).Return([]int32{0, 1}, nil) - tm.EXPECT().EnsureStreamExists("group-loop", 2).Return(errSome) - tm.EXPECT().Close().Return(nil) - _, err = NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Loop(rawCodec, cb), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - ) - ensure.NotNil(t, err) - - // error ensuring table - tm.EXPECT().Partitions(topic).Return([]int32{0, 1}, nil) - tm.EXPECT().EnsureTableExists("group-table", 2).Return(errSome) - tm.EXPECT().Close().Return(nil) - _, err = NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Persist(rawCodec), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - WithStorageBuilder(storage.MemoryBuilder()), - ) - ensure.NotNil(t, err) - - // error creating views - tm.EXPECT().Partitions(topic).Return([]int32{0, 1}, nil) - // lookup table is allowed to be not copartitioned with input - tm.EXPECT().Partitions(table).Return([]int32{0, 1, 2}, errSome) - tm.EXPECT().Close().Return(nil).Times(2) - _, err = NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Lookup(table, rawCodec)), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - WithStorageBuilder(storage.MemoryBuilder()), - ) - ensure.NotNil(t, err) - - // successfully create processor - tm.EXPECT().Partitions(topic).Return([]int32{0, 1}, nil) - tm.EXPECT().Partitions(string(topic2)).Return([]int32{0, 1}, nil) - tm.EXPECT().EnsureStreamExists(loopName(group), 2).Return(nil) - tm.EXPECT().EnsureTableExists(tableName(group), 2).Return(nil) - tm.EXPECT().Close().Return(nil) - p, err := NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Input(topic2, rawCodec, cb), - Loop(rawCodec, cb), - Persist(rawCodec), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - WithStorageBuilder(storage.MemoryBuilder()), - ) - ensure.Nil(t, err) - ensure.DeepEqual(t, p.graph.GroupTable().Topic(), tableName(group)) - ensure.DeepEqual(t, p.graph.LoopStream().Topic(), loopName(group)) - ensure.True(t, p.partitionCount == 2) - ensure.True(t, len(p.graph.inputs()) == 2) - ensure.False(t, p.isStateless()) - - // successfully create stateless processor - tm.EXPECT().Partitions(topic).Return([]int32{0, 1}, nil) - tm.EXPECT().Partitions(string(topic2)).Return([]int32{0, 1}, nil) - tm.EXPECT().Close().Return(nil) - p, err = NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Input(topic2, rawCodec, cb), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - WithConsumerBuilder(createConsumerBuilder(consumer)), - WithProducerBuilder(createProducerBuilder(producer)), - ) - ensure.Nil(t, err) - ensure.True(t, p.graph.GroupTable() == nil) - ensure.True(t, p.graph.LoopStream() == nil) - ensure.True(t, p.partitionCount == 2) - ensure.True(t, len(p.graph.inputs()) == 2) - ensure.True(t, p.isStateless()) - - // successfully create a processor with tables - tm.EXPECT().Partitions(topic).Return([]int32{0, 1}, nil) - tm.EXPECT().Partitions(table).Return([]int32{0, 1}, nil) - tm.EXPECT().Partitions(table2).Return([]int32{0, 1, 2}, nil) - tm.EXPECT().Close().Return(nil).Times(2) - p, err = NewProcessor(nil, - DefineGroup(group, - Input(topic, rawCodec, cb), - Join(table, rawCodec), - Lookup(table2, rawCodec), - ), - WithTopicManagerBuilder(createTopicManagerBuilder(tm)), - WithConsumerBuilder(createConsumerBuilder(consumer)), - WithProducerBuilder(createProducerBuilder(producer)), - WithStorageBuilder(storage.MemoryBuilder()), - ) - ensure.Nil(t, err) - ensure.True(t, p.graph.GroupTable() == nil) - ensure.True(t, p.graph.LoopStream() == nil) - ensure.True(t, p.partitionCount == 2) - ensure.True(t, len(p.views[table2].partitions) == 3) - ensure.True(t, len(p.graph.copartitioned()) == 2) - ensure.True(t, len(p.graph.inputs()) == 3) - ensure.True(t, p.isStateless()) + return current + }(), nil) + bm.tmgr.EXPECT().GetOffset(table, gomock.Any(), sarama.OffsetOldest).Return(func() int64 { + return 0 + }(), nil) } -func TestProcessor_StartFails(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - consumer := mock.NewMockConsumer(ctrl) - - // error creating consumer - done := make(chan bool) - p := createProcessor(t, ctrl, consumer, 2, nullStorageBuilder()) - p.opts.builders.consumer = createFailedConsumerBuilder() - go func() { - errs := p.Run(context.Background()) - ensure.NotNil(t, errs) - close(done) - }() - err := doTimed(t, func() { <-done }) - ensure.Nil(t, err) - - // error creating producer and closing consumer - done = make(chan bool) - p = createProcessor(t, ctrl, consumer, 2, nullStorageBuilder()) - p.opts.builders.producer = createFailedProducerBuilder() - consumer.EXPECT().Close().Return(errSome) - go func() { - errs := p.Run(context.Background()) - ensure.NotNil(t, errs) - ensure.StringContains(t, errs.Error(), "creating producer") - ensure.StringContains(t, errs.Error(), "closing consumer") - close(done) - }() - err = doTimed(t, func() { <-done }) - ensure.Nil(t, err) - - // error starting lookup tables and closing producer - done = make(chan bool) - st := mock.NewMockStorage(ctrl) - p = createProcessorWithLookupTable(t, ctrl, consumer, 2, createStorageBuilder(st)) - producer := mock.NewMockProducer(ctrl) - // tm := mock.NewMockTopicManager(ctrl) - p.opts.builders.producer = createProducerBuilder(producer) - // p.opts.builders.topicmgr = createTopicManagerBuilder(tm) - wait := make(chan bool) - ch := make(chan kafka.Event) - consumer.EXPECT().Subscribe(map[string]int64{topic: -1}).Return(nil) - consumer.EXPECT().Events().Return(ch).Do(func() { <-wait }).Times(2) // view + processor - st.EXPECT().Open().Do(func() { close(wait) }).Return(errSome) - st.EXPECT().Open().Return(errSome) - st.EXPECT().Close().Times(2) - consumer.EXPECT().Close().Return(nil).Times(2) // view + processor - producer.EXPECT().Close().Return(errSome) - go func() { - errs := p.Run(context.Background()) - ensure.NotNil(t, errs) - ensure.StringContains(t, errs.Error(), "closing producer") - ensure.StringContains(t, errs.Error(), "opening storage") - close(done) - }() - err = doTimed(t, func() { <-done }) - ensure.Nil(t, err) - - // error subscribing topics - done = make(chan bool) - p = createProcessor(t, ctrl, consumer, 2, nullStorageBuilder()) - consumer.EXPECT().Subscribe(topOff).Return(errSome) - consumer.EXPECT().Close().Return(nil) - go func() { - errs := p.Run(context.Background()) - ensure.NotNil(t, errs) - ensure.StringContains(t, errs.Error(), "subscribing topics") - close(done) - }() - err = doTimed(t, func() { <-done }) - ensure.Nil(t, err) -} - -func TestProcessor_StartStopEmpty(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - consumer = mock.NewMockConsumer(ctrl) - wait = make(chan bool) - final = make(chan bool) - ch = make(chan kafka.Event) - p = createProcessor(t, ctrl, consumer, 2, nullStorageBuilder()) - ) - - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).Do(func() { close(wait) }) - ctx, cancel := context.WithCancel(context.Background()) - go func() { - err := p.Run(ctx) - ensure.Nil(t, err) - close(final) - }() - - consumer.EXPECT().Close().Return(nil).Do(func() { close(ch) }) - err := doTimed(t, func() { - <-wait - cancel() - <-final - }) - ensure.Nil(t, err) -} - -func TestProcessor_StartStopEmptyError(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - consumer = mock.NewMockConsumer(ctrl) - final = make(chan bool) - wait = make(chan bool) - ch = make(chan kafka.Event) - p = createProcessor(t, ctrl, consumer, 2, nullStorageBuilder()) - ) - - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).Do(func() { close(wait) }) - ctx, cancel := context.WithCancel(context.Background()) - go func() { - err := p.Run(ctx) - ensure.NotNil(t, err) - close(final) - }() - - consumer.EXPECT().Close().Return(errors.New("some error")).Do(func() { close(ch) }) - err := doTimed(t, func() { - <-wait - cancel() - <-final - }) - ensure.Nil(t, err) -} - -// start processor and receives an error from Kafka in the events -// channel before rebalance. -func TestProcessor_StartWithErrorBeforeRebalance(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - type TestCase struct { - name string - event kafka.Event - } - tests := []TestCase{ - {"error", &kafka.Error{Err: errors.New("something")}}, - {"message", new(kafka.Message)}, - {"EOF", new(kafka.EOF)}, - {"BOF", new(kafka.BOF)}, - } - - for _, tc := range tests { - t.Run(tc.name, func(t *testing.T) { - var ( - err error - consumer = mock.NewMockConsumer(ctrl) - st = mock.NewMockStorage(ctrl) - sb = func(topic string, par int32) (storage.Storage, error) { - return st, nil - } - final = make(chan bool) - ch = make(chan kafka.Event) - p = createProcessor(t, ctrl, consumer, 3, sb) - ) - - gomock.InOrder( - consumer.EXPECT().Subscribe(topOff).Return(nil), - consumer.EXPECT().Events().Return(ch), - consumer.EXPECT().Close().Do(func() { close(ch) }), - ) - go func() { - err = p.Run(context.Background()) - ensure.NotNil(t, err) - close(final) - }() - - ch <- tc.event - - err = doTimed(t, func() { - <-final - }) - ensure.Nil(t, err) - }) +// accumulate is a callback that increments the +// table value by the incoming message. +// Persist and incoming codecs must be codec.Int64 +func accumulate(ctx Context, msg interface{}) { + inc := msg.(int64) + val := ctx.Value() + if val == nil { + ctx.SetValue(inc) + } else { + ctx.SetValue(val.(int64) + inc) } } -// start processor and receives an error from Kafka in the events -// channel after rebalance. -func TestProcessor_StartWithErrorAfterRebalance(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - err error - consumer = mock.NewMockConsumer(ctrl) - st = mock.NewMockStorage(ctrl) - sb = func(topic string, par int32) (storage.Storage, error) { - return st, nil - } - final = make(chan bool) - ch = make(chan kafka.Event) - p = createProcessor(t, ctrl, consumer, 3, sb) - value = []byte("value") - ) - // -- expectations -- - // 1. start - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).AnyTimes() - // 2. rebalance - st.EXPECT().Open().Times(3) - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil).Times(3) - consumer.EXPECT().AddPartition(tableName(group), int32(0), int64(123)) - consumer.EXPECT().AddPartition(tableName(group), int32(1), int64(123)) - consumer.EXPECT().AddPartition(tableName(group), int32(2), int64(123)) - // 3. message - gomock.InOrder( - st.EXPECT().Set("key", value).Return(nil), - st.EXPECT().SetOffset(int64(1)), - st.EXPECT().MarkRecovered(), - ) - // 4. error - consumer.EXPECT().RemovePartition(tableName(group), int32(0)) - consumer.EXPECT().RemovePartition(tableName(group), int32(1)) - consumer.EXPECT().RemovePartition(tableName(group), int32(2)) - st.EXPECT().Close().Times(3) - consumer.EXPECT().Close().Do(func() { close(ch) }) - - // -- test -- - // 1. start - go func() { - err = p.Run(context.Background()) - ensure.NotNil(t, err) - close(final) - }() - - // 2. rebalance - ensure.True(t, len(p.partitions) == 0) - ch <- (*kafka.Assignment)(&map[int32]int64{0: -1, 1: -1, 2: -1}) - err = syncWith(t, ch, -1) // with processor - ensure.Nil(t, err) - ensure.True(t, len(p.partitions) == 3) - - // 3. message - ch <- &kafka.Message{ - Topic: tableName(group), - Partition: 1, - Offset: 1, - Key: "key", - Value: value, - } - err = syncWith(t, ch, 1) // with partition - ensure.Nil(t, err) - - // 4. receive error - ch <- new(kafka.Error) - - // 5. stop - err = doTimed(t, func() { <-final }) - ensure.Nil(t, err) -} - -// start processor with table and receives an error from Kafka in the events -// channel after rebalance. -func TestProcessor_StartWithTableWithErrorAfterRebalance(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - err error - consumer = mock.NewMockConsumer(ctrl) - producer = mock.NewMockProducer(ctrl) - st = mock.NewMockStorage(ctrl) - sb = func(topic string, par int32) (storage.Storage, error) { - return st, nil - } - final = make(chan bool) - ch = make(chan kafka.Event) - p = createProcessorWithTable(t, ctrl, consumer, producer, 3, sb) - value = []byte("value") - blockit = make(chan bool) - unblocked = make(chan bool) - ) - p.graph.callbacks[topic] = func(ctx Context, msg interface{}) { - fmt.Println("hallodfads", msg) - defer close(unblocked) - <-blockit - fmt.Println("unblocked") - } - - // -- expectations -- - // 1. start - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).AnyTimes() - // 2. rebalance - st.EXPECT().Open().Times(6) - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil).Times(6) - consumer.EXPECT().AddPartition(tableName(group), int32(0), int64(123)) - consumer.EXPECT().AddPartition(tableName(group), int32(1), int64(123)) - consumer.EXPECT().AddPartition(tableName(group), int32(2), int64(123)) - consumer.EXPECT().AddPartition(table, int32(0), int64(123)) - consumer.EXPECT().AddPartition(table, int32(1), int64(123)) - consumer.EXPECT().AddPartition(table, int32(2), int64(123)) - // 3. EOF messages - st.EXPECT().MarkRecovered().Times(3) - // 4. messages - consumer.EXPECT().Commit(topic, int32(1), int64(2)) - // 5. error - consumer.EXPECT().Close().Do(func() { close(ch) }) - consumer.EXPECT().RemovePartition(tableName(group), int32(0)) - consumer.EXPECT().RemovePartition(tableName(group), int32(1)) - consumer.EXPECT().RemovePartition(tableName(group), int32(2)) - consumer.EXPECT().RemovePartition(table, int32(0)) - consumer.EXPECT().RemovePartition(table, int32(1)) - consumer.EXPECT().RemovePartition(table, int32(2)) - st.EXPECT().Close().Times(6) - producer.EXPECT().Close() - - // -- test -- - // 1. start - go func() { - err = p.Run(context.Background()) - ensure.NotNil(t, err) - close(final) - }() - - // 2. rebalance - ensure.True(t, len(p.partitions) == 0) - ensure.True(t, len(p.partitionViews) == 0) - ch <- (*kafka.Assignment)(&map[int32]int64{0: -1, 1: -1, 2: -1}) - err = syncWith(t, ch, -1) // with processor - ensure.Nil(t, err) - ensure.True(t, len(p.partitions) == 3) - ensure.True(t, len(p.partitionViews) == 3) - - // 3. message - ch <- &kafka.EOF{ - Topic: tableName(group), - Hwm: 0, - Partition: 0, - } - err = syncWith(t, ch, 0) // with partition - ensure.Nil(t, err) - ch <- &kafka.EOF{ - Topic: tableName(group), - Hwm: 0, - Partition: 1, - } - err = syncWith(t, ch, 1) // with partition - ensure.Nil(t, err) - ch <- &kafka.EOF{ - Topic: tableName(group), - Hwm: 0, - Partition: 2, - } - err = syncWith(t, ch, 2) // with partition - ensure.Nil(t, err) - - // 4. heavy message - ch <- &kafka.Message{ - Topic: topic, - Partition: 1, - Offset: 2, - Key: "key", - Value: value, - } - // dont wait for that - - // 4. receive error - ch <- new(kafka.Error) - - // sync with partition (should be unblocked) - close(blockit) - <-unblocked +func TestProcessor_Run(t *testing.T) { + + t.Run("input-persist", func(t *testing.T) { + ctrl, bm := createMockBuilder(t) + defer ctrl.Finish() + var ( + topic = "test-table" + toEmit = []*sarama.ConsumerMessage{ + &sarama.ConsumerMessage{Topic: "input", + Value: []byte(strconv.FormatInt(3, 10)), + Key: []byte("test-key-1"), + }, + &sarama.ConsumerMessage{Topic: "input", + Value: []byte(strconv.FormatInt(3, 10)), + Key: []byte("test-key-2"), + }, + } + ) + + expectCGConsume(bm, topic, toEmit) + expectCGEmit(bm, topic, toEmit) + + groupBuilder, cg := createTestConsumerGroupBuilder(t) + consBuilder, cons := createTestConsumerBuilder(t) + _ = cg + _ = cons + + graph := DefineGroup("test", + Input("input", new(codec.Int64), accumulate), + Persist(new(codec.Int64)), + ) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*1000) + defer cancel() + + newProc, err := NewProcessor([]string{"localhost:9092"}, graph, + bm.createProcessorOptions(consBuilder, groupBuilder)..., + ) + test.AssertNil(t, err) + var ( + procErr error + done = make(chan struct{}) + ) + + cons.ExpectConsumePartition(topic, 0, 0) + + go func() { + defer close(done) + procErr = newProc.Run(ctx) + }() - // 5. stop - err = doTimed(t, func() { - <-final - }) - ensure.Nil(t, err) -} + newProc.WaitForReady() -func TestProcessor_Start(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() + // if there was an error during startup, no point in sending messages + // and waiting for them to be delivered + test.AssertNil(t, procErr) - var ( - err error - consumer = mock.NewMockConsumer(ctrl) - st = mock.NewMockStorage(ctrl) - sb = func(topic string, par int32) (storage.Storage, error) { - return st, nil + for _, msg := range toEmit { + cg.SendMessageWait(msg) } - final = make(chan bool) - ch = make(chan kafka.Event) - p = createProcessor(t, ctrl, consumer, 3, sb) - value = []byte("value") - ) - - // -- expectations -- - // 1. start - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).AnyTimes() - // 2. rebalance - st.EXPECT().Open().Times(4) - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil).Times(4) - consumer.EXPECT().AddPartition(tableName(group), int32(0), int64(123)) - consumer.EXPECT().AddPartition(tableName(group), int32(1), int64(123)) - consumer.EXPECT().AddPartition(tableName(group), int32(2), int64(123)) - // 3. load message partition 1 - st.EXPECT().Set("key", value).Return(nil) - st.EXPECT().SetOffset(int64(1)) - st.EXPECT().MarkRecovered() - // 4. end of recovery partition 1 - gomock.InOrder( - consumer.EXPECT().RemovePartition(tableName(group), int32(1)), - consumer.EXPECT().AddGroupPartition(int32(1)), - ) - // 5. process message partition 1 - consumer.EXPECT().Commit(topic, int32(1), int64(1)) - // 6. new assignment remove partition 1 and 2 - st.EXPECT().Close() // partition 0 close (only temporarily) - consumer.EXPECT().RemovePartition(tableName(group), int32(0)) - st.EXPECT().Close() // partition 1 close - consumer.EXPECT().RemovePartition(tableName(group), int32(2)) - st.EXPECT().Close() // partition 2 close - // add partition 0 again - consumer.EXPECT().AddPartition(tableName(group), int32(0), int64(123)) - // 7. stop processor - consumer.EXPECT().Close() //.Do(func() { close(ch) }) - consumer.EXPECT().RemovePartition(tableName(group), int32(0)) - st.EXPECT().Close() - - // -- test -- - ctx, cancel := context.WithCancel(context.Background()) - // 1. start - go func() { - err = p.Run(ctx) - ensure.Nil(t, err) - close(final) - }() - // 2. rebalance - ensure.True(t, len(p.partitions) == 0) - ch <- (*kafka.Assignment)(&map[int32]int64{0: -1, 1: -1, 2: -1}) - err = syncWith(t, ch, -1) // with processor - ensure.Nil(t, err) - ensure.True(t, len(p.partitions) == 3) + val, err := newProc.Get("test-key-1") + test.AssertNil(t, err) + test.AssertEqual(t, val.(int64), int64(3)) - // 3. load message partition 1 - ch <- &kafka.Message{ - Topic: tableName(group), - Partition: 1, - Offset: 1, - Key: "key", - Value: value, - } - err = syncWith(t, ch, 1) // with partition 1 - ensure.Nil(t, err) - - // 4. end of recovery partition 1 - ch <- &kafka.EOF{Partition: 1} - err = syncWith(t, ch, 1) // with partition 1 - ensure.Nil(t, err) - - // 5. process message partition 1 - ch <- &kafka.Message{ - Topic: topic, - Partition: 1, - Offset: 1, - Key: "key", - Value: value, - } - err = syncWith(t, ch, 1) // with partition 1 - ensure.Nil(t, err) + val, err = newProc.Get("test-key-2") + test.AssertNil(t, err) + test.AssertEqual(t, val.(int64), int64(3)) - // 6. new assignment remove partition 1 and 2 - ch <- (*kafka.Assignment)(&map[int32]int64{0: -1}) - err = syncWith(t, ch, 1, 2) // with partition 1 and 2 - ensure.Nil(t, err) - ensure.True(t, len(p.partitions) == 1) - - // 7. stop processor - err = doTimed(t, func() { - cancel() - <-final + // shutdown + newProc.Stop() + <-done + test.AssertNil(t, procErr) }) - ensure.Nil(t, err) -} - -func TestProcessor_StartStateless(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - var ( - consumer = mock.NewMockConsumer(ctrl) - producer = mock.NewMockProducer(ctrl) - final = make(chan bool) - ch = make(chan kafka.Event) - p = createProcessorStateless(ctrl, consumer, producer, 3, emptyRebalanceCallback) - ) - - // -- expectactions -- - // 1. start - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).AnyTimes() - // 2. rebalance - consumer.EXPECT().AddGroupPartition(int32(0)) - consumer.EXPECT().AddGroupPartition(int32(1)) - // 3. stop processor - consumer.EXPECT().Close().Return(nil).Do(func() { close(ch) }) - producer.EXPECT().Close().Return(nil) - - // -- test -- - ctx, cancel := context.WithCancel(context.Background()) - // 1. start - go func() { - err := p.Run(ctx) - ensure.Nil(t, err) - close(final) - }() - - // 2. rebalance - ensure.True(t, len(p.partitions) == 0) - ch <- (*kafka.Assignment)(&map[int32]int64{0: -1, 1: -1}) - err := syncWith(t, ch, -1, 1, 2) - ensure.Nil(t, err) - ensure.True(t, len(p.partitions) == 2) + t.Run("loopback", func(t *testing.T) { + ctrl, bm := createMockBuilder(t) + defer ctrl.Finish() + + var ( + topic = "test-table" + loop = "test-loop" + toEmit = []*sarama.ConsumerMessage{ + &sarama.ConsumerMessage{Topic: "input", + Value: []byte(strconv.FormatInt(23, 10)), + Key: []byte("test-key"), + }, + } + ) + + expectCGConsume(bm, topic, toEmit) + expectCGLoop(bm, loop, toEmit) + + groupBuilder, cg := createTestConsumerGroupBuilder(t) + consBuilder, cons := createTestConsumerBuilder(t) + _ = cg + _ = cons + + graph := DefineGroup("test", + // input passes to loopback + Input("input", new(codec.Int64), func(ctx Context, msg interface{}) { + ctx.Loopback(ctx.Key(), msg) + }), + // this will not be called in the test but we define it, otherwise the context will raise an error + Loop(new(codec.Int64), accumulate), + Persist(new(codec.Int64)), + ) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + + newProc, err := NewProcessor([]string{"localhost:9092"}, graph, + bm.createProcessorOptions(consBuilder, groupBuilder)..., + ) + test.AssertNil(t, err) + var ( + procErr error + done = make(chan struct{}) + ) + + cons.ExpectConsumePartition(topic, 0, 0) + + go func() { + defer close(done) + procErr = newProc.Run(ctx) + }() - // 3. stop processor - err = doTimed(t, func() { - cancel() - <-final - }) - ensure.Nil(t, err) -} + newProc.WaitForReady() -func TestProcessor_StartWithTable(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() + // if there was an error during startup, no point in sending messages + // and waiting for them to be delivered + test.AssertNil(t, procErr) - var ( - err error - consumer = mock.NewMockConsumer(ctrl) - producer = mock.NewMockProducer(ctrl) - st = mock.NewMockStorage(ctrl) - sb = func(topic string, par int32) (storage.Storage, error) { - return st, nil + for _, msg := range toEmit { + cg.SendMessageWait(msg) } - final = make(chan bool) - ch = make(chan kafka.Event) - p = createProcessorWithTable(t, ctrl, consumer, producer, 3, sb) - value = []byte("value") - ) - - // -- expectations -- - // 1. start - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).AnyTimes() - // 2. rebalance - st.EXPECT().Open().Times(6) - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil).Times(6) - consumer.EXPECT().AddPartition(tableName(group), int32(0), int64(123)) - consumer.EXPECT().AddPartition(tableName(group), int32(1), int64(123)) - consumer.EXPECT().AddPartition(tableName(group), int32(2), int64(123)) - consumer.EXPECT().AddPartition(table, int32(0), int64(123)) - consumer.EXPECT().AddPartition(table, int32(1), int64(123)).Times(2) - consumer.EXPECT().AddPartition(table, int32(2), int64(123)) - // 3. message to group table - st.EXPECT().Set("key", value).Return(nil) - st.EXPECT().SetOffset(int64(1)) - st.EXPECT().MarkRecovered() - // 4. finish recovery of partition 1 - gomock.InOrder( - consumer.EXPECT().RemovePartition(tableName(group), int32(1)), - consumer.EXPECT().AddGroupPartition(int32(1)), - ) - // 5. process messages in partition 1 - gomock.InOrder( - consumer.EXPECT().Commit(topic, int32(1), int64(1)), - ) - // 6. rebalance (only keep partition 0) - st.EXPECT().Close().Times(4) // close group and other table partitions - consumer.EXPECT().RemovePartition(table, int32(1)).Times(2) - consumer.EXPECT().RemovePartition(table, int32(2)) - consumer.EXPECT().RemovePartition(tableName(group), int32(2)) - // also partition 0 will be temporarily closed - st.EXPECT().Close().Times(2) // close group and other table of partition 0 - consumer.EXPECT().RemovePartition(table, int32(0)) - consumer.EXPECT().RemovePartition(tableName(group), int32(0)) - // add partition 0 again - st.EXPECT().Open().Times(2) - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil).Times(2) - consumer.EXPECT().AddPartition(tableName(group), int32(0), int64(123)) - consumer.EXPECT().AddPartition(table, int32(0), int64(123)) - // 7. stop processor - consumer.EXPECT().Close().Do(func() { close(ch) }) - consumer.EXPECT().RemovePartition(table, int32(0)) - consumer.EXPECT().RemovePartition(tableName(group), int32(0)) - st.EXPECT().MarkRecovered() - st.EXPECT().Close().Times(2) // close group table and other table - producer.EXPECT().Close().Return(nil) - - // -- test -- - ctx, cancel := context.WithCancel(context.Background()) - // 1. start - go func() { - procErrs := p.Run(ctx) - ensure.Nil(t, procErrs) - close(final) - }() - - // 2. rebalance - ensure.True(t, len(p.partitions) == 0) - ch <- (*kafka.Assignment)(&map[int32]int64{0: -1, 1: -1, 2: -1}) - err = syncWith(t, ch) - ensure.Nil(t, err) - ensure.True(t, len(p.partitions) == 3) - - // 3. message to group table - ch <- &kafka.Message{ - Topic: tableName(group), - Partition: 1, - Offset: 1, - Key: "key", - Value: value, - } - err = syncWith(t, ch, 1) - ensure.Nil(t, err) - - // 4. finish recovery of partition 1 - ch <- &kafka.EOF{ - Partition: 1, - } - ensure.False(t, p.partitionViews[1][table].recovered()) - time.Sleep(delayProxyInterval) - ensure.False(t, p.partitionViews[1][table].recovered()) - ch <- &kafka.EOF{ - Topic: table, - Partition: 1, - Hwm: 123, - } - err = syncWith(t, ch) - ensure.Nil(t, err) - time.Sleep(delayProxyInterval) - ensure.True(t, p.partitionViews[1][table].recovered()) - // 5. process messages in partition 1 - ch <- &kafka.Message{ - Topic: topic, - Partition: 1, - Offset: 1, - Key: "key", - Value: value, - } - err = syncWith(t, ch, 1) - ensure.Nil(t, err) - - // 6. rebalance - ch <- (*kafka.Assignment)(&map[int32]int64{0: -1}) - err = syncWith(t, ch, 1, 2) // synchronize with partitions 1 and 2 - ensure.Nil(t, err) - ensure.True(t, len(p.partitions) == 1) - - // 7. stop processor - err = doTimed(t, func() { - cancel() - <-final + // shutdown + newProc.Stop() + <-done + test.AssertNil(t, procErr) }) - ensure.Nil(t, err) -} - -func TestProcessor_rebalanceError(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - consumer = mock.NewMockConsumer(ctrl) - wait = make(chan bool) - ch = make(chan kafka.Event) - p = createProcessor(t, ctrl, consumer, 1, - func(topic string, partition int32) (storage.Storage, error) { - return nil, errors.New("some error") - }) - ) - - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).AnyTimes() - consumer.EXPECT().Close().Return(nil).Do(func() { - close(ch) - }) - - ctx, cancel := context.WithCancel(context.Background()) - go func() { - err := p.Run(ctx) - ensure.NotNil(t, err) - close(wait) - }() + t.Run("consume-error", func(t *testing.T) { + ctrl, bm := createMockBuilder(t) + defer ctrl.Finish() + + bm.tmgr.EXPECT().Close().Times(1) + bm.tmgr.EXPECT().Partitions(gomock.Any()).Return([]int32{0}, nil).Times(1) + bm.producer.EXPECT().Close().Times(1) + + groupBuilder, cg := createTestConsumerGroupBuilder(t) + consBuilder, cons := createTestConsumerBuilder(t) + _ = cg + _ = cons + + graph := DefineGroup("test", + // not really used, we're failing anyway + Input("input", new(codec.Int64), accumulate), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + newProc, err := NewProcessor([]string{"localhost:9092"}, graph, + bm.createProcessorOptions(consBuilder, groupBuilder)..., + ) + test.AssertNil(t, err) + var ( + procErr error + done = make(chan struct{}) + ) + + go func() { + defer close(done) + procErr = newProc.Run(ctx) + }() - // assignment arrives - ensure.True(t, len(p.partitions) == 0) - ch <- (*kafka.Assignment)(&map[int32]int64{0: -1}) + newProc.WaitForReady() - // stop processor - err := doTimed(t, func() { + // if there was an error during startup, no point in sending messages + // and waiting for them to be delivered + test.AssertNil(t, procErr) + cg.SendError(fmt.Errorf("test-error")) cancel() - <-wait - }) - ensure.Nil(t, err) -} - -func TestProcessor_HasGet(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - st = mock.NewMockStorage(ctrl) - sb = func(topic string, partition int32) (storage.Storage, error) { - return st, nil - } - consumer = mock.NewMockConsumer(ctrl) - ch = make(chan kafka.Event) - wait = make(chan bool) - p = createProcessor(t, ctrl, consumer, 1, sb) - ) - - ensure.True(t, p.partitionCount == 1) - - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).AnyTimes() - - ctx, cancel := context.WithCancel(context.Background()) - go func() { - procErrs := p.Run(ctx) - ensure.Nil(t, procErrs) - close(wait) - }() - - // assignment arrives - - ensure.True(t, len(p.partitions) == 0) - gomock.InOrder( - st.EXPECT().Open(), - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil), - consumer.EXPECT().AddPartition(tableName(group), int32(0), int64(123)), - ) - ch <- (*kafka.Assignment)(&map[int32]int64{0: -1}) - ch <- new(kafka.NOP) - ensure.True(t, len(p.partitions) == 1) + <-done + // the errors sent back by the consumergroup do not lead to a failure of the processor + test.AssertNil(t, procErr) - gomock.InOrder( - st.EXPECT().Get("item1").Return([]byte("item1-value"), nil), - ) - - value, err := p.Get("item1") - ensure.Nil(t, err) - ensure.DeepEqual(t, value.(string), "item1-value") - - // stop processor - consumer.EXPECT().Close().Do(func() { close(ch) }) - consumer.EXPECT().RemovePartition(tableName(group), int32(0)) - st.EXPECT().Close() - - err = doTimed(t, func() { - cancel() - <-wait }) - ensure.Nil(t, err) -} - -func TestProcessor_HasGetStateless(t *testing.T) { - p := &Processor{graph: DefineGroup(group), opts: &poptions{hasher: DefaultHasher()}} - _, err := p.Get("item1") - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "stateless processor") - - p = &Processor{graph: DefineGroup(group, Persist(c)), opts: &poptions{hasher: DefaultHasher()}} - p.partitions = map[int32]*partition{ - 0: new(partition), - } - p.partitionCount = 0 - _, err = p.Get("item1") - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "0 partitions") - - p = &Processor{graph: DefineGroup(group, Persist(c)), opts: &poptions{hasher: DefaultHasher()}} - p.partitions = map[int32]*partition{ - 0: new(partition), - } - p.partitionCount = 2 - _, err = p.Get("item1") - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "does not contain partition 1") - - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - st := mock.NewMockStorage(ctrl) - p = &Processor{graph: DefineGroup(group, Persist(c)), opts: &poptions{hasher: DefaultHasher()}} - p.partitions = map[int32]*partition{ - 0: &partition{log: logger.Default(), st: &storageProxy{Storage: st, partition: 0}}, - } - p.partitionCount = 1 - - st.EXPECT().Get("item1").Return(nil, errors.New("some error")) - _, err = p.Get("item1") - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "error getting item1") - - st.EXPECT().Get("item1").Return(nil, nil) - value, err := p.Get("item1") - ensure.Nil(t, err) - ensure.True(t, value == nil) -} - -func TestProcessor_RebalanceCallback(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - var ( - consumer = mock.NewMockConsumer(ctrl) - producer = mock.NewMockProducer(ctrl) - final = make(chan bool) - ch = make(chan kafka.Event) - asmt = (*kafka.Assignment)(&map[int32]int64{0: -1, 1: -1}) - i = 0 - eAsmt = []kafka.Assignment{{}, *asmt} - rcb = func(a kafka.Assignment){ - ensure.DeepEqual(t, a, eAsmt[i]) - i += 1 - } - p = createProcessorStateless(ctrl, consumer, producer, 3, rcb) - ) - - // -- expectactions -- - // 1. start - consumer.EXPECT().Subscribe(topOff).Return(nil) - consumer.EXPECT().Events().Return(ch).AnyTimes() - // 2. rebalance - consumer.EXPECT().AddGroupPartition(int32(0)) - consumer.EXPECT().AddGroupPartition(int32(1)) - // 3. stop processor - consumer.EXPECT().Close().Return(nil).Do(func() { close(ch) }) - producer.EXPECT().Close().Return(nil) - - // -- test -- - ctx, cancel := context.WithCancel(context.Background()) - // 1. start - go func() { - err := p.Run(ctx) - ensure.Nil(t, err) - close(final) - }() + t.Run("consgroup-error", func(t *testing.T) { + ctrl, bm := createMockBuilder(t) + defer ctrl.Finish() + + bm.tmgr.EXPECT().Close().Times(1) + bm.tmgr.EXPECT().Partitions(gomock.Any()).Return([]int32{0}, nil).Times(1) + bm.producer.EXPECT().Close().Times(1) + + groupBuilder, cg := createTestConsumerGroupBuilder(t) + consBuilder, cons := createTestConsumerBuilder(t) + _ = cg + _ = cons + + graph := DefineGroup("test", + // not really used, we're failing anyway + Input("input", new(codec.Int64), accumulate), + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + newProc, err := NewProcessor([]string{"localhost:9092"}, graph, + bm.createProcessorOptions(consBuilder, groupBuilder)..., + ) + test.AssertNil(t, err) + var ( + procErr error + done = make(chan struct{}) + ) + + cg.FailOnConsume(fmt.Errorf("consume-error")) + + go func() { + defer close(done) + procErr = newProc.Run(ctx) + }() - // 2. rebalance - ensure.True(t, len(p.partitions) == 0) - ch <- asmt - err := syncWith(t, ch, -1, 1, 2) - ensure.Nil(t, err) - ensure.True(t, len(p.partitions) == 2) + newProc.WaitForReady() - // 3. stop processor - err = doTimed(t, func() { - cancel() - <-final + // if there was an error during startup, no point in sending messages + // and waiting for them to be delivered + <-done + // the errors sent back by the consumergroup do not lead to a failure of the processor + test.AssertTrue(t, strings.Contains(procErr.Error(), "consume-error")) }) - ensure.Nil(t, err) -} - -// Example shows how to use a callback. For each partition of the topics, a new -// goroutine will be created. Topics should be co-partitioned (they should have -// the same number of partitions and be partitioned by the same key). -func ExampleProcessor_simplest() { - var ( - brokers = []string{"127.0.0.1:9092"} - group Group = "group" - topic Stream = "topic" - ) - - consume := func(ctx Context, m interface{}) { - fmt.Printf("Hello world: %v", m) - } - - p, err := NewProcessor(brokers, DefineGroup(group, Input(topic, rawCodec, consume))) - if err != nil { - log.Fatalln(err) - } - - // start consumer with a goroutine (blocks) - ctx, cancel := context.WithCancel(context.Background()) - go func() { - err := p.Run(ctx) - panic(err) - }() - - // wait for bad things to happen - wait := make(chan os.Signal, 1) - signal.Notify(wait, syscall.SIGHUP, syscall.SIGINT, syscall.SIGTERM) - <-wait - cancel() } diff --git a/kafka/producer.go b/producer.go similarity index 94% rename from kafka/producer.go rename to producer.go index c576be6d..e5040c13 100644 --- a/kafka/producer.go +++ b/producer.go @@ -1,4 +1,4 @@ -package kafka +package goka import ( "fmt" @@ -62,6 +62,7 @@ func (p *producer) Close() error { // can be checked for errors asynchronously func (p *producer) Emit(topic string, key string, value []byte) *Promise { promise := NewPromise() + p.producer.Input() <- &sarama.ProducerMessage{ Topic: topic, Key: sarama.StringEncoder(key), @@ -83,7 +84,7 @@ func (p *producer) run() { if !ok { return } - err.Msg.Metadata.(*Promise).Finish(err.Err) + err.Msg.Metadata.(*Promise).Finish(nil, err.Err) } }() @@ -95,7 +96,7 @@ func (p *producer) run() { if !ok { return } - msg.Metadata.(*Promise).Finish(nil) + msg.Metadata.(*Promise).Finish(msg, nil) } }() } diff --git a/kafka/producer_test.go b/producer_test.go similarity index 92% rename from kafka/producer_test.go rename to producer_test.go index db50d0d3..096e1acc 100644 --- a/kafka/producer_test.go +++ b/producer_test.go @@ -1,6 +1,6 @@ -// +build kafka +// +build goka -package kafka +package goka import ( "log" @@ -15,8 +15,8 @@ import ( // examples-directory and do `make restart`), then run the tests with // `go test -v github.com/lovoo/goka/kafka/ -tags=kafka` func TestProducerError(t *testing.T) { - cfg := NewConfig().Config - p, err := NewProducer([]string{"localhost:9092"}, &cfg) + cfg := DefaultConfig() + p, err := NewProducer([]string{"localhost:9092"}, cfg) if err != nil { t.Fatalf("error creating producer: %v", err) diff --git a/kafka/promise.go b/promise.go similarity index 57% rename from kafka/promise.go rename to promise.go index c2f69c7e..18ef92f1 100644 --- a/kafka/promise.go +++ b/promise.go @@ -1,14 +1,19 @@ -package kafka +package goka -import "sync" +import ( + "sync" + + "github.com/Shopify/sarama" +) // Promise as in https://en.wikipedia.org/wiki/Futures_and_promises type Promise struct { sync.Mutex err error + msg *sarama.ProducerMessage finished bool - callbacks []func(err error) + callbacks []func(msg *sarama.ProducerMessage, err error) } // NewPromise creates a new Promise @@ -24,33 +29,41 @@ func (p *Promise) executeCallbacks() { return } for _, s := range p.callbacks { - s(p.err) + s(p.msg, p.err) } // mark as finished p.finished = true } // Then chains a callback to the Promise -func (p *Promise) Then(s func(err error)) *Promise { +func (p *Promise) Then(callback func(err error)) *Promise { + return p.ThenWithMessage(func(_ *sarama.ProducerMessage, err error) { + callback(err) + }) +} + +// ThenWithMessage chains a callback to the Promise +func (p *Promise) ThenWithMessage(callback func(msg *sarama.ProducerMessage, err error)) *Promise { p.Lock() defer p.Unlock() // promise already run, call the callback immediately if p.finished { - s(p.err) + callback(p.msg, p.err) // append it to the subscribers otherwise } else { - p.callbacks = append(p.callbacks, s) + p.callbacks = append(p.callbacks, callback) } return p } // Finish finishes the promise by executing all callbacks and saving the message/error for late subscribers -func (p *Promise) Finish(err error) *Promise { +func (p *Promise) Finish(msg *sarama.ProducerMessage, err error) *Promise { p.Lock() defer p.Unlock() p.err = err + p.msg = msg p.executeCallbacks() return p diff --git a/kafka/promise_test.go b/promise_test.go similarity index 53% rename from kafka/promise_test.go rename to promise_test.go index b284acf5..e3e03abc 100644 --- a/kafka/promise_test.go +++ b/promise_test.go @@ -1,10 +1,10 @@ -package kafka +package goka import ( "errors" "testing" - "github.com/facebookgo/ensure" + "github.com/lovoo/goka/internal/test" ) func TestPromise_thenBeforeFinish(t *testing.T) { @@ -15,24 +15,24 @@ func TestPromise_thenBeforeFinish(t *testing.T) { promiseErr = err }) - p.Finish(errors.New("test")) + p.Finish(nil, errors.New("test")) - ensure.DeepEqual(t, promiseErr.Error(), "test") + test.AssertEqual(t, promiseErr.Error(), "test") // repeating finish won't change result - p.Finish(errors.New("test-whatever")) + p.Finish(nil, errors.New("test-whatever")) - ensure.DeepEqual(t, promiseErr.Error(), "test") + test.AssertEqual(t, promiseErr.Error(), "test") } func TestPromise_thenAfterFinish(t *testing.T) { p := new(Promise) var promiseErr error - p.Finish(errors.New("test")) + p.Finish(nil, errors.New("test")) p.Then(func(err error) { promiseErr = err }) - ensure.DeepEqual(t, promiseErr.Error(), "test") + test.AssertEqual(t, promiseErr.Error(), "test") } diff --git a/proxy.go b/proxy.go index 48946ab0..4c89f141 100644 --- a/proxy.go +++ b/proxy.go @@ -1,90 +1,9 @@ package goka import ( - "fmt" - "sync" - "time" - - "github.com/lovoo/goka/kafka" "github.com/lovoo/goka/storage" ) -const ( - delayProxyInterval = 1 * time.Second -) - -type proxy struct { - partition int32 - consumer kafka.Consumer -} - -func (p *proxy) Add(topic string, offset int64) error { - if err := p.consumer.AddPartition(topic, p.partition, offset); err != nil { - return fmt.Errorf("error adding %s/%d: %v", topic, p.partition, err) - } - return nil -} - -func (p *proxy) Remove(topic string) error { - if err := p.consumer.RemovePartition(topic, p.partition); err != nil { - return fmt.Errorf("error removing %s/%d: %v", topic, p.partition, err) - } - return nil -} - -func (p *proxy) AddGroup() { - p.consumer.AddGroupPartition(p.partition) -} - -func (p *proxy) Stop() {} - -type delayProxy struct { - proxy - stop bool - m sync.Mutex - wait []func() bool -} - -func (p *delayProxy) waitersDone() bool { - for _, r := range p.wait { - if !r() { - return false - } - } - return true -} - -func (p *delayProxy) AddGroup() { - if len(p.wait) == 0 { - p.consumer.AddGroupPartition(p.partition) - return - } - - go func() { - ticker := time.NewTicker(delayProxyInterval) - defer ticker.Stop() - for range ticker.C { - p.m.Lock() - if p.stop { - p.m.Unlock() - return - } - if p.waitersDone() { - p.consumer.AddGroupPartition(p.partition) - p.m.Unlock() - return - } - p.m.Unlock() - } - }() -} - -func (p *delayProxy) Stop() { - p.m.Lock() - p.stop = true - p.m.Unlock() -} - type storageProxy struct { storage.Storage partition int32 diff --git a/tester/signal.go b/signal.go similarity index 52% rename from tester/signal.go rename to signal.go index 040c5a09..24ae0aa8 100644 --- a/tester/signal.go +++ b/signal.go @@ -1,4 +1,4 @@ -package tester +package goka import ( "fmt" @@ -8,19 +8,24 @@ import ( // State types a state of the Signal type State int +type waiter struct { + done chan struct{} + state State + minState bool +} + // Signal allows synchronization on a state, waiting for that state and checking // the current state type Signal struct { - sync.Mutex + m sync.Mutex state State - waitChans map[State][]chan struct{} + waiters []*waiter allowedStates map[State]bool } // NewSignal creates a new Signal based on the states func NewSignal(states ...State) *Signal { s := &Signal{ - waitChans: make(map[State][]chan struct{}), allowedStates: make(map[State]bool), } for _, state := range states { @@ -33,18 +38,23 @@ func NewSignal(states ...State) *Signal { // SetState changes the state of the signal // and notifies all goroutines waiting for the new state func (s *Signal) SetState(state State) *Signal { - s.Lock() - defer s.Unlock() + s.m.Lock() + defer s.m.Unlock() if !s.allowedStates[state] { panic(fmt.Errorf("trying to set illegal state %v", state)) } // set the state and notify all channels waiting for it. s.state = state - for _, waitChan := range s.waitChans[state] { - close(waitChan) + var newWaiters []*waiter + for _, w := range s.waiters { + if w.state == state || (w.minState && state >= w.state) { + close(w.done) + continue + } + newWaiters = append(newWaiters, w) } - delete(s.waitChans, state) + s.waiters = newWaiters return s } @@ -59,18 +69,42 @@ func (s *Signal) State() State { return s.state } +func (s *Signal) WaitForStateMin(state State) chan struct{} { + s.m.Lock() + defer s.m.Unlock() + + w := &waiter{ + done: make(chan struct{}), + state: state, + minState: true, + } + + return s.waitForWaiter(state, w) +} + // WaitForState returns a channel that closes when the signal reaches passed // state. func (s *Signal) WaitForState(state State) chan struct{} { - s.Lock() - defer s.Unlock() - cb := make(chan struct{}) + s.m.Lock() + defer s.m.Unlock() + + w := &waiter{ + done: make(chan struct{}), + state: state, + } + + return s.waitForWaiter(state, w) +} + +func (s *Signal) waitForWaiter(state State, w *waiter) chan struct{} { - if s.IsState(state) { - close(cb) + // if the signal is currently in that state (or in a higher state if minState is set) + // then close the waiter immediately + if curState := s.State(); state == curState || (w.minState && curState >= state) { + close(w.done) } else { - s.waitChans[state] = append(s.waitChans[state], cb) + s.waiters = append(s.waiters, w) } - return cb + return w.done } diff --git a/signal_test.go b/signal_test.go new file mode 100644 index 00000000..762e442a --- /dev/null +++ b/signal_test.go @@ -0,0 +1,71 @@ +package goka + +import ( + "testing" + + "github.com/lovoo/goka/internal/test" +) + +func TestSignal_SetState(t *testing.T) { + + sig := NewSignal(0, 1, 2) + test.AssertTrue(t, sig.IsState(0)) + test.AssertFalse(t, sig.IsState(1)) + + sig.SetState(1) + test.AssertTrue(t, sig.IsState(1)) + test.AssertFalse(t, sig.IsState(0)) + + defer func() { + err := recover() + if err == nil { + t.Fatalf("Expected panic, which didn't occur") + } + }() + + // set some invalid state, this will panic + sig.SetState(3) +} + +func TestSignal_Wait(t *testing.T) { + sig := NewSignal(0, 1, 2) + + <-sig.WaitForState(0) + // should continue right now since + + var ( + done = make(chan struct{}) + hasState bool + ) + go func() { + defer close(done) + <-sig.WaitForState(1) + hasState = true + }() + + test.AssertFalse(t, hasState) + sig.SetState(1) + // wait for the goroutine to catchup with the state + <-done + test.AssertTrue(t, hasState) +} + +func TestSignalWaitMin(t *testing.T) { + sig := NewSignal(0, 1, 2) + + var ( + done = make(chan struct{}) + hasState bool + ) + go func() { + defer close(done) + <-sig.WaitForStateMin(1) + hasState = true + }() + + test.AssertFalse(t, hasState) + sig.SetState(2) + // wait for the goroutine to catchup with the state + <-done + test.AssertTrue(t, hasState) +} diff --git a/stats.go b/stats.go index 8308bfd6..9d45c693 100644 --- a/stats.go +++ b/stats.go @@ -1,29 +1,22 @@ package goka -import "time" - -// InputStats represents the number of messages and the number of bytes consumed -// from a stream or table topic since the process started. -type InputStats struct { - Count uint - Bytes int - Delay time.Duration -} - -// OutputStats represents the number of messages and the number of bytes emitted -// into a stream or table since the process started. -type OutputStats struct { - Count uint - Bytes int -} +import ( + "log" + "time" +) // PartitionStatus is the status of the partition of a table (group table or joined table). type PartitionStatus int const ( + // PartitionStopped indicates the partition stopped and should not be used anymore. + PartitionStopped PartitionStatus = iota + // PartitionInitializing indicates that the underlying storage is initializing (e.g. opening leveldb files), + // and has not actually started working yet. + PartitionInitializing // PartitionRecovering indicates the partition is recovering and the storage // is writing updates in bulk-mode (if the storage implementation supports it). - PartitionRecovering PartitionStatus = iota + PartitionRecovering // PartitionPreparing indicates the end of the bulk-mode. Depending on the storage // implementation, the Preparing phase may take long because the storage compacts its logs. PartitionPreparing @@ -32,82 +25,193 @@ const ( PartitionRunning ) -// PartitionStats represents metrics and measurements of a partition. -type PartitionStats struct { +const ( + statsHwmUpdateInterval = 5 * time.Second + fetchStatsTimeout = 10 * time.Second +) + +// InputStats represents the number of messages and the number of bytes consumed +// from a stream or table topic since the process started. +type InputStats struct { + Count uint + Bytes int + OffsetLag int64 + LastOffset int64 + Delay time.Duration +} + +// OutputStats represents the number of messages and the number of bytes emitted +// into a stream or table since the process started. +type OutputStats struct { + Count uint + Bytes int +} + +// PartitionProcStats represents metrics and measurements of a partition processor +type PartitionProcStats struct { Now time.Time - Table struct { - Status PartitionStatus - Stalled bool + TableStats *TableStats + + Joined map[string]*TableStats + + Input map[string]*InputStats + Output map[string]*OutputStats +} + +type RecoveryStats struct { + StartTime time.Time + RecoveryTime time.Time + + Offset int64 // last offset processed or recovered + Hwm int64 // next offset to be written +} + +// TableStats represents stats for a table partition +type TableStats struct { + Stalled bool - Offset int64 // last offset processed or recovered - Hwm int64 // next offset to be written + Status PartitionStatus - StartTime time.Time - RecoveryTime time.Time + Recovery *RecoveryStats + + Input *InputStats + Writes *OutputStats +} + +func newInputStats() *InputStats { + return &InputStats{} +} + +func newOutputStats() *OutputStats { + return &OutputStats{} +} + +func (is *InputStats) clone() *InputStats { + return &(*is) +} + +func (os *OutputStats) clone() *OutputStats { + return &(*os) +} + +type InputStatsMap map[string]*InputStats +type OutputStatsMap map[string]*OutputStats + +func (isp InputStatsMap) clone() map[string]*InputStats { + var c = map[string]*InputStats{} + if isp == nil { + return c } - Input map[string]InputStats - Output map[string]OutputStats + for k, v := range isp { + c[k] = v.clone() + } + return c +} + +func (osp OutputStatsMap) clone() map[string]*OutputStats { + var c = map[string]*OutputStats{} + if osp == nil { + return c + } + for k, v := range osp { + c[k] = v.clone() + } + return c +} + +func newRecoveryStats() *RecoveryStats { + return new(RecoveryStats) +} + +func (rs *RecoveryStats) clone() *RecoveryStats { + var rsCopy = *rs + return &rsCopy } -func newPartitionStats() *PartitionStats { - return &PartitionStats{ - Now: time.Now(), - Input: make(map[string]InputStats), - Output: make(map[string]OutputStats), +func newPartitionProcStats(inputs []string, outputs []string) *PartitionProcStats { + procStats := &PartitionProcStats{ + Now: time.Now(), + + Input: make(map[string]*InputStats), + Output: make(map[string]*OutputStats), } + + for _, input := range inputs { + procStats.Input[input] = newInputStats() + } + + for _, output := range outputs { + procStats.Output[output] = newOutputStats() + } + + return procStats } -func (s *PartitionStats) init(o *PartitionStats, offset, hwm int64) *PartitionStats { - s.Table.Status = o.Table.Status - s.Table.Stalled = o.Table.Stalled - s.Table.StartTime = o.Table.StartTime - s.Table.RecoveryTime = o.Table.RecoveryTime - s.Table.Offset = offset - s.Table.Hwm = hwm - s.Now = time.Now() - for k, v := range o.Input { - s.Input[k] = v +func newTableStats() *TableStats { + return &TableStats{ + Input: newInputStats(), + Writes: newOutputStats(), + Recovery: newRecoveryStats(), } - for k, v := range o.Output { - s.Output[k] = v +} + +func (ts *TableStats) reset() { + ts.Input = newInputStats() + ts.Writes = newOutputStats() +} + +func (ts *TableStats) clone() *TableStats { + return &TableStats{ + Input: ts.Input.clone(), + Writes: ts.Writes.clone(), + Recovery: ts.Recovery.clone(), + Stalled: ts.Stalled, } - return s } -func (s *PartitionStats) reset() { - s.Input = make(map[string]InputStats) - s.Output = make(map[string]OutputStats) +func (s *PartitionProcStats) clone() *PartitionProcStats { + pps := newPartitionProcStats(nil, nil) + pps.Now = time.Now() + pps.Joined = make(map[string]*TableStats) + pps.Input = InputStatsMap(s.Input).clone() + pps.Output = OutputStatsMap(s.Output).clone() + + return pps +} + +func (s *PartitionProcStats) trackOutput(topic string, valueLen int) { + outStats := s.Output[topic] + if outStats == nil { + log.Printf("no out stats for topic %s", topic) + } + outStats.Count++ + outStats.Bytes += valueLen } // ViewStats represents the metrics of all partitions of a view. type ViewStats struct { - Partitions map[int32]*PartitionStats + Partitions map[int32]*TableStats } func newViewStats() *ViewStats { return &ViewStats{ - Partitions: make(map[int32]*PartitionStats), + Partitions: make(map[int32]*TableStats), } } // ProcessorStats represents the metrics of all partitions of the processor, // including its group, joined tables and lookup tables. type ProcessorStats struct { - Group map[int32]*PartitionStats - Joined map[int32]map[string]*PartitionStats + Group map[int32]*PartitionProcStats Lookup map[string]*ViewStats } func newProcessorStats(partitions int) *ProcessorStats { stats := &ProcessorStats{ - Group: make(map[int32]*PartitionStats), - Joined: make(map[int32]map[string]*PartitionStats), + Group: make(map[int32]*PartitionProcStats), Lookup: make(map[string]*ViewStats), } - for i := int32(0); i < int32(partitions); i++ { - stats.Joined[i] = make(map[string]*PartitionStats) - } return stats } diff --git a/storage/iterator_test.go b/storage/iterator_test.go index b28c7f24..dc5180b0 100644 --- a/storage/iterator_test.go +++ b/storage/iterator_test.go @@ -4,19 +4,18 @@ import ( "io/ioutil" "testing" - "github.com/facebookgo/ensure" "github.com/syndtr/goleveldb/leveldb" ) func TestIterator(t *testing.T) { tmpdir, err := ioutil.TempDir("", "goka_storage_TestIterator") - ensure.Nil(t, err) + assertNil(t, err) db, err := leveldb.OpenFile(tmpdir, nil) - ensure.Nil(t, err) + assertNil(t, err) st, err := New(db) - ensure.Nil(t, err) + assertNil(t, err) kv := map[string]string{ "key-1": "val-1", @@ -25,20 +24,20 @@ func TestIterator(t *testing.T) { } for k, v := range kv { - ensure.Nil(t, st.Set(k, []byte(v))) + assertNil(t, st.Set(k, []byte(v))) } - ensure.Nil(t, st.SetOffset(777)) + assertNil(t, st.SetOffset(777)) iter, err := st.Iterator() - ensure.Nil(t, err) + assertNil(t, err) defer iter.Release() count := 0 // accessing iterator before Next should only return nils val, err := iter.Value() - ensure.True(t, val == nil) - ensure.Nil(t, err) + assertTrue(t, val == nil) + assertNil(t, err) for iter.Next() { count++ @@ -49,8 +48,8 @@ func TestIterator(t *testing.T) { } val, err := iter.Value() - ensure.Nil(t, err) - ensure.DeepEqual(t, expected, string(val)) + assertNil(t, err) + assertEqual(t, expected, string(val)) } - ensure.DeepEqual(t, count, len(kv)) + assertEqual(t, count, len(kv)) } diff --git a/storage/leveldb_test.go b/storage/leveldb_test.go index cec6f715..49d6fadd 100644 --- a/storage/leveldb_test.go +++ b/storage/leveldb_test.go @@ -5,7 +5,7 @@ import ( "io/ioutil" "testing" - "github.com/facebookgo/ensure" + "github.com/lovoo/goka/internal/test" "github.com/syndtr/goleveldb/leveldb" ) @@ -21,14 +21,14 @@ func init() { func BenchmarkStateStorage_unbatched(b *testing.B) { tmpdir, err := ioutil.TempDir("", "benchmark_statestorage_unbatched") - ensure.Nil(b, err) + test.AssertNil(b, err) db, err := leveldb.OpenFile(tmpdir, nil) - ensure.Nil(b, err) + test.AssertNil(b, err) storage, err := New(db) - ensure.Nil(b, err) - ensure.Nil(b, storage.MarkRecovered()) + test.AssertNil(b, err) + test.AssertNil(b, storage.MarkRecovered()) b.ResetTimer() for i := 0; i < b.N*numWrites; i++ { storage.Set(keys[i%len(keys)], []byte(fmt.Sprintf("value-%d", i))) @@ -38,17 +38,17 @@ func BenchmarkStateStorage_unbatched(b *testing.B) { func BenchmarkStateStorage_transactioned(b *testing.B) { tmpdir, err := ioutil.TempDir("", "benchmark_statestorage_transactioned") - ensure.Nil(b, err) + test.AssertNil(b, err) db, err := leveldb.OpenFile(tmpdir, nil) - ensure.Nil(b, err) + test.AssertNil(b, err) storage, err := New(db) - ensure.Nil(b, err) + test.AssertNil(b, err) b.ResetTimer() for i := 0; i < b.N*numWrites; i++ { storage.Set(keys[i%len(keys)], []byte(fmt.Sprintf("value-%d", i))) } - ensure.Nil(b, storage.MarkRecovered()) + test.AssertNil(b, storage.MarkRecovered()) storage.Close() } diff --git a/storage/storage.go b/storage/storage.go index 107607c2..7246ab53 100644 --- a/storage/storage.go +++ b/storage/storage.go @@ -64,8 +64,6 @@ type storage struct { db *leveldb.DB // tx is the transaction used for recovery tx *leveldb.Transaction - - currentOffset int64 } // New creates a new Storage backed by LevelDB. @@ -161,10 +159,6 @@ func (s *storage) Set(key string, value []byte) error { } func (s *storage) SetOffset(offset int64) error { - if offset > s.currentOffset { - s.currentOffset = offset - } - return s.Set(offsetKey, []byte(strconv.FormatInt(offset, 10))) } diff --git a/storage/storage_test.go b/storage/storage_test.go index a3e033ef..1ef42fe8 100644 --- a/storage/storage_test.go +++ b/storage/storage_test.go @@ -1,36 +1,34 @@ package storage import ( - "fmt" "io/ioutil" "sort" "testing" + "github.com/lovoo/goka/internal/test" "github.com/syndtr/goleveldb/leveldb" - - "github.com/facebookgo/ensure" ) func TestMemStorageDelete(t *testing.T) { storage := NewMemory() has, err := storage.Has("key-1") - ensure.Nil(t, err) - ensure.False(t, has) + test.AssertNil(t, err) + test.AssertFalse(t, has) err = storage.Set("key-1", []byte("content-1")) - ensure.Nil(t, err) + test.AssertNil(t, err) has, err = storage.Has("key-1") - ensure.Nil(t, err) - ensure.True(t, has) + test.AssertNil(t, err) + test.AssertTrue(t, has) err = storage.Delete("key-1") - ensure.Nil(t, err) + test.AssertNil(t, err) has, err = storage.Has("key-1") - ensure.Nil(t, err) - ensure.False(t, has) + test.AssertNil(t, err) + test.AssertFalse(t, has) } func TestMemIter(t *testing.T) { @@ -51,40 +49,40 @@ func TestMemIter(t *testing.T) { // released iterator should be immediately exhausted iter, err := storage.Iterator() - ensure.Nil(t, err) + test.AssertNil(t, err) iter.Release() - ensure.False(t, iter.Next(), "released iterator had a next") + test.AssertFalse(t, iter.Next()) iter, err = storage.Iterator() - ensure.Nil(t, err) + test.AssertNil(t, err) for iter.Next() { raw, err := iter.Value() - ensure.Nil(t, err) + test.AssertNil(t, err) key := string(iter.Key()) val := string(raw) v, ok := kv[key] - ensure.True(t, ok, fmt.Sprintf("unexpected key returned from iterator: %s", key)) - ensure.DeepEqual(t, val, v, fmt.Sprintf("iterator returned wrong value: %s, expected: %s", val, v)) + test.AssertTrue(t, ok) + test.AssertEqual(t, val, v) found[key] = val } key := iter.Key() val, err := iter.Value() - ensure.Nil(t, err, "exhausted iterator should not return error") - ensure.True(t, key == nil, fmt.Sprintf("exhausted iterator should return nil key, returned: %s", key)) - ensure.True(t, val == nil, "exhausted iterator should return nil value, returned %s", val) + test.AssertNil(t, err) + test.AssertTrue(t, key == nil) + test.AssertTrue(t, val == nil) - ensure.DeepEqual(t, found, kv, "found doesn't match kv, iterator probably didn't return all values") + test.AssertEqual(t, found, kv) k := []byte("key-1") iter, err = storage.IteratorWithRange(k, nil) sort.Strings(iter.(*memiter).keys) // make iteration order deterministic - ensure.True(t, iter.Next(), "next should return true after a IteratorWithRange") - ensure.DeepEqual(t, iter.Key(), k, "the first matching key in IteratorWithRange is not corresponding to the value") + test.AssertTrue(t, iter.Next()) + test.AssertEqual(t, iter.Key(), k) } @@ -97,30 +95,30 @@ func TestGetHas(t *testing.T) { ) hasKey, err = storage.Has("test-key") - ensure.Nil(t, err) - ensure.False(t, hasKey) + test.AssertNil(t, err) + test.AssertFalse(t, hasKey) value, err := storage.Get("test-key") - ensure.True(t, value == nil) - ensure.Nil(t, err) + test.AssertTrue(t, value == nil) + test.AssertNil(t, err) err = storage.Set("test-key", []byte("test")) - ensure.Nil(t, err) + test.AssertNil(t, err) hasKey, err = storage.Has("test-key") - ensure.Nil(t, err) - ensure.True(t, hasKey) + test.AssertNil(t, err) + test.AssertTrue(t, hasKey) value, err = storage.Get("test-key") - ensure.Nil(t, err) - ensure.DeepEqual(t, value, []byte("test")) + test.AssertNil(t, err) + test.AssertEqual(t, value, []byte("test")) hasKey, err = storage.Has("nil-value") - ensure.Nil(t, err) - ensure.False(t, hasKey) + test.AssertNil(t, err) + test.AssertFalse(t, hasKey) err = storage.Set("nil-value", nil) - ensure.NotNil(t, err) + test.AssertNotNil(t, err) } func TestSetGet(t *testing.T) { @@ -130,55 +128,55 @@ func TestSetGet(t *testing.T) { ) tmpdir, err := ioutil.TempDir("", "goka_storage_TestSetGet") - ensure.Nil(t, err) + test.AssertNil(t, err) db, err := leveldb.OpenFile(tmpdir, nil) - ensure.Nil(t, err) + test.AssertNil(t, err) storage, err := New(db) - ensure.Nil(t, err) + test.AssertNil(t, err) hasKey, err = storage.Has("example1") - ensure.Nil(t, err) - ensure.False(t, hasKey) + test.AssertNil(t, err) + test.AssertFalse(t, hasKey) value, err := storage.Get("example1") - ensure.True(t, value == nil) - ensure.Nil(t, err) + test.AssertTrue(t, value == nil) + test.AssertNil(t, err) err = storage.Set("example1", []byte("example-message")) - ensure.Nil(t, err) + test.AssertNil(t, err) hasKey, err = storage.Has("example1") - ensure.Nil(t, err) - ensure.True(t, hasKey) + test.AssertNil(t, err) + test.AssertTrue(t, hasKey) value, err = storage.Get("example1") - ensure.Nil(t, err) + test.AssertNil(t, err) - ensure.Nil(t, storage.Delete("example1")) + test.AssertNil(t, storage.Delete("example1")) hasKey, err = storage.Has("example1") - ensure.Nil(t, err) - ensure.False(t, hasKey) + test.AssertNil(t, err) + test.AssertFalse(t, hasKey) // test iteration - ensure.Nil(t, storage.Set("key1", []byte("value1"))) - ensure.Nil(t, storage.Set("key2", []byte("value2"))) + test.AssertNil(t, storage.Set("key1", []byte("value1"))) + test.AssertNil(t, storage.Set("key2", []byte("value2"))) iter, err := storage.Iterator() - ensure.Nil(t, err) + test.AssertNil(t, err) defer iter.Release() messages := make(map[string]string) for iter.Next() { key := string(iter.Key()) val, err := iter.Value() - ensure.Nil(t, err) + test.AssertNil(t, err) messages[key] = string(val) } - ensure.True(t, len(messages) == 2, fmt.Sprintf("expected 2 messages, got: %d", len(messages))) - ensure.DeepEqual(t, messages["key1"], "value1") - ensure.DeepEqual(t, messages["key2"], "value2") + test.AssertTrue(t, len(messages) == 2) + test.AssertEqual(t, messages["key1"], "value1") + test.AssertEqual(t, messages["key2"], "value2") recoveredValue := string(value) - ensure.DeepEqual(t, recoveredValue, "example-message") + test.AssertEqual(t, recoveredValue, "example-message") } diff --git a/storage/test.go b/storage/test.go new file mode 100644 index 00000000..5d97c892 --- /dev/null +++ b/storage/test.go @@ -0,0 +1,94 @@ +package storage + +import ( + "reflect" + "regexp" + "runtime/debug" + "strings" + "testing" +) + +func assertNil(t *testing.T, actual interface{}) { + value := reflect.ValueOf(actual) + if value.IsValid() { + if !value.IsNil() { + t.Fatalf("Expected value to be nil, but was not nil in %s", string(debug.Stack())) + } + } +} + +func assertNotNil(t *testing.T, actual interface{}) { + value := reflect.ValueOf(actual) + if !value.IsValid() || value.IsNil() { + t.Fatalf("Expected value to be not nil, but was nil in %s", string(debug.Stack())) + } +} + +func assertTrue(t *testing.T, value bool) { + if !value { + t.Fatalf("Expected value to be true, but was false in %s", string(debug.Stack())) + } +} +func assertFalse(t *testing.T, value bool) { + if value { + t.Fatalf("Expected value to be false, but was true in %s", string(debug.Stack())) + } +} + +func assertEqual(t *testing.T, actual, expected interface{}) { + if !reflect.DeepEqual(actual, expected) { + t.Fatalf("Expected values were not equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) + } +} + +func panicAssertEqual(t *testing.T, expected interface{}) { + if expected == nil { + panic("can't pass nil to panicAssertEqual") + } + if actual := recover(); actual != nil { + if !reflect.DeepEqual(actual, expected) { + t.Fatalf("Expected values were equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) + } + } else { + t.Errorf("panic expected") + t.FailNow() + } + +} + +func assertNotEqual(t *testing.T, actual, expected interface{}) { + if reflect.DeepEqual(actual, expected) { + t.Fatalf("Expected values were equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) + } +} + +func assertFuncEqual(t *testing.T, actual, expected interface{}) { + if !(reflect.ValueOf(actual).Pointer() == reflect.ValueOf(expected).Pointer()) { + t.Fatalf("Expected functions were equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) + } +} + +func assertError(t *testing.T, actual error, reg *regexp.Regexp) { + if actual == nil || reg == nil { + t.Fatalf("Error or regexp is nil.\nactual=%#v\nregexp=%#v in %s", actual, reg, string(debug.Stack())) + } + if !reg.MatchString(actual.(error).Error()) { + t.Fatalf("Expected but got.\nactual=%#v\nregexp=%#v in %s", actual, reg, string(debug.Stack())) + } +} + +func assertStringContains(t *testing.T, actual string, contains string) { + if !strings.Contains(actual, contains) { + t.Fatalf("Expected string to contain substring \nactual=%#v\nexpected=%#v in %s", actual, contains, string(debug.Stack())) + } +} + +func panicAssertStringContains(t *testing.T, s string) { + if r := recover(); r != nil { + err := r.(error) + assertStringContains(t, err.Error(), s) + } else { + t.Errorf("panic expected") + t.FailNow() + } +} diff --git a/tester/client.go b/tester/client.go new file mode 100644 index 00000000..6467f307 --- /dev/null +++ b/tester/client.go @@ -0,0 +1,30 @@ +package tester + +type client struct { + clientID string + consumerGroup *consumerGroup + consumer *consumerMock +} + +func (c *client) waitStartup() { + if c.consumerGroup != nil { + c.consumerGroup.waitRunning() + } + + c.consumer.waitRequiredConsumersStartup() +} + +func (c *client) requireConsumer(topic string) { + c.consumer.requirePartConsumer(topic) +} + +func (c *client) catchup() int { + var catchup int + if c.consumerGroup != nil { + catchup += c.consumerGroup.catchupAndWait() + } + + catchup += c.consumer.catchup() + + return catchup +} diff --git a/tester/consumer.go b/tester/consumer.go new file mode 100644 index 00000000..ef11654a --- /dev/null +++ b/tester/consumer.go @@ -0,0 +1,148 @@ +package tester + +import ( + "fmt" + "sync" + "time" + + "github.com/Shopify/sarama" +) + +type consumerMock struct { + sync.RWMutex + tester *Tester + requiredTopics map[string]bool + partConsumers map[string]*partConsumerMock +} + +func newConsumerMock(tt *Tester) *consumerMock { + return &consumerMock{ + tester: tt, + requiredTopics: make(map[string]bool), + partConsumers: make(map[string]*partConsumerMock), + } +} + +func (cm *consumerMock) catchup() int { + cm.RLock() + defer cm.RUnlock() + var catchup int + for _, pc := range cm.partConsumers { + catchup += pc.catchup() + } + return catchup +} + +func (cm *consumerMock) Topics() ([]string, error) { + return nil, fmt.Errorf("Not implemented") +} + +func (cm *consumerMock) Partitions(topic string) ([]int32, error) { + return nil, fmt.Errorf("not implemented") +} + +func (cm *consumerMock) ConsumePartition(topic string, partition int32, offset int64) (sarama.PartitionConsumer, error) { + cm.Lock() + defer cm.Unlock() + if _, exists := cm.partConsumers[topic]; exists { + return nil, fmt.Errorf("Got duplicate consume partition for topic %s", topic) + } + cons := &partConsumerMock{ + hwm: offset, + queue: cm.tester.getOrCreateQueue(topic), + messages: make(chan *sarama.ConsumerMessage), + errors: make(chan *sarama.ConsumerError), + closer: func() error { + cm.Lock() + defer cm.Unlock() + if _, exists := cm.partConsumers[topic]; !exists { + return fmt.Errorf("partition consumer seems already closed") + } + delete(cm.partConsumers, topic) + return nil + }, + } + + cm.partConsumers[topic] = cons + + return cons, nil +} +func (cm *consumerMock) HighWaterMarks() map[string]map[int32]int64 { + return nil +} +func (cm *consumerMock) Close() error { + return nil +} + +func (cm *consumerMock) waitRequiredConsumersStartup() { + doCheck := func() bool { + cm.RLock() + defer cm.RUnlock() + + for topic := range cm.requiredTopics { + _, ok := cm.partConsumers[topic] + if !ok { + return false + } + } + return true + } + for !doCheck() { + time.Sleep(50 * time.Millisecond) + } +} + +func (cm *consumerMock) requirePartConsumer(topic string) { + cm.requiredTopics[topic] = true +} + +type partConsumerMock struct { + hwm int64 + closer func() error + messages chan *sarama.ConsumerMessage + errors chan *sarama.ConsumerError + queue *queue +} + +func (pcm *partConsumerMock) catchup() int { + var numCatchup int + for _, msg := range pcm.queue.messagesFromOffset(pcm.hwm) { + pcm.messages <- &sarama.ConsumerMessage{ + Key: []byte(msg.key), + Value: msg.value, + Topic: pcm.queue.topic, + Partition: 0, + Offset: msg.offset, + } + + // we'll send a nil that is being ignored by the partition_table to make sure the other message + // really went through the channel + pcm.messages <- nil + numCatchup++ + pcm.hwm = msg.offset + 1 + } + + return numCatchup +} + +func (pcm *partConsumerMock) Close() error { + close(pcm.messages) + close(pcm.errors) + return pcm.closer() +} + +func (pcm *partConsumerMock) AsyncClose() { + go pcm.Close() +} + +func (pcm *partConsumerMock) Messages() <-chan *sarama.ConsumerMessage { + return pcm.messages +} + +func (pcm *partConsumerMock) Errors() <-chan *sarama.ConsumerError { + return pcm.errors +} + +func (pcm *partConsumerMock) HighWaterMarkOffset() int64 { + return pcm.queue.Hwm() +} diff --git a/tester/consumergroup.go b/tester/consumergroup.go new file mode 100644 index 00000000..09715168 --- /dev/null +++ b/tester/consumergroup.go @@ -0,0 +1,351 @@ +package tester + +import ( + "context" + "fmt" + "sync" + "sync/atomic" + + "github.com/Shopify/sarama" + "github.com/lovoo/goka" + "github.com/lovoo/goka/multierr" +) + +// consumerGroup mocks the consumergroup +type consumerGroup struct { + errs chan error + + // use the same offset counter for all topics + offset int64 + currentGeneration int32 + + state *goka.Signal + + currentSession *cgSession + + tt *Tester +} + +const ( + cgStateStopped goka.State = iota + cgStateRebalancing + cgStateSetup + cgStateConsuming + cgStateCleaning +) + +func newConsumerGroup(t T, tt *Tester) *consumerGroup { + return &consumerGroup{ + errs: make(chan error, 1), + state: goka.NewSignal(cgStateStopped, cgStateRebalancing, cgStateSetup, cgStateConsuming, cgStateCleaning).SetState(cgStateStopped), + tt: tt, + } +} + +func (cg *consumerGroup) catchupAndWait() int { + if cg.currentSession == nil { + panic("There is currently no session. Cannot catchup, but we shouldn't be at this point") + } + return cg.currentSession.catchupAndWait() +} + +// Consume starts consuming from the consumergroup +func (cg *consumerGroup) Consume(ctx context.Context, topics []string, handler sarama.ConsumerGroupHandler) error { + if !cg.state.IsState(cgStateStopped) { + return fmt.Errorf("Tried to double-consume this consumer-group, which is not supported by the mock") + } + logger.Printf("consuming consumergroup with topics %v", topics) + defer cg.state.SetState(cgStateStopped) + if len(topics) == 0 { + return fmt.Errorf("no topics specified") + } + + for { + cg.state.SetState(cgStateRebalancing) + cg.currentGeneration++ + session := newCgSession(ctx, cg.currentGeneration, cg, topics) + + cg.currentSession = session + + cg.state.SetState(cgStateSetup) + err := handler.Setup(session) + if err != nil { + return fmt.Errorf("Error setting up: %v", err) + } + errg, _ := multierr.NewErrGroup(ctx) + for _, claim := range session.claims { + claim := claim + errg.Go(func() error { + <-ctx.Done() + claim.close() + return nil + }) + errg.Go(func() error { + return handler.ConsumeClaim(session, claim) + }) + } + cg.state.SetState(cgStateConsuming) + + errs := new(multierr.Errors) + + // wait for runner errors and collect error + errs.Collect(errg.Wait().NilOrError()) + cg.state.SetState(cgStateCleaning) + + // cleanup and collect errors + errs.Collect(handler.Cleanup(session)) + + // remove current sessions + cg.currentSession = nil + + err = errs.NilOrError() + if err != nil { + return fmt.Errorf("Error running or cleaning: %v", err) + } + + select { + // if the session was terminated because of a cancelled context, + // stop the loop + case <-ctx.Done(): + return nil + // otherwise just continue with the next generation + default: + } + } +} + +// SendError sends an error the consumergroup +func (cg *consumerGroup) SendError(err error) { + cg.errs <- err +} + +// Errors returns the errors channel +func (cg *consumerGroup) Errors() <-chan error { + return cg.errs +} + +func (cg *consumerGroup) waitRunning() { + <-cg.state.WaitForState(cgStateConsuming) +} + +func (cg *consumerGroup) nextOffset() int64 { + return atomic.AddInt64(&cg.offset, 1) +} + +// Close closes the consumergroup +func (cg *consumerGroup) Close() error { + // close old errs chan and create new one + close(cg.errs) + cg.errs = make(chan error) + + cg.offset = 0 + cg.currentGeneration = 0 + return nil +} + +type cgSession struct { + ctx context.Context + generation int32 + claims map[string]*cgClaim + queues map[string]*queueSession + + mCatchup sync.Mutex + + waitingMessages map[string]bool + mMessages sync.Mutex + wgMessages sync.WaitGroup + consumerGroup *consumerGroup +} + +func newCgSession(ctx context.Context, generation int32, cg *consumerGroup, topics []string) *cgSession { + + cgs := &cgSession{ + ctx: ctx, + generation: generation, + consumerGroup: cg, + waitingMessages: make(map[string]bool), + queues: make(map[string]*queueSession), + claims: make(map[string]*cgClaim), + } + + for _, topic := range topics { + cgs.queues[topic] = &queueSession{ + queue: cg.tt.getOrCreateQueue(topic), + } + cgs.claims[topic] = newCgClaim(topic, 0) + } + + return cgs +} + +// queue session pairs a queue and an offset for a consumerGroupClaim. +// This allows to consume messages continuing from the last marked offset. +type queueSession struct { + sync.Mutex + queue *queue + hwm int64 +} + +func (qs *queueSession) setHwmIfNewer(hwm int64) { + qs.Lock() + defer qs.Unlock() + if qs.hwm < hwm { + qs.hwm = hwm + } +} + +func (qs *queueSession) getHwm() int64 { + qs.Lock() + defer qs.Unlock() + return qs.hwm +} + +// Claims returns the number of partitions assigned in the group session for each topic +func (cgs *cgSession) Claims() map[string][]int32 { + claims := make(map[string][]int32) + for topic := range cgs.claims { + claims[topic] = []int32{0} + } + return claims +} + +// MemberID returns the member ID +// TOOD: clarify what that actually means and whether we need to mock taht somehow +func (cgs *cgSession) MemberID() string { + panic("MemberID not provided by mock") +} + +// GenerationID returns the generation ID of the group consumer +func (cgs *cgSession) GenerationID() int32 { + return cgs.generation +} + +// MarkOffset marks the passed offset consumed in topic/partition +func (cgs *cgSession) MarkOffset(topic string, partition int32, offset int64, metadata string) { + cgs.mMessages.Lock() + defer cgs.mMessages.Unlock() + + msgKey := cgs.makeMsgKey(topic, offset) + if !cgs.waitingMessages[msgKey] { + logger.Printf("Message topic/partition/offset %s/%d/%d was already marked as consumed. We should only mark the message once", topic, partition, offset) + } else { + cgs.wgMessages.Done() + delete(cgs.waitingMessages, msgKey) + } + + cgs.queues[topic].setHwmIfNewer(offset + 1) +} + +// ResetOffset resets the offset to be consumed from +func (cgs *cgSession) ResetOffset(topic string, partition int32, offset int64, metadata string) { + panic("reset offset is not implemented by the mock") +} + +// MarkMessage marks the passed message as consumed +func (cgs *cgSession) MarkMessage(msg *sarama.ConsumerMessage, metadata string) { + cgs.MarkOffset(msg.Topic, msg.Partition, msg.Offset, metadata) +} + +// Context returns the consumer group's context +func (cgs *cgSession) Context() context.Context { + return cgs.ctx +} + +func (cgs *cgSession) catchupAndWait() int { + cgs.mCatchup.Lock() + defer cgs.mCatchup.Unlock() + + var msgPushed int + for _, queue := range cgs.queues { + + cgs.mMessages.Lock() + queueHwm := queue.getHwm() + cgs.mMessages.Unlock() + + for _, msg := range queue.queue.messagesFromOffset(queueHwm) { + cgs.pushMessageToClaim(cgs.claims[queue.queue.topic], msg) + msgPushed++ + } + } + + done := make(chan struct{}) + go func() { + defer close(done) + cgs.wgMessages.Wait() + }() + + select { + case <-cgs.ctx.Done(): + return 0 + case <-done: + return msgPushed + } +} + +func (cgs *cgSession) makeMsgKey(topic string, offset int64) string { + return fmt.Sprintf("%s-%d", topic, offset) +} + +func (cgs *cgSession) pushMessageToClaim(claim *cgClaim, msg *message) { + cgs.mMessages.Lock() + defer cgs.mMessages.Unlock() + + msgKey := cgs.makeMsgKey(claim.Topic(), msg.offset) + + if cgs.waitingMessages[msgKey] { + panic(fmt.Sprintf("There's a duplicate message offset in the same topic/partition %s/%d: %d. The tester has a bug!", claim.Topic(), 0, msg.offset)) + } + + cgs.waitingMessages[msgKey] = true + cgs.wgMessages.Add(1) + + claim.msgs <- &sarama.ConsumerMessage{ + Key: []byte(msg.key), + Value: msg.value, + Topic: claim.Topic(), + Offset: msg.offset, + } +} + +type cgClaim struct { + topic string + partition int32 + msgs chan *sarama.ConsumerMessage +} + +func newCgClaim(topic string, partition int32) *cgClaim { + return &cgClaim{ + topic: topic, + partition: partition, + msgs: make(chan *sarama.ConsumerMessage), + } +} + +// Topic returns the current topic of the claim +func (cgc *cgClaim) Topic() string { + return cgc.topic +} + +// Partition returns the partition +func (cgc *cgClaim) Partition() int32 { + return cgc.partition +} + +// InitialOffset returns the initial offset +func (cgc *cgClaim) InitialOffset() int64 { + return 0 +} + +// HighWaterMarkOffset returns the hwm offset +func (cgc *cgClaim) HighWaterMarkOffset() int64 { + return 0 +} + +// Messages returns the message channel that must be +func (cgc *cgClaim) Messages() <-chan *sarama.ConsumerMessage { + return cgc.msgs +} + +func (cgc *cgClaim) close() { + close(cgc.msgs) +} diff --git a/tester/producer.go b/tester/producer.go new file mode 100644 index 00000000..610e1fe1 --- /dev/null +++ b/tester/producer.go @@ -0,0 +1,52 @@ +package tester + +import ( + "github.com/lovoo/goka" +) + +// emitHandler abstracts a function that allows to overwrite kafkamock's Emit function to +// simulate producer errors +type emitHandler func(topic string, key string, value []byte) *goka.Promise + +type producerMock struct { + emitter emitHandler +} + +func newProducerMock(emitter emitHandler) *producerMock { + return &producerMock{ + emitter: emitter, + } +} + +// Emit emits messages to arbitrary topics. +// The mock simply forwards the emit to the KafkaMock which takes care of queueing calls +// to handled topics or putting the emitted messages in the emitted-messages-list +func (p *producerMock) Emit(topic string, key string, value []byte) *goka.Promise { + return p.emitter(topic, key, value) +} + +// Close closes the producer mock +// No action required in the mock. +func (p *producerMock) Close() error { + logger.Printf("Closing producer mock") + return nil +} + +// flushingProducer wraps the producer and +// waits for all consumers after the Emit. +type flushingProducer struct { + tester *Tester + producer goka.Producer +} + +// Emit using the underlying producer +func (e *flushingProducer) Emit(topic string, key string, value []byte) *goka.Promise { + prom := e.producer.Emit(topic, key, value) + e.tester.waitForClients() + return prom +} + +// Close using the underlying producer +func (e *flushingProducer) Close() error { + return e.producer.Close() +} diff --git a/tester/queue.go b/tester/queue.go index a18496f8..6b7dfeac 100644 --- a/tester/queue.go +++ b/tester/queue.go @@ -1,8 +1,6 @@ package tester import ( - "fmt" - "log" "sync" ) @@ -14,119 +12,116 @@ type message struct { type queue struct { sync.Mutex - topic string - messages []*message - hwm int64 - waitConsumerInit sync.WaitGroup - simpleConsumers map[*queueConsumer]int64 - groupConsumers map[*queueConsumer]int64 + topic string + messages []*message + hwm int64 } func newQueue(topic string) *queue { return &queue{ - topic: topic, - simpleConsumers: make(map[*queueConsumer]int64), - groupConsumers: make(map[*queueConsumer]int64), + topic: topic, } } -func (q *queue) size() int { - return len(q.messages) -} +func (q *queue) Hwm() int64 { + q.Lock() + defer q.Unlock() -func (q *queue) message(offset int) *message { - return q.messages[offset] + hwm := q.hwm + return hwm } -func (q *queue) messagesFrom(from int) []*message { - return q.messages[from:] +func (q *queue) push(key string, value []byte) int64 { + q.Lock() + defer q.Unlock() + offset := q.hwm + q.messages = append(q.messages, &message{ + offset: offset, + key: key, + value: value, + }) + q.hwm++ + return offset } -func (q *queue) expectGroupConsumer() { +func (q *queue) message(offset int) *message { q.Lock() defer q.Unlock() - q.groupConsumers[newQueueConsumer(q.topic, q)] = 0 + return q.messages[offset] } -func (q *queue) expectSimpleConsumer() { +func (q *queue) messagesFromOffset(offset int64) []*message { q.Lock() defer q.Unlock() - q.simpleConsumers[newQueueConsumer(q.topic, q)] = 0 + return q.messages[offset:] } -func (q *queue) bindConsumer(cons *consumer, groupConsumer bool) *queueConsumer { +func (q *queue) size() int { q.Lock() defer q.Unlock() + return len(q.messages) +} - consumers := q.simpleConsumers - if groupConsumer { - consumers = q.groupConsumers - } - for qCons := range consumers { - if !qCons.isBound() { - qCons.bindToConsumer(cons) - return qCons - } - } - panic(fmt.Errorf("did not find an unbound consumer for %s. The group graph was not parsed correctly", q.topic)) +// QueueTracker tracks message offsets for each topic for convenient +// 'expect message x to be in topic y' in unit tests +type QueueTracker struct { + t T + topic string + nextOffset int64 + tester *Tester } -func (q *queue) messagesFromOffset(offset int64) []*message { - q.Lock() - defer q.Unlock() - return q.messages[offset:] +func newQueueTracker(tester *Tester, t T, topic string) *QueueTracker { + return &QueueTracker{ + t: t, + topic: topic, + tester: tester, + nextOffset: tester.getOrCreateQueue(topic).hwm, + } } -// wait until all consumers are ready to consume (only for startup) -func (q *queue) waitConsumersInit() { - logger.Printf("Consumers in Queue %s", q.topic) - for cons := range q.groupConsumers { - logger.Printf("waiting for group consumer %s to be running or killed (state=%v)", cons.queue.topic, cons.state.State()) - - select { - case <-cons.state.WaitForState(killed): - log.Printf("At least one consumer was killed. No point in waiting for it") - return - case <-cons.state.WaitForState(running): - logger.Printf(" --> %s is running", cons.queue.topic) - } +// Next returns the next message since the last time this +// function was called (or MoveToEnd) +// It uses the known codec for the topic to decode the message +func (mt *QueueTracker) Next() (string, interface{}, bool) { + + key, msgRaw, hasNext := mt.NextRaw() + + if !hasNext { + return key, msgRaw, hasNext } - for cons := range q.simpleConsumers { - logger.Printf("waiting for simple consumer %s to be ready", cons.queue.topic) - select { - case <-cons.state.WaitForState(running): - case <-cons.state.WaitForState(stopped): - case <-cons.state.WaitForState(killed): - } - logger.Printf(" --> %s is ready", cons.queue.topic) + decoded, err := mt.tester.codecForTopic(mt.topic).Decode(msgRaw) + if err != nil { + mt.t.Fatalf("Error decoding message: %v", err) } + return key, decoded, true } -func (q *queue) waitForConsumers() int { - // wait until all consumers for the queue have processed all the messages - var numMessagesConsumed int - for sub := range q.simpleConsumers { - logger.Printf("waiting for simple consumer %s to finish up", q.topic) - numMessagesConsumed += sub.catchupAndSync() - logger.Printf(">> done waiting for simple consumer %s to finish up", q.topic) - } - for sub := range q.groupConsumers { - logger.Printf("waiting for simple consumer %s to finish up", q.topic) - numMessagesConsumed += sub.catchupAndSync() - logger.Printf(">> done waiting for simple consumer %s to finish up", q.topic) +// NextRaw returns the next message similar to Next(), but without the decoding +func (mt *QueueTracker) NextRaw() (string, []byte, bool) { + q := mt.tester.getOrCreateQueue(mt.topic) + if int(mt.nextOffset) >= q.size() { + return "", nil, false } - return numMessagesConsumed + msg := q.message(int(mt.nextOffset)) + + mt.nextOffset++ + return msg.key, msg.value, true } -func (q *queue) push(key string, value []byte) { - q.Lock() - defer q.Unlock() - q.messages = append(q.messages, &message{ - offset: q.hwm, - key: key, - value: value, - }) - q.hwm++ +// Seek moves the index pointer of the queue tracker to passed offset +func (mt *QueueTracker) Seek(offset int64) { + mt.nextOffset = offset +} + +// Hwm returns the tracked queue's hwm value +func (mt *QueueTracker) Hwm() int64 { + return mt.tester.getOrCreateQueue(mt.topic).Hwm() +} + +// NextOffset returns the tracker's next offset +func (mt *QueueTracker) NextOffset() int64 { + return mt.nextOffset } diff --git a/tester/queueconsumer.go b/tester/queueconsumer.go deleted file mode 100644 index 2616e6ee..00000000 --- a/tester/queueconsumer.go +++ /dev/null @@ -1,351 +0,0 @@ -package tester - -import ( - "fmt" - "log" - "sync" - "time" - - "github.com/lovoo/goka/kafka" -) - -type consumer struct { - tester *Tester - events chan kafka.Event - subscribedTopics map[string]*queueConsumer - simpleConsumers map[string]*queueConsumer - closeOnce sync.Once - sync.Mutex -} - -const ( - unbound State = iota - bound - running - stopping - stopped - killed -) - -const ( - eventBufferQueueSize = 100000 -) - -type queueConsumer struct { - queue *queue - nextOffset int64 - waitEventBuffer sync.WaitGroup - state *Signal - eventBuffer chan kafka.Event - events chan kafka.Event - consumer *consumer -} - -func newQueueConsumer(topic string, queue *queue) *queueConsumer { - qc := &queueConsumer{ - queue: queue, - eventBuffer: make(chan kafka.Event, eventBufferQueueSize), - state: NewSignal(unbound, bound, stopped, stopping, running, killed).SetState(unbound), - } - return qc -} - -func (qc *queueConsumer) bindToConsumer(cons *consumer) { - logger.Printf("binding consumer to topic %s", qc.queue.topic) - if !qc.state.IsState(unbound) { - panic(fmt.Errorf("error binding %s to consumer. Already bound", qc.queue.topic)) - } - qc.state.SetState(bound) - qc.consumer = cons - qc.events = cons.events -} - -func (qc *queueConsumer) isBound() bool { - return !qc.state.IsState(unbound) -} - -func (qc *queueConsumer) isRunning() bool { - return qc.state.IsState(running) -} - -func (qc *queueConsumer) setRunning() { - qc.state.SetState(running) -} - -func (qc *queueConsumer) stop() { - logger.Printf("closing the queueConsumer for topic %s", qc.queue.topic) - if !qc.state.IsState(running) { - panic(fmt.Sprintf("trying to stop consumer %s which is not running (state=%d)", qc.queue.topic, qc.state.State())) - } - qc.state.SetState(stopping) - logger.Printf("[consumer %s]waiting for stopped", qc.queue.topic) - <-qc.state.WaitForState(stopped) - logger.Printf("[consumer %s] stopped", qc.queue.topic) -} - -func (qc *queueConsumer) kill() { - qc.stop() - qc.state.SetState(killed) -} - -func (qc *queueConsumer) startLoop(setRunning bool) { - logger.Printf("starting queue consumer %s (set-running=%t)", qc.queue.topic, setRunning) - // not bound or already running - if qc.state.IsState(unbound) || qc.state.IsState(running) || qc.state.IsState(stopping) { - panic(fmt.Errorf("the queue consumer %s is in state %v. Cannot start", qc.queue.topic, qc.state.State())) - } - if setRunning { - qc.state.SetState(running) - } - go qc.consumeBuffer() -} - -func (qc *queueConsumer) consumeBuffer() { - defer func() { - err := recover() - if err != nil { - logger.Printf("Error consuming the buffer: %v", err) - } - qc.state.SetState(stopped) - }() - - for { - select { - case event, ok := <-qc.eventBuffer: - if !ok { - return - } - logger.Printf("[consumer %s]: From Buffer %#v", qc.queue.topic, event) - - select { - case qc.events <- event: - qc.waitEventBuffer.Done() - - logger.Printf("[consumer %s]: Buffer->Events %#v", qc.queue.topic, event) - case <-qc.state.WaitForState(stopping): - logger.Printf("[consumer %s] received stopping signal", qc.queue.topic) - - logger.Printf("[consumer %s] DROPPING MESSAGE (%#v) because the consumer is closed", qc.queue.topic, event) - qc.waitEventBuffer.Done() - return - } - - case <-qc.state.WaitForState(stopping): - logger.Printf("[consumer %s] received stopping signal", qc.queue.topic) - return - } - } -} - -func (qc *queueConsumer) catchupAndSync() int { - logger.Printf("[consumer %s] catching up", qc.queue.topic) - numMessages := qc.catchupQueue(-1) - logger.Printf("[consumer %s] catching up DONE (%d messages)", qc.queue.topic, numMessages) - - eventsProcessed := make(chan struct{}) - go func() { - logger.Printf("[consumer %s] wait for all events to be processed", qc.queue.topic) - qc.waitEventBuffer.Wait() - logger.Printf("[consumer %s] done processing events", qc.queue.topic) - close(eventsProcessed) - }() - - select { - case <-eventsProcessed: - case <-qc.state.WaitForState(killed): - // The consumer was killed, so we assume the test is done already. - return 0 - case <-qc.state.WaitForState(stopped): - } - return numMessages -} - -func (qc *queueConsumer) startGroupConsumer() { - logger.Printf("[consumer %s] starting group consumer", qc.queue.topic) - qc.catchupQueue(-1) -} - -func (qc *queueConsumer) addToBuffer(event kafka.Event) { - qc.waitEventBuffer.Add(1) - - qc.eventBuffer <- event - - if len(qc.eventBuffer) > eventBufferQueueSize*0.9 { - logger.Printf("buffer nearly full: %d, %s. Will drop event.", len(qc.eventBuffer), qc.queue.topic) - <-qc.eventBuffer - } -} - -func (qc *queueConsumer) startSimpleConsumer(offset int64, firstStart bool) { - logger.Printf("[consumer %s] starting simple consumer (offset=%d)", qc.queue.topic, offset) - if firstStart { - qc.addToBuffer(&kafka.BOF{ - Hwm: qc.queue.hwm, - Offset: 0, - Partition: 0, - Topic: qc.queue.topic, - }) - qc.catchupQueue(offset) - qc.addToBuffer(&kafka.EOF{ - Hwm: qc.queue.hwm, - Partition: 0, - Topic: qc.queue.topic, - }) - } - qc.startLoop(true) -} - -func (qc *queueConsumer) catchupQueue(fromOffset int64) int { - // we'll always get from the beginning when the consumer - // requests -1 or -2 (for end or beginning resp) - if fromOffset < 0 { - fromOffset = qc.nextOffset - } - - // count how many messages we had to catch up on - var forwardedMessages int - for _, msg := range qc.queue.messagesFromOffset(fromOffset) { - qc.addToBuffer(&kafka.Message{ - Key: string(msg.key), - Offset: msg.offset, - Partition: 0, - Timestamp: time.Unix(msg.offset, 0), - Topic: qc.queue.topic, - Value: msg.value, - }) - forwardedMessages++ - // mark the next offset to consume in case we stop here - qc.nextOffset = msg.offset + 1 - } - - qc.addToBuffer(&kafka.EOF{ - Hwm: qc.queue.hwm, - Partition: 0, - Topic: qc.queue.topic, - }) - - // push some more NOPs - for i := 0; i < 2; i++ { - qc.addToBuffer(&kafka.NOP{ - Partition: 0, - Topic: qc.queue.topic, - }) - } - return forwardedMessages -} - -func (qc *queueConsumer) rebalance() { - qc.addToBuffer(&kafka.Assignment{ - 0: -1, - }) -} - -func newConsumer(tester *Tester) *consumer { - return &consumer{ - tester: tester, - events: make(chan kafka.Event, 0), - simpleConsumers: make(map[string]*queueConsumer), - subscribedTopics: make(map[string]*queueConsumer), - } -} - -// Events returns the event channel of the consumer mock -func (tc *consumer) Events() <-chan kafka.Event { - return tc.events -} - -// Subscribe marks the consumer to subscribe to passed topics. -// The consumerMock simply marks the topics as handled to make sure to -// pass emitted messages back to the processor. -func (tc *consumer) Subscribe(topics map[string]int64) error { - log.Printf("consumer: subscribing to topics: %v", topics) - var anyTopic string - for topic := range topics { - anyTopic = topic - if _, exists := tc.subscribedTopics[topic]; exists { - logger.Printf("consumer for %s already exists. This is strange", topic) - } - logger.Printf("Subscribe %s", topic) - tc.subscribedTopics[topic] = tc.tester.getOrCreateQueue(topic).bindConsumer(tc, true) - tc.subscribedTopics[topic].startLoop(false) - } - - tc.subscribedTopics[anyTopic].rebalance() - return nil -} - -// AddGroupPartition adds a partition for group consumption. -// No action required in the mock. -func (tc *consumer) AddGroupPartition(partition int32) { - for _, consumer := range tc.subscribedTopics { - logger.Printf("AddGroupPartition %s", consumer.queue.topic) - consumer.startGroupConsumer() - consumer.setRunning() - } -} - -// Commit commits an offest. -// No action required in the mock. -func (tc *consumer) Commit(topic string, partition int32, offset int64) error { - return nil -} - -// AddPartition marks the topic as a table topic. -// The mock has to know the group table topic to ignore emit calls (which would never be consumed) -func (tc *consumer) AddPartition(topic string, partition int32, initialOffset int64) error { - tc.Lock() - defer tc.Unlock() - logger.Printf("AddPartition %s", topic) - var firstStart bool - if _, exists := tc.simpleConsumers[topic]; !exists { - firstStart = true - tc.simpleConsumers[topic] = tc.tester.getOrCreateQueue(topic).bindConsumer(tc, false) - } else { - logger.Printf("AddPartition %s: consumer already existed. Will reuse the one", topic) - } - if tc.simpleConsumers[topic].isRunning() { - panic(fmt.Errorf("simple consumer for %s already running. RemovePartition not called or race condition", topic)) - } - tc.simpleConsumers[topic].startSimpleConsumer(initialOffset, firstStart) - - return nil -} - -// RemovePartition removes a partition from a topic. -// No action required in the mock. -func (tc *consumer) RemovePartition(topic string, partition int32) error { - logger.Printf("consumer RemovePartition %s", topic) - if cons, exists := tc.simpleConsumers[topic]; exists { - cons.stop() - } else { - logger.Printf("consumer for topic %s did not exist. Cannot Remove partition", topic) - } - return nil -} - -// Close closes the consumer. -func (tc *consumer) Close() error { - tc.closeOnce.Do(func() { - logger.Printf("closing tester consumer. Will close all subscribed topics") - for _, cons := range tc.subscribedTopics { - if cons.isRunning() { - logger.Printf("closing queue consumer for %s", cons.queue.topic) - cons.kill() - } else { - logger.Printf("queue consumer for %s is not running", cons.queue.topic) - } - } - - for _, cons := range tc.simpleConsumers { - if cons.isRunning() { - logger.Printf("closing simple consumer for %s", cons.queue.topic) - cons.kill() - } else { - logger.Printf("queue consumer for %s is not running", cons.queue.topic) - } - } - - close(tc.events) - }) - return nil -} diff --git a/tester/queuetracker.go b/tester/queuetracker.go deleted file mode 100644 index 32061af4..00000000 --- a/tester/queuetracker.go +++ /dev/null @@ -1,63 +0,0 @@ -package tester - -// QueueTracker tracks message offsets for each topic for convenient -// 'expect message x to be in topic y' in unit tests -type QueueTracker struct { - t T - topic string - nextOffset int64 - tester *Tester -} - -func newQueueTracker(tester *Tester, t T, topic string) *QueueTracker { - return &QueueTracker{ - t: t, - topic: topic, - tester: tester, - } -} - -// Next returns the next message since the last time this -// function was called (or MoveToEnd) -// It uses the known codec for the topic to decode the message -func (mt *QueueTracker) Next() (string, interface{}, bool) { - - key, msgRaw, hasNext := mt.NextRaw() - - if !hasNext { - return key, msgRaw, hasNext - } - - decoded, err := mt.tester.codecForTopic(mt.topic).Decode(msgRaw) - if err != nil { - mt.t.Fatalf("Error decoding message: %v", err) - } - return key, decoded, true -} - -// NextRaw returns the next message similar to Next(), but without the decoding -func (mt *QueueTracker) NextRaw() (string, []byte, bool) { - q := mt.tester.queueForTopic(mt.topic) - if int(mt.nextOffset) >= q.size() { - return "", nil, false - } - msg := q.message(int(mt.nextOffset)) - - mt.nextOffset++ - return msg.key, msg.value, true -} - -// Seek moves the index pointer of the queue tracker to passed offset -func (mt *QueueTracker) Seek(offset int64) { - mt.nextOffset = offset -} - -// Hwm returns the tracked queue's hwm value -func (mt *QueueTracker) Hwm() int64 { - return mt.tester.queueForTopic(mt.topic).hwm -} - -// NextOffset returns the tracker's next offset -func (mt *QueueTracker) NextOffset() int64 { - return mt.nextOffset -} diff --git a/tester/tester.go b/tester/tester.go index 347e75cb..595ceea5 100644 --- a/tester/tester.go +++ b/tester/tester.go @@ -1,24 +1,16 @@ package tester import ( - "flag" "fmt" "hash" - "log" - "os" "reflect" "sync" "github.com/lovoo/goka" - "github.com/lovoo/goka/kafka" "github.com/lovoo/goka/storage" -) -// Codec decodes and encodes from and to []byte -type Codec interface { - Encode(value interface{}) (data []byte, err error) - Decode(data []byte) (value interface{}, err error) -} + "github.com/Shopify/sarama" +) type debugLogger interface { Printf(s string, args ...interface{}) @@ -29,342 +21,257 @@ type nilLogger int func (*nilLogger) Printf(s string, args ...interface{}) {} var ( - debug = flag.Bool("tester-debug", false, "show debug prints of the tester.") logger debugLogger = new(nilLogger) ) -// EmitHandler abstracts a function that allows to overwrite kafkamock's Emit function to -// simulate producer errors -type EmitHandler func(topic string, key string, value []byte) *kafka.Promise - -type queuedMessage struct { - topic string - key string - value []byte +// T abstracts the interface we assume from the test case. +// Will most likely be T +type T interface { + Errorf(format string, args ...interface{}) + Fatalf(format string, args ...interface{}) + Fatal(a ...interface{}) } -// Tester allows interacting with a test processor +// Tester mimicks kafka for complex highlevel testing of single or multiple processors/views/emitters type Tester struct { - t T + t T + producer *producerMock + tmgr goka.TopicManager - producerMock *producerMock - topicMgrMock *topicMgrMock - emitHandler EmitHandler - storages map[string]storage.Storage + mClients sync.RWMutex + clients map[string]*client codecs map[string]goka.Codec + mQueues sync.Mutex topicQueues map[string]*queue - mQueues sync.RWMutex - mStorages sync.RWMutex - queuedMessages []*queuedMessage + mStorages sync.Mutex + storages map[string]storage.Storage } -func (km *Tester) queueForTopic(topic string) *queue { - km.mQueues.RLock() - defer km.mQueues.RUnlock() - q, exists := km.topicQueues[topic] - if !exists { - panic(fmt.Errorf("No queue for topic %s", topic)) - } - return q -} +// New creates a new tester instance +func New(t T) *Tester { -// NewQueueTracker creates a message tracker that starts tracking -// the messages from the end of the current queues -func (km *Tester) NewQueueTracker(topic string) *QueueTracker { - km.waitStartup() + tt := &Tester{ + t: t, - mt := newQueueTracker(km, km.t, topic) - km.mQueues.RLock() - defer km.mQueues.RUnlock() - mt.Seek(mt.Hwm()) - return mt -} + clients: make(map[string]*client), -func (km *Tester) getOrCreateQueue(topic string) *queue { - km.mQueues.RLock() - _, exists := km.topicQueues[topic] - km.mQueues.RUnlock() - if !exists { - km.mQueues.Lock() - if _, exists = km.topicQueues[topic]; !exists { - km.topicQueues[topic] = newQueue(topic) - } - km.mQueues.Unlock() + codecs: make(map[string]goka.Codec), + topicQueues: make(map[string]*queue), + storages: make(map[string]storage.Storage), } + tt.tmgr = NewMockTopicManager(tt, 1, 1) + tt.producer = newProducerMock(tt.handleEmit) - km.mQueues.RLock() - defer km.mQueues.RUnlock() - return km.topicQueues[topic] + return tt } -// T abstracts the interface we assume from the test case. -// Will most likely be *testing.T -type T interface { - Errorf(format string, args ...interface{}) - Fatalf(format string, args ...interface{}) - Fatal(a ...interface{}) +func (tt *Tester) nextClient() *client { + tt.mClients.Lock() + defer tt.mClients.Unlock() + c := &client{ + clientID: fmt.Sprintf("client-%d", len(tt.clients)), + consumer: newConsumerMock(tt), + } + tt.clients[c.clientID] = c + return c +} + +// ConsumerGroupBuilder builds a builder. The builder returns the consumergroup for passed client-ID +// if it was expected by registering the processor to the Tester +func (tt *Tester) ConsumerGroupBuilder() goka.ConsumerGroupBuilder { + return func(brokers []string, group, clientID string) (sarama.ConsumerGroup, error) { + tt.mClients.RLock() + defer tt.mClients.RUnlock() + client, exists := tt.clients[clientID] + if !exists { + return nil, fmt.Errorf("cannot create consumergroup because no client registered with ID: %s", clientID) + } + + if client.consumerGroup == nil { + return nil, fmt.Errorf("Did not expect a group graph") + } + + return client.consumerGroup, nil + } } -// New returns a new Tester. -// It should be passed as goka.WithTester to goka.NewProcessor. -func New(t T) *Tester { +// ConsumerBuilder creates a consumerbuilder that builds consumers for passed clientID +func (tt *Tester) ConsumerBuilder() goka.SaramaConsumerBuilder { + return func(brokers []string, clientID string) (sarama.Consumer, error) { + tt.mClients.RLock() + defer tt.mClients.RUnlock() + + client, exists := tt.clients[clientID] + if !exists { + return nil, fmt.Errorf("cannot create sarama consumer because no client registered with ID: %s", clientID) + } - // activate the logger if debug is turned on - if *debug { - logger = log.New(os.Stderr, " ", 0) + return client.consumer, nil } +} - tester := &Tester{ - t: t, - codecs: make(map[string]goka.Codec), - topicQueues: make(map[string]*queue), - storages: make(map[string]storage.Storage), +// EmitterProducerBuilder creates a producer builder used for Emitters. +// Emitters need to flush when emitting messages. +func (tt *Tester) EmitterProducerBuilder() goka.ProducerBuilder { + builder := tt.ProducerBuilder() + return func(b []string, cid string, hasher func() hash.Hash32) (goka.Producer, error) { + prod, err := builder(b, cid, hasher) + return &flushingProducer{ + tester: tt, + producer: prod, + }, err } - tester.producerMock = newProducerMock(tester.handleEmit) - tester.topicMgrMock = newTopicMgrMock(tester) - return tester } -func (km *Tester) registerCodec(topic string, codec goka.Codec) { - if existingCodec, exists := km.codecs[topic]; exists { - if reflect.TypeOf(codec) != reflect.TypeOf(existingCodec) { - panic(fmt.Errorf("There are different codecs for the same topic. This is messed up (%#v, %#v)", codec, existingCodec)) - } +// handleEmit handles an Emit-call on the producerMock. +// This takes care of queueing calls +// to handled topics or putting the emitted messages in the emitted-messages-list +func (tt *Tester) handleEmit(topic string, key string, value []byte) *goka.Promise { + promise := goka.NewPromise() + offset := tt.pushMessage(topic, key, value) + return promise.Finish(&sarama.ProducerMessage{Offset: offset}, nil) +} + +func (tt *Tester) pushMessage(topic string, key string, data []byte) int64 { + return tt.getOrCreateQueue(topic).push(key, data) +} + +func (tt *Tester) ProducerBuilder() goka.ProducerBuilder { + return func(b []string, cid string, hasher func() hash.Hash32) (goka.Producer, error) { + return tt.producer, nil } - km.codecs[topic] = codec } -func (km *Tester) codecForTopic(topic string) goka.Codec { - codec, exists := km.codecs[topic] - if !exists { - panic(fmt.Errorf("No codec for topic %s registered.", topic)) +func (tt *Tester) TopicManagerBuilder() goka.TopicManagerBuilder { + return func(brokers []string) (goka.TopicManager, error) { + return tt.tmgr, nil } - return codec } // RegisterGroupGraph is called by a processor when the tester is passed via // `WithTester(..)`. // This will setup the tester with the neccessary consumer structure -func (km *Tester) RegisterGroupGraph(gg *goka.GroupGraph) { +func (tt *Tester) RegisterGroupGraph(gg *goka.GroupGraph) string { + + client := tt.nextClient() + // we need to expect a consumer group so we're creating one in the client + if gg.GroupTable() != nil || len(gg.InputStreams()) > 0 { + client.consumerGroup = newConsumerGroup(tt.t, tt) + } + + // register codecs if gg.GroupTable() != nil { - km.getOrCreateQueue(gg.GroupTable().Topic()).expectSimpleConsumer() - km.registerCodec(gg.GroupTable().Topic(), gg.GroupTable().Codec()) + tt.registerCodec(gg.GroupTable().Topic(), gg.GroupTable().Codec()) } for _, input := range gg.InputStreams() { - km.getOrCreateQueue(input.Topic()).expectGroupConsumer() - km.registerCodec(input.Topic(), input.Codec()) + tt.registerCodec(input.Topic(), input.Codec()) } for _, output := range gg.OutputStreams() { - km.registerCodec(output.Topic(), output.Codec()) - km.getOrCreateQueue(output.Topic()) + tt.registerCodec(output.Topic(), output.Codec()) } + for _, join := range gg.JointTables() { - km.getOrCreateQueue(join.Topic()).expectSimpleConsumer() - km.registerCodec(join.Topic(), join.Codec()) + tt.registerCodec(join.Topic(), join.Codec()) } if loop := gg.LoopStream(); loop != nil { - km.getOrCreateQueue(loop.Topic()).expectGroupConsumer() - km.registerCodec(loop.Topic(), loop.Codec()) + tt.registerCodec(loop.Topic(), loop.Codec()) } for _, lookup := range gg.LookupTables() { - km.getOrCreateQueue(lookup.Topic()).expectSimpleConsumer() - km.registerCodec(lookup.Topic(), lookup.Codec()) + tt.registerCodec(lookup.Topic(), lookup.Codec()) } + return client.clientID } -// RegisterView registers a view to be working with the tester. -func (km *Tester) RegisterView(table goka.Table, c goka.Codec) { - km.getOrCreateQueue(string(table)).expectSimpleConsumer() - km.registerCodec(string(table), c) +// RegisterView registers a new view to the tester +func (tt *Tester) RegisterView(table goka.Table, c goka.Codec) string { + tt.registerCodec(string(table), c) + client := tt.nextClient() + client.requireConsumer(string(table)) + return client.clientID } // RegisterEmitter registers an emitter to be working with the tester. -func (km *Tester) RegisterEmitter(topic goka.Stream, codec goka.Codec) { - km.registerCodec(string(topic), codec) - km.getOrCreateQueue(string(topic)) -} - -// TopicManagerBuilder returns the topicmanager builder when this tester is used as an option -// to a processor -func (km *Tester) TopicManagerBuilder() kafka.TopicManagerBuilder { - return func(brokers []string) (kafka.TopicManager, error) { - return km.topicMgrMock, nil - } -} - -// ConsumerBuilder returns the consumer builder when this tester is used as an option -// to a processor -func (km *Tester) ConsumerBuilder() kafka.ConsumerBuilder { - return func(b []string, group, clientID string) (kafka.Consumer, error) { - return newConsumer(km), nil - } +func (tt *Tester) RegisterEmitter(topic goka.Stream, codec goka.Codec) { + tt.registerCodec(string(topic), codec) } -// ProducerBuilder returns the producer builder when this tester is used as an option -// to a processor -func (km *Tester) ProducerBuilder() kafka.ProducerBuilder { - return func(b []string, cid string, hasher func() hash.Hash32) (kafka.Producer, error) { - return km.producerMock, nil - } -} - -// EmitterProducerBuilder creates a producer builder used for Emitters. -// Emitters need to flush when emitting messages. -func (km *Tester) EmitterProducerBuilder() kafka.ProducerBuilder { - builder := km.ProducerBuilder() - return func(b []string, cid string, hasher func() hash.Hash32) (kafka.Producer, error) { - prod, err := builder(b, cid, hasher) - return &flushingProducer{ - tester: km, - producer: prod, - }, err - } -} - -// StorageBuilder returns the storage builder when this tester is used as an option -// to a processor -func (km *Tester) StorageBuilder() storage.Builder { - return func(topic string, partition int32) (storage.Storage, error) { - km.mStorages.RLock() - if st, exists := km.storages[topic]; exists { - km.mStorages.RUnlock() - return st, nil - } - km.mStorages.RUnlock() - st := storage.NewMemory() - km.mStorages.Lock() - km.storages[topic] = st - km.mStorages.Unlock() - return st, nil - } -} - -func (km *Tester) waitForConsumers() { - - logger.Printf("waiting for consumers") - for { - if len(km.queuedMessages) == 0 { - break - } - next := km.queuedMessages[0] - km.queuedMessages = km.queuedMessages[1:] - - km.getOrCreateQueue(next.topic).push(next.key, next.value) - - km.mQueues.RLock() - for { - var messagesConsumed int - for _, queue := range km.topicQueues { - messagesConsumed += queue.waitForConsumers() - } - if messagesConsumed == 0 { - break - } - } - km.mQueues.RUnlock() +func (tt *Tester) getOrCreateQueue(topic string) *queue { + tt.mQueues.Lock() + defer tt.mQueues.Unlock() + queue, exists := tt.topicQueues[topic] + if !exists { + queue = newQueue(topic) + tt.topicQueues[topic] = queue } - - logger.Printf("waiting for consumers done") + return queue } -func (km *Tester) waitStartup() { - logger.Printf("Waiting for startup") - km.mQueues.RLock() - defer km.mQueues.RUnlock() - for _, queue := range km.topicQueues { - queue.waitConsumersInit() +func (tt *Tester) codecForTopic(topic string) goka.Codec { + codec, exists := tt.codecs[topic] + if !exists { + panic(fmt.Errorf("no codec for topic %s registered", topic)) } - logger.Printf("Waiting for startup done") + return codec } -// Consume a message using the topic's configured codec -func (km *Tester) Consume(topic string, key string, msg interface{}) { - km.waitStartup() +func (tt *Tester) registerCodec(topic string, codec goka.Codec) { + // create a queue, we're going to need it anyway + tt.getOrCreateQueue(topic) - // if the user wants to send a nil for some reason, - // just let her. Goka should handle it accordingly :) - value := reflect.ValueOf(msg) - if msg == nil || (value.Kind() == reflect.Ptr && value.IsNil()) { - km.pushMessage(topic, key, nil) - } else { - data, err := km.codecForTopic(topic).Encode(msg) - if err != nil { - panic(fmt.Errorf("Error encoding value %v: %v", msg, err)) + if existingCodec, exists := tt.codecs[topic]; exists { + if reflect.TypeOf(codec) != reflect.TypeOf(existingCodec) { + panic(fmt.Errorf("There are different codecs for the same topic. This is messed up (%#v, %#v)", codec, existingCodec)) } - km.pushMessage(topic, key, data) } - - km.waitForConsumers() -} - -// ConsumeData pushes a marshalled byte slice to a topic and a key -func (km *Tester) ConsumeData(topic string, key string, data []byte) { - km.waitStartup() - km.pushMessage(topic, key, data) - km.waitForConsumers() + tt.codecs[topic] = codec } -func (km *Tester) pushMessage(topic string, key string, data []byte) { - km.queuedMessages = append(km.queuedMessages, &queuedMessage{topic: topic, key: key, value: data}) -} - -// handleEmit handles an Emit-call on the producerMock. -// This takes care of queueing calls -// to handled topics or putting the emitted messages in the emitted-messages-list -func (km *Tester) handleEmit(topic string, key string, value []byte) *kafka.Promise { - promise := kafka.NewPromise() - km.pushMessage(topic, key, value) - return promise.Finish(nil) -} - -// TableValue attempts to get a value from any table that is used in the kafka mock. -func (km *Tester) TableValue(table goka.Table, key string) interface{} { - km.waitStartup() +// TableValue attempts to get a value from any table that is used in the tester +func (tt *Tester) TableValue(table goka.Table, key string) interface{} { + tt.waitStartup() topic := string(table) - km.mStorages.RLock() - st, exists := km.storages[topic] - km.mStorages.RUnlock() + tt.mStorages.Lock() + st, exists := tt.storages[topic] + tt.mStorages.Unlock() if !exists { panic(fmt.Errorf("topic %s does not exist", topic)) } item, err := st.Get(key) if err != nil { - km.t.Fatalf("Error getting table value from storage (table=%s, key=%s): %v", table, key, err) + tt.t.Fatalf("Error getting table value from storage (table=%s, key=%s): %v", table, key, err) } if item == nil { return nil } - value, err := km.codecForTopic(topic).Decode(item) + value, err := tt.codecForTopic(topic).Decode(item) if err != nil { - km.t.Fatalf("error decoding value from storage (table=%s, key=%s, value=%v): %v", table, key, item, err) + tt.t.Fatalf("error decoding value from storage (table=%s, key=%s, value=%v): %v", table, key, item, err) } return value } // SetTableValue sets a value in a processor's or view's table direcly via storage -func (km *Tester) SetTableValue(table goka.Table, key string, value interface{}) { - km.waitStartup() - - logger.Printf("setting value is not implemented yet.") +// This method blocks until all expected clients are running, so make sure +// to call it *after* you have started all processors/views, otherwise it'll deadlock. +func (tt *Tester) SetTableValue(table goka.Table, key string, value interface{}) { + tt.waitStartup() topic := string(table) - km.mStorages.RLock() - st, exists := km.storages[topic] - km.mStorages.RUnlock() - if !exists { - panic(fmt.Errorf("storage for topic %s does not exist", topic)) + st, err := tt.getOrCreateStorage(topic) + if err != nil { + panic(fmt.Errorf("error creating storage for topic %s: %v", topic, err)) } - data, err := km.codecForTopic(topic).Encode(value) + data, err := tt.codecForTopic(topic).Encode(value) if err != nil { - km.t.Fatalf("error decoding value from storage (table=%s, key=%s, value=%v): %v", table, key, value, err) + tt.t.Fatalf("error decoding value from storage (table=%s, key=%s, value=%v): %v", table, key, value, err) } err = st.Set(key, data) @@ -372,101 +279,87 @@ func (km *Tester) SetTableValue(table goka.Table, key string, value interface{}) panic(fmt.Errorf("Error setting key %s in storage %s: %v", key, table, err)) } } +func (tt *Tester) getOrCreateStorage(table string) (storage.Storage, error) { + tt.mStorages.Lock() + defer tt.mStorages.Unlock() -// ReplaceEmitHandler replaces the emitter. -func (km *Tester) ReplaceEmitHandler(emitter EmitHandler) { - km.producerMock.emitter = emitter + st := tt.storages[table] + if st == nil { + st = storage.NewMemory() + tt.storages[table] = st + } + return st, nil } -// ClearValues resets all table values -func (km *Tester) ClearValues() { - km.mStorages.Lock() - for topic, st := range km.storages { +// StorageBuilder builds inmemory storages +func (tt *Tester) StorageBuilder() storage.Builder { + return func(topic string, partition int32) (storage.Storage, error) { + return tt.getOrCreateStorage(topic) + } +} + +// ClearValues clears all table values in all storages +func (tt *Tester) ClearValues() { + tt.mStorages.Lock() + defer tt.mStorages.Unlock() + for topic, st := range tt.storages { logger.Printf("clearing all values from storage for topic %s", topic) it, _ := st.Iterator() for it.Next() { st.Delete(string(it.Key())) } } - km.mStorages.Unlock() -} - -type topicMgrMock struct { - tester *Tester -} -// EnsureTableExists checks that a table (log-compacted topic) exists, or create one if possible -func (tm *topicMgrMock) EnsureTableExists(topic string, npar int) error { - return nil } -// EnsureStreamExists checks that a stream topic exists, or create one if possible -func (tm *topicMgrMock) EnsureStreamExists(topic string, npar int) error { - return nil +// NewQueueTracker creates a new queue tracker +func (tt *Tester) NewQueueTracker(topic string) *QueueTracker { + return newQueueTracker(tt, tt.t, topic) } -// EnsureTopicExists checks that a stream exists, or create one if possible -func (tm *topicMgrMock) EnsureTopicExists(topic string, npar, rfactor int, config map[string]string) error { - return nil -} +func (tt *Tester) waitStartup() { + tt.mClients.RLock() + defer tt.mClients.RUnlock() -// Partitions returns the number of partitions of a topic, that are assigned to the running -// instance, i.e. it doesn't represent all partitions of a topic. -func (tm *topicMgrMock) Partitions(topic string) ([]int32, error) { - return []int32{0}, nil -} - -// Close closes the topic manager. -// No action required in the mock. -func (tm *topicMgrMock) Close() error { - return nil -} - -func newTopicMgrMock(tester *Tester) *topicMgrMock { - return &topicMgrMock{ - tester: tester, + for _, client := range tt.clients { + client.waitStartup() } } -type producerMock struct { - emitter EmitHandler -} +func (tt *Tester) waitForClients() { + logger.Printf("waiting for consumers") -func newProducerMock(emitter EmitHandler) *producerMock { - return &producerMock{ - emitter: emitter, - } -} + tt.mClients.RLock() + defer tt.mClients.RUnlock() + for { + var totalCatchup int + for _, client := range tt.clients { + totalCatchup += client.catchup() + } -// Emit emits messages to arbitrary topics. -// The mock simply forwards the emit to the KafkaMock which takes care of queueing calls -// to handled topics or putting the emitted messages in the emitted-messages-list -func (p *producerMock) Emit(topic string, key string, value []byte) *kafka.Promise { - return p.emitter(topic, key, value) -} + if totalCatchup == 0 { + break + } + } -// Close closes the producer mock -// No action required in the mock. -func (p *producerMock) Close() error { - logger.Printf("Closing producer mock") - return nil + logger.Printf("waiting for consumers done") } -// flushingProducer wraps the producer and -// waits for all consumers after the Emit. -type flushingProducer struct { - tester *Tester - producer kafka.Producer -} +// Consume pushes a message for topic/key to be consumed by all processors/views +// whoever is using it being registered to the Tester +func (tt *Tester) Consume(topic string, key string, msg interface{}) { + tt.waitStartup() -// Emit using the underlying producer -func (e *flushingProducer) Emit(topic string, key string, value []byte) *kafka.Promise { - prom := e.producer.Emit(topic, key, value) - e.tester.waitForConsumers() - return prom -} + value := reflect.ValueOf(msg) + if msg == nil || (value.Kind() == reflect.Ptr && value.IsNil()) { + tt.pushMessage(topic, key, nil) + } else { + data, err := tt.codecForTopic(topic).Encode(msg) + if err != nil { + panic(fmt.Errorf("Error encoding value %v: %v", msg, err)) + } + tt.pushMessage(topic, key, data) + } -// Close using the underlying producer -func (e *flushingProducer) Close() error { - return e.producer.Close() + tt.waitForClients() } diff --git a/tester/tester_test.go b/tester/tester_test.go deleted file mode 100644 index 33ec2977..00000000 --- a/tester/tester_test.go +++ /dev/null @@ -1,565 +0,0 @@ -package tester - -import ( - "context" - "fmt" - "log" - "reflect" - "sync" - "testing" - "time" - - "github.com/lovoo/goka" - "github.com/lovoo/goka/codec" -) - -// simple consume function that is used in different tests -func increment(ctx goka.Context, msg interface{}) { - value := ctx.Value() - var state int64 - if value != nil { - state = value.(int64) - } - state++ - ctx.SetValue(state) -} - -func runProcOrFail(proc *goka.Processor) { - go func() { - err := proc.Run(context.Background()) - panic(fmt.Errorf("Processor run errors: %v", err)) - }() -} - -func Test_SimpleConsume(t *testing.T) { - gkt := New(t) - - var receivedMessage string - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("group", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - receivedMessage = msg.(string) - }), - ), - goka.WithTester(gkt), - ) - go proc.Run(context.Background()) - for i := 0; i < 101; i++ { - gkt.Consume("input", "key", fmt.Sprintf("%d", i)) - } - - if receivedMessage != "100" { - t.Fatalf("Message did not get through...") - } -} - -func Test_InputOutput(t *testing.T) { - gkt := New(t) - - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("group", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.Emit("output", ctx.Key(), msg) - }), - goka.Output("output", new(codec.String)), - ), - goka.WithTester(gkt), - ) - go proc.Run(context.Background()) - - mt := gkt.NewQueueTracker("output") - - if mt.NextOffset() != mt.Hwm() { - t.Fatalf("expected queue to be at end but wasn't") - } - - gkt.Consume("input", "key", "value") - - key, value, ok := mt.Next() - - if key != "key" || !reflect.DeepEqual(value, "value") || !ok { - t.Fatalf("Message was not received in the output queue") - } -} - -func Test_SimplePersist(t *testing.T) { - - gkt := New(t) - - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("group", - goka.Input("input", new(codec.String), increment), - goka.Persist(new(codec.Int64)), - ), - goka.WithTester(gkt), - ) - go func() { - err := proc.Run(context.Background()) - t.Fatalf("Processor run errors: %v", err) - }() - - for i := 0; i < 100; i++ { - gkt.Consume("input", "key", fmt.Sprintf("message - %d", i)) - } - - value := gkt.TableValue("group-table", "key") - if value.(int64) != 100 { - t.Fatalf("Message did not get through. was %d", value.(int64)) - } -} - -func Test_Persist_InitialState(t *testing.T) { - - gkt := New(t) - - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("group", - goka.Input("input", new(codec.String), increment), - goka.Persist(new(codec.Int64)), - ), - goka.WithTester(gkt), - ) - - go func() { - err := proc.Run(context.Background()) - t.Fatalf("Processor run errors: %v", err) - }() - - gkt.SetTableValue("group-table", "existing", int64(150)) - gkt.Consume("input", "existing", "") - - if gkt.TableValue("group-table", "existing").(int64) != 151 { - t.Fatalf("initial state was not loaded. Expected 151, got %v", gkt.TableValue("group-table", "existing")) - } -} - -// Tests multiple processors in a single mock -func Test_MultiProcessor(t *testing.T) { - - gkt := New(t) - - // first processor gets input and emits to increment topic - input, _ := goka.NewProcessor([]string{}, goka.DefineGroup("numbers", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - time.Sleep(10 * time.Millisecond) - ctx.Emit("forward1", ctx.Key(), "") - }), - goka.Output("forward1", new(codec.String)), - ), - goka.WithTester(gkt), - ) - - forward, _ := goka.NewProcessor([]string{}, goka.DefineGroup("forward1", - goka.Input("forward1", new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.Emit("forward2", ctx.Key(), "") - }), - goka.Output("forward2", new(codec.String)), - ), - goka.WithTester(gkt), - ) - forward2, _ := goka.NewProcessor([]string{}, goka.DefineGroup("forward2", - goka.Input("forward2", new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.Emit("forward3", ctx.Key(), "") - }), - goka.Output("forward3", new(codec.String)), - ), - goka.WithTester(gkt), - ) - forward3, _ := goka.NewProcessor([]string{}, goka.DefineGroup("forward3", - goka.Input("forward3", new(codec.String), func(ctx goka.Context, msg interface{}) { - // sleep in between so we know for sure when the waiting implementation is somehow buggy - time.Sleep(10 * time.Millisecond) - ctx.Emit("increment", ctx.Key(), "") - }), - goka.Output("increment", new(codec.String)), - ), - goka.WithTester(gkt), - ) - // second processor increments its state - incrementer, _ := goka.NewProcessor([]string{}, goka.DefineGroup("accu", - goka.Input("increment", new(codec.String), increment), - goka.Persist(new(codec.Int64)), - ), - goka.WithTester(gkt), - ) - - runProcOrFail(input) - runProcOrFail(forward) - runProcOrFail(forward2) - runProcOrFail(forward3) - runProcOrFail(incrementer) - - gkt.Consume("input", "test", "") - - if gkt.TableValue("accu-table", "test").(int64) != 1 { - t.Fatalf("the message did not reached the end") - } -} - -func Test_Loop(t *testing.T) { - gkt := New(t) - - // first processor gets input and emits to increment topi c - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("looptest", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.Loopback("loop-key", "loopvalue") - }), - goka.Persist(new(codec.Int64)), - goka.Loop(new(codec.String), increment), - ), - goka.WithTester(gkt), - ) - runProcOrFail(proc) - - gkt.Consume("input", "test", "") - if gkt.TableValue("looptest-table", "loop-key").(int64) != 1 { - t.Fatalf("loop failed") - } -} - -func Test_Lookup(t *testing.T) { - - gkt := New(t) - - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("lookup", - goka.Input("set", new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.SetValue(msg) - }), - goka.Persist(new(codec.String)), - ), - goka.WithTester(gkt), - ) - - // add a lookup table - lookupProc, err := goka.NewProcessor([]string{}, goka.DefineGroup("group", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - val := ctx.Lookup("lookup-table", "somekey").(string) - if val != "42" { - ctx.Fail(fmt.Errorf("lookup value was unexpected")) - } - }), - goka.Lookup("lookup-table", new(codec.String)), - ), - goka.WithTester(gkt), - ) - - if err != nil { - t.Fatalf("Error creating processor: %v", err) - } - runProcOrFail(proc) - runProcOrFail(lookupProc) - gkt.Consume("set", "somekey", "42") - gkt.Consume("input", "sender", "message") -} - -func Test_Join(t *testing.T) { - - gkt := New(t) - - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("join", - goka.Input("set", new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.SetValue(msg) - }), - goka.Persist(new(codec.String)), - ), - goka.WithTester(gkt), - ) - - // add a lookup table - lookupProc, err := goka.NewProcessor([]string{}, goka.DefineGroup("group", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - val := ctx.Lookup("join-table", "somekey").(string) - if val != "42" { - ctx.Fail(fmt.Errorf("join value was unexpected")) - } - }), - goka.Lookup("join-table", new(codec.String)), - ), - goka.WithTester(gkt), - ) - - if err != nil { - t.Fatalf("Error creating processor: %v", err) - } - runProcOrFail(proc) - runProcOrFail(lookupProc) - gkt.Consume("set", "somekey", "42") - gkt.Consume("input", "sender", "message") -} - -func Test_QueueTracker_Default(t *testing.T) { - - gkt := New(t) - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("lookup", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.Emit("output", ctx.Key(), msg) - }), - goka.Output("output", new(codec.String)), - ), - goka.WithTester(gkt), - ) - runProcOrFail(proc) - mt := gkt.NewQueueTracker("output") - gkt.Consume("input", "somekey", "124") - key, value, hasNext := mt.Next() - if key != "somekey" || value.(string) != "124" || !hasNext { - t.Fatalf("next emitted was something unexpected (key=%s, value=%s, hasNext=%t)", key, value.(string), hasNext) - } - _, _, hasNext = mt.Next() - if hasNext { - t.Fatalf("got another emitted message which shouldn't be there") - } - - if mt.NextOffset() != mt.Hwm() { - t.Fatalf("expected queue to be at end but wasn't") - } - -} - -func Test_QueueTracker_Extra(t *testing.T) { - - gkt := New(t) - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("lookup", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.Emit("output", ctx.Key(), msg) - }), - goka.Output("output", new(codec.String)), - ), - goka.WithTester(gkt), - ) - runProcOrFail(proc) - gkt.Consume("input", "somekey", "123") - - tracker := gkt.NewQueueTracker("output") - - // the new message tracker should start at the end, so the already emitted message - // shouldn't appear - if tracker.NextOffset() != tracker.Hwm() { - t.Fatalf("expected queue to be at end but wasn't") - } - - gkt.Consume("input", "somekey", "124") - key, value, hasNext := tracker.Next() - if key != "somekey" || value.(string) != "124" || !hasNext { - t.Fatalf("next emitted was something unexpected (key=%s, value=%s, hasNext=%t)", key, value.(string), hasNext) - } -} - -func Test_Shutdown(t *testing.T) { - gkt := New(t) - proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("lookup", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - }), - ), - goka.WithTester(gkt), - ) - - ctx, cancel := context.WithCancel(context.Background()) - var ( - wg sync.WaitGroup - procErr error - ) - wg.Add(1) - go func() { - defer wg.Done() - procErr = proc.Run(ctx) - }() - - gkt.Consume("input", "test", "test") - - time.Sleep(10 * time.Millisecond) - cancel() - - gkt.Consume("input", "test", "test") - - wg.Wait() - if procErr != nil { - t.Fatalf("got error for shutting down processor: %v", procErr) - } -} - -func Test_LookupWithInitialData(t *testing.T) { - gkt := New(t) - - proc, _ := goka.NewProcessor([]string{}, - goka.DefineGroup("group", - goka.Inputs(goka.Streams{"input-a", "input-b"}, - new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.Loopback(ctx.Key(), "first-loop") - }), - goka.Loop(new(codec.String), func(ctx goka.Context, msg interface{}) { - if msg.(string) == "first-loop" { - ctx.Loopback(ctx.Key(), "second-loop") - } else { - lookupValue := ctx.Lookup("lookup-table", "somekey") - if lookupValue != nil { - ctx.SetValue(fmt.Sprintf("%d", lookupValue)) - } - ctx.Emit("output", ctx.Key(), msg) - } - }), - goka.Output("output", new(codec.String)), - goka.Lookup("lookup-table", new(codec.Int64)), - goka.Persist(new(codec.String)), - ), - goka.WithTester(gkt), - ) - - go proc.Run(context.Background()) - gkt.Consume("lookup-table", "somekey", int64(123)) - - // regression test: this used to block - gkt.Consume("input-a", "key", "value") -} - -func Test_MultiLookup(t *testing.T) { - gkt := New(t) - proc, _ := goka.NewProcessor([]string{}, - goka.DefineGroup("group", - goka.Inputs(goka.Streams{"input"}, - new(codec.String), func(ctx goka.Context, msg interface{}) { - ctx.SetValue(msg) - }), - goka.Persist(new(codec.String)), - ), - goka.WithTester(gkt), - ) - - var foundValue int - - lookup1, _ := goka.NewProcessor([]string{}, - goka.DefineGroup("lookup1", - goka.Inputs(goka.Streams{"trigger"}, - new(codec.String), func(ctx goka.Context, msg interface{}) { - lookupValue := ctx.Lookup("group-table", ctx.Key()) - if lookupValue.(string) != msg.(string) { - t.Fatalf("expected %s, got %s", msg, lookupValue) - } else { - foundValue++ - } - }), - goka.Lookup("group-table", new(codec.String)), - ), - goka.WithTester(gkt), - ) - lookup2, _ := goka.NewProcessor([]string{}, - goka.DefineGroup("lookup2", - goka.Inputs(goka.Streams{"trigger"}, - new(codec.String), func(ctx goka.Context, msg interface{}) { - lookupValue := ctx.Lookup("group-table", ctx.Key()) - if lookupValue.(string) != msg.(string) { - t.Fatalf("expected %s, got %s", msg, lookupValue) - } else { - foundValue++ - } - }), - goka.Lookup("group-table", new(codec.String)), - ), - goka.WithTester(gkt), - ) - - go proc.Run(context.Background()) - go lookup1.Run(context.Background()) - go lookup2.Run(context.Background()) - - // set the lookup table value - gkt.Consume("input", "value-from-input", "43") - gkt.Consume("trigger", "value-from-input", "43") - if foundValue != 2 { - t.Fatalf("did not find value in lookup table") - } - - foundValue = 0 - - gkt.SetTableValue("group-table", "set-in-table", "44") - gkt.Consume("trigger", "set-in-table", "44") - if foundValue != 2 { - t.Fatalf("did not find value in lookup table") - } -} - -func Test_ManyConsume(t *testing.T) { - var inputs goka.Streams - for i := 0; i < 100; i++ { - inputs = append(inputs, goka.Stream(fmt.Sprintf("input-%d", i))) - } - - received := map[string]bool{} - - gkt := New(t) - proc, _ := goka.NewProcessor([]string{}, - goka.DefineGroup("group", - goka.Inputs(inputs, new(codec.String), func(ctx goka.Context, msg interface{}) { received[string(ctx.Topic())] = true }), - ), - goka.WithTester(gkt), - ) - go proc.Run(context.Background()) - - // we'll just try to get something consumed - for i := 0; i < 100; i++ { - gkt.Consume(fmt.Sprintf("input-%d", i), "something", "something") - } - if len(received) != 100 { - t.Fatalf("did not receive all messages") - } -} - -func TestEmitterStandalone(t *testing.T) { - gkt := New(t) - - em, _ := goka.NewEmitter(nil, "test", new(codec.String), goka.WithEmitterTester(gkt)) - est := gkt.NewQueueTracker("test") - - em.Emit("key", "value") - - _, _, ok := est.Next() - if !ok { - t.Errorf("No message emitted") - } -} - -// Tests an emitter used inside a processor. -// For this the user has to start the emitter with -// a separate tester, otherwise it will deadlock. -func TestEmitterInProcessor(t *testing.T) { - - gktProcessor := New(t) - - // create a new emitter mocked with an extra tester - gktEmitter := New(t) - em, _ := goka.NewEmitter(nil, "output", new(codec.String), goka.WithEmitterTester(gktEmitter)) - - // create a new processor that uses the emitter internally - proc, err := goka.NewProcessor(nil, goka.DefineGroup("test-group", - goka.Input("input", new(codec.String), func(ctx goka.Context, msg interface{}) { - log.Printf("sending") - - prom, err := em.Emit(ctx.Key(), msg) - log.Printf("sending done") - if err != nil { - t.Errorf("Error emitting in processor: %v", err) - } - prom.Then(func(err error) { - if err != nil { - t.Errorf("Error emitting in processor (in promise): %v", err) - } - }) - log.Printf("done") - }), - ), - goka.WithTester(gktProcessor), - ) - - if err != nil { - log.Fatalf("Error creating processor: %v", err) - } - runProcOrFail(proc) - - // create a queue tracker from the extra tester - est := gktEmitter.NewQueueTracker("output") - - // consume a message - gktProcessor.Consume("input", "key", "value") - - // ensure the message is there - _, _, ok := est.Next() - if !ok { - t.Errorf("No message emitted") - } -} diff --git a/tester/topic_manager.go b/tester/topic_manager.go new file mode 100644 index 00000000..af448281 --- /dev/null +++ b/tester/topic_manager.go @@ -0,0 +1,74 @@ +package tester + +import ( + "fmt" + + "github.com/Shopify/sarama" +) + +// MockTopicManager mimicks the behavior of the real topic manager +type MockTopicManager struct { + DefaultNumPartitions int + DefaultReplicationFactor int + tt *Tester +} + +// NewMockTopicManager creates a new topic manager mock +func NewMockTopicManager(tt *Tester, defaultNumPartitions int, defaultReplFactor int) *MockTopicManager { + return &MockTopicManager{ + tt: tt, + DefaultNumPartitions: defaultNumPartitions, + DefaultReplicationFactor: defaultReplFactor, + } +} + +// EnsureTableExists ensures a table exists +func (tm *MockTopicManager) EnsureTableExists(topic string, npar int) error { + if npar != 1 { + return fmt.Errorf("Mock only supports 1 partition") + } + tm.tt.getOrCreateQueue(topic) + return nil +} + +// EnsureStreamExists ensures a stream exists +func (tm *MockTopicManager) EnsureStreamExists(topic string, npar int) error { + if npar != 1 { + return fmt.Errorf("Mock only supports 1 partition") + } + tm.tt.getOrCreateQueue(topic) + return nil +} + +// EnsureTopicExists ensures a topic exists +func (tm *MockTopicManager) EnsureTopicExists(topic string, npar, rfactor int, config map[string]string) error { + if npar != 1 { + return fmt.Errorf("Mock only supports 1 partition") + } + tm.tt.getOrCreateQueue(topic) + return nil +} + +// Partitions returns all partitions for a topic +func (tm *MockTopicManager) Partitions(topic string) ([]int32, error) { + return []int32{0}, nil +} + +// GetOffset returns the offset closest to the passed time (or exactly time, if the offsets are empty) +func (tm *MockTopicManager) GetOffset(topicName string, partitionID int32, time int64) (int64, error) { + topic := tm.tt.getOrCreateQueue(topicName) + + switch time { + case sarama.OffsetOldest: + return 0, nil + case sarama.OffsetNewest: + return topic.Hwm(), nil + default: + return 0, fmt.Errorf("only oldest and newest are supported in the mock") + } +} + +// Close has no action on the mock +func (tm *MockTopicManager) Close() error { + return nil +} diff --git a/topic_manager.go b/topic_manager.go new file mode 100644 index 00000000..1a6ed4fa --- /dev/null +++ b/topic_manager.go @@ -0,0 +1,222 @@ +package goka + +import ( + "errors" + "fmt" + "strings" + "time" + + "github.com/Shopify/sarama" + "github.com/lovoo/goka/multierr" +) + +// TopicManager provides an interface to create/check topics and their partitions +type TopicManager interface { + // EnsureTableExists checks that a table (log-compacted topic) exists, or create one if possible + EnsureTableExists(topic string, npar int) error + // EnsureStreamExists checks that a stream topic exists, or create one if possible + EnsureStreamExists(topic string, npar int) error + // EnsureTopicExists checks that a topic exists, or create one if possible, + // enforcing the given configuration + EnsureTopicExists(topic string, npar, rfactor int, config map[string]string) error + + // Partitions returns the number of partitions of a topic, that are assigned to the running + // instance, i.e. it doesn't represent all partitions of a topic. + Partitions(topic string) ([]int32, error) + + GetOffset(topic string, partitionID int32, time int64) (int64, error) + + // Close closes the topic manager + Close() error +} + +type topicManager struct { + brokers []string + broker Broker + client sarama.Client + topicManagerConfig *TopicManagerConfig +} + +// NewTopicManager creates a new topic manager using the sarama library +func NewTopicManager(brokers []string, saramaConfig *sarama.Config, topicManagerConfig *TopicManagerConfig) (TopicManager, error) { + client, err := sarama.NewClient(brokers, saramaConfig) + if err != nil { + return nil, fmt.Errorf("Error creating the kafka client: %v", err) + } + return newTopicManager(brokers, saramaConfig, topicManagerConfig, client, checkBroker) +} + +func newTopicManager(brokers []string, saramaConfig *sarama.Config, topicManagerConfig *TopicManagerConfig, client sarama.Client, check checkFunc) (*topicManager, error) { + if client == nil { + return nil, errors.New("cannot create topic manager with nil client") + } + + if topicManagerConfig == nil { + return nil, errors.New("cannot create topic manager with nil config") + } + + activeBrokers := client.Brokers() + if len(activeBrokers) == 0 { + return nil, errors.New("no brokers active in current client") + } + + broker := activeBrokers[0] + err := check(broker, saramaConfig) + if err != nil { + return nil, err + } + + return &topicManager{ + brokers: brokers, + client: client, + broker: broker, + topicManagerConfig: topicManagerConfig, + }, nil +} + +type checkFunc func(broker Broker, config *sarama.Config) error + +func checkBroker(broker Broker, config *sarama.Config) error { + if config == nil { + config = DefaultConfig() + } + + err := broker.Open(config) + if err != nil { + return fmt.Errorf("error opening broker connection: %v", err) + } + connected, err := broker.Connected() + if err != nil { + return fmt.Errorf("cannot connect to broker %s: %v", broker.Addr(), err) + } + + if !connected { + return fmt.Errorf("cannot connect to broker %s: not connected", broker.Addr()) + } + + return nil +} + +func (m *topicManager) Close() error { + errs := new(multierr.Errors) + errs.Collect(m.client.Close()) + + return errs.NilOrError() +} + +func (m *topicManager) Partitions(topic string) ([]int32, error) { + return m.client.Partitions(topic) +} + +func (m *topicManager) GetOffset(topic string, partitionID int32, time int64) (int64, error) { + return m.client.GetOffset(topic, partitionID, time) +} + +func (m *topicManager) checkTopicExistsWithPartitions(topic string, npar int) (bool, error) { + par, err := m.client.Partitions(topic) + if err != nil { + if err == sarama.ErrUnknownTopicOrPartition { + return false, nil + } + return false, fmt.Errorf("Error checking partitions for topic %s: %v", topic, err) + } + if len(par) != npar { + return false, fmt.Errorf("topic %s has %d partitions instead of %d", topic, len(par), npar) + } + return true, nil +} + +func (m *topicManager) createTopic(topic string, npar, rfactor int, config map[string]string) error { + topicDetail := &sarama.TopicDetail{} + topicDetail.NumPartitions = int32(npar) + topicDetail.ReplicationFactor = int16(rfactor) + topicDetail.ConfigEntries = make(map[string]*string) + + for k, v := range config { + topicDetail.ConfigEntries[k] = &v + } + + topicDetails := make(map[string]*sarama.TopicDetail) + topicDetails[topic] = topicDetail + + request := sarama.CreateTopicsRequest{ + Timeout: time.Second * 15, + TopicDetails: topicDetails, + } + response, err := m.broker.CreateTopics(&request) + + if err != nil { + var errs []string + for k, topicErr := range response.TopicErrors { + errs = append(errs, fmt.Sprintf("%s: %s (%v)", k, topicErr.Err.Error(), topicErr.ErrMsg)) + } + return fmt.Errorf("error creating topic %s, npar=%d, rfactor=%d, config=%#v: %v\ntopic errors:\n%s", + topic, npar, rfactor, config, err, strings.Join(errs, "\n")) + } + + return nil +} + +func (m *topicManager) ensureExists(topic string, npar, rfactor int, config map[string]string) error { + exists, err := m.checkTopicExistsWithPartitions(topic, npar) + if err != nil { + return fmt.Errorf("error checking topic exists: %v", err) + } + if exists { + return nil + } + return m.createTopic(topic, + npar, + rfactor, + config) +} + +func (m *topicManager) EnsureStreamExists(topic string, npar int) error { + return m.ensureExists( + topic, + npar, + m.topicManagerConfig.Stream.Replication, + map[string]string{ + "retention.ms": fmt.Sprintf("%d", m.topicManagerConfig.Stream.Retention), + }) +} + +func (m *topicManager) EnsureTopicExists(topic string, npar, rfactor int, config map[string]string) error { + return m.ensureExists( + topic, + npar, + rfactor, + config) +} + +func (m *topicManager) EnsureTableExists(topic string, npar int) error { + return m.ensureExists( + topic, + npar, + m.topicManagerConfig.Table.Replication, + map[string]string{ + "cleanup.policy": "compact", + }) +} + +// TopicManagerConfig contains the configuration to access the Zookeeper servers +// as well as the desired options of to create tables and stream topics. +type TopicManagerConfig struct { + Table struct { + Replication int + } + Stream struct { + Replication int + Retention time.Duration + } +} + +// NewTopicManagerConfig provides a default configuration for auto-creation +// with replication factor of 1 and rentention time of 1 hour. +func NewTopicManagerConfig() *TopicManagerConfig { + cfg := new(TopicManagerConfig) + cfg.Table.Replication = 2 + cfg.Stream.Replication = 2 + cfg.Stream.Retention = 1 * time.Hour + return cfg +} diff --git a/topic_manager_test.go b/topic_manager_test.go new file mode 100644 index 00000000..4418ef79 --- /dev/null +++ b/topic_manager_test.go @@ -0,0 +1,404 @@ +package goka + +import ( + "errors" + "testing" + "time" + + "github.com/Shopify/sarama" + "github.com/golang/mock/gomock" + "github.com/lovoo/goka/internal/test" +) + +var ( + tmTestBrokers []string = []string{"0"} +) + +func trueCheckFunc(broker Broker, config *sarama.Config) error { + return nil +} + +func falseCheckFunc(broker Broker, config *sarama.Config) error { + return errors.New("broker check error") +} + +func createTopicManager(t *testing.T) (*topicManager, *builderMock, *gomock.Controller) { + ctrl := NewMockController(t) + bm := newBuilderMock(ctrl) + return &topicManager{ + brokers: tmTestBrokers, + broker: bm.broker, + client: bm.client, + topicManagerConfig: NewTopicManagerConfig(), + }, bm, ctrl +} + +func TestTM_checkBroker(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + ctrl := NewMockController(t) + broker := NewMockBroker(ctrl) + defer ctrl.Finish() + + var ( + config *sarama.Config = DefaultConfig() + connected bool = true + ) + broker.EXPECT().Open(config).Return(nil) + broker.EXPECT().Connected().Return(connected, nil) + + err := checkBroker(broker, config) + test.AssertNil(t, err) + }) + t.Run("fail_open", func(t *testing.T) { + ctrl := NewMockController(t) + broker := NewMockBroker(ctrl) + defer ctrl.Finish() + + var ( + config *sarama.Config = DefaultConfig() + errRet error = errors.New("some-error") + ) + broker.EXPECT().Open(config).Return(errRet) + + err := checkBroker(broker, config) + test.AssertNotNil(t, err) + }) + t.Run("fail_connected", func(t *testing.T) { + ctrl := NewMockController(t) + broker := NewMockBroker(ctrl) + defer ctrl.Finish() + + var ( + config *sarama.Config = DefaultConfig() + connected bool = false + ) + broker.EXPECT().Open(config).Return(nil) + broker.EXPECT().Connected().Return(connected, nil) + broker.EXPECT().Addr().Return("127.0.0.1") + + err := checkBroker(broker, config) + test.AssertNotNil(t, err) + }) + t.Run("fail_not_connected", func(t *testing.T) { + ctrl := NewMockController(t) + broker := NewMockBroker(ctrl) + defer ctrl.Finish() + + var ( + config *sarama.Config = DefaultConfig() + connected bool = false + errRet error = errors.New("some-error") + ) + broker.EXPECT().Open(config).Return(nil) + broker.EXPECT().Connected().Return(connected, errRet) + broker.EXPECT().Addr().Return("127.0.0.1") + + err := checkBroker(broker, config) + test.AssertNotNil(t, err) + }) +} + +func TestTM_newTopicManager(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + ctrl := NewMockController(t) + defer ctrl.Finish() + bm := newBuilderMock(ctrl) + + var ( + broker *sarama.Broker = &sarama.Broker{} + ) + + bm.client.EXPECT().Brokers().Return([]*sarama.Broker{ + broker, + }) + + tm, err := newTopicManager(tmTestBrokers, DefaultConfig(), NewTopicManagerConfig(), bm.client, trueCheckFunc) + test.AssertNil(t, err) + test.AssertEqual(t, tm.brokers, tmTestBrokers) + test.AssertEqual(t, tm.client, bm.client) + test.AssertEqual(t, tm.broker, broker) + }) + t.Run("fail_missing_stuff", func(t *testing.T) { + ctrl := NewMockController(t) + defer ctrl.Finish() + bm := newBuilderMock(ctrl) + + _, err := newTopicManager(tmTestBrokers, nil, nil, bm.client, trueCheckFunc) + test.AssertNotNil(t, err) + + _, err = newTopicManager(tmTestBrokers, nil, NewTopicManagerConfig(), nil, trueCheckFunc) + test.AssertNotNil(t, err) + }) + t.Run("fail_check", func(t *testing.T) { + ctrl := NewMockController(t) + defer ctrl.Finish() + bm := newBuilderMock(ctrl) + + var ( + broker *sarama.Broker = &sarama.Broker{} + ) + + bm.client.EXPECT().Brokers().Return([]*sarama.Broker{ + broker, + }) + + _, err := newTopicManager(tmTestBrokers, DefaultConfig(), NewTopicManagerConfig(), bm.client, falseCheckFunc) + test.AssertNotNil(t, err) + }) +} + +func TestTM_Close(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + bm.client.EXPECT().Close().Return(nil) + err := tm.Close() + test.AssertNil(t, err) + }) + t.Run("fail", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + bm.client.EXPECT().Close().Return(errors.New("some-error")) + err := tm.Close() + test.AssertNotNil(t, err) + }) +} + +func TestTM_Partitions(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + ) + bm.client.EXPECT().Partitions(topic).Return([]int32{0}, nil) + _, err := tm.Partitions(topic) + test.AssertNil(t, err) + }) + t.Run("fail", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + ) + bm.client.EXPECT().Partitions(topic).Return([]int32{0}, errors.New("some-error")) + _, err := tm.Partitions(topic) + test.AssertNotNil(t, err) + }) +} + +func TestTM_GetOffset(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + partition int32 = 0 + offset int64 = sarama.OffsetNewest + ) + bm.client.EXPECT().GetOffset(topic, partition, offset).Return(sarama.OffsetNewest, nil) + _, err := tm.GetOffset(topic, partition, offset) + test.AssertNil(t, err) + }) + t.Run("fail", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + partition int32 = 0 + offset int64 = sarama.OffsetNewest + ) + bm.client.EXPECT().GetOffset(topic, partition, offset).Return(sarama.OffsetNewest, errors.New("some-error")) + _, err := tm.GetOffset(topic, partition, offset) + test.AssertNotNil(t, err) + }) +} + +func TestTM_checkTopicExistsWithPartitions(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + ) + bm.client.EXPECT().Partitions(topic).Return([]int32{0}, nil) + correct, err := tm.checkTopicExistsWithPartitions(topic, npar) + test.AssertNil(t, err) + test.AssertTrue(t, correct) + }) + t.Run("unknown", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + ) + bm.client.EXPECT().Partitions(topic).Return(nil, sarama.ErrUnknownTopicOrPartition) + correct, err := tm.checkTopicExistsWithPartitions(topic, npar) + test.AssertNil(t, err) + test.AssertTrue(t, !correct) + }) + t.Run("fail", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + falseNPar int = 2 + ) + bm.client.EXPECT().Partitions(topic).Return(nil, errors.New("some-error")) + correct, err := tm.checkTopicExistsWithPartitions(topic, npar) + test.AssertNotNil(t, err) + test.AssertTrue(t, !correct) + bm.client.EXPECT().Partitions(topic).Return([]int32{0}, nil) + correct, err = tm.checkTopicExistsWithPartitions(topic, falseNPar) + test.AssertNotNil(t, err) + test.AssertTrue(t, !correct) + }) +} + +func TestTM_EnsureStreamExists(t *testing.T) { + t.Run("exists", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + ) + + bm.client.EXPECT().Partitions(topic).Return([]int32{0}, nil) + + err := tm.EnsureStreamExists(topic, npar) + test.AssertNil(t, err) + }) + t.Run("create", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + rfactor int = 1 + ) + + tm.topicManagerConfig.Stream.Replication = rfactor + tm.topicManagerConfig.Stream.Retention = time.Second + bm.client.EXPECT().Partitions(topic).Return(nil, sarama.ErrUnknownTopicOrPartition) + bm.broker.EXPECT().CreateTopics(gomock.Any()).Return(nil, nil) + + err := tm.EnsureStreamExists(topic, npar) + test.AssertNil(t, err) + }) + t.Run("fail", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + retErr error = errors.New("some-error") + ) + + bm.client.EXPECT().Partitions(topic).Return([]int32{0}, retErr) + + err := tm.EnsureStreamExists(topic, npar) + test.AssertNotNil(t, err) + }) +} + +func TestTM_createTopic(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + rfactor int = 1 + config map[string]string = map[string]string{ + "a": "a", + } + ) + bm.broker.EXPECT().CreateTopics(gomock.Any()).Return(nil, nil) + err := tm.createTopic(topic, npar, rfactor, config) + test.AssertNil(t, err) + }) + t.Run("fail", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + rfactor int = 1 + config map[string]string = map[string]string{ + "a": "a", + } + retErr error = errors.New("some-error") + errMsg string = "some-error-msg" + ) + bm.broker.EXPECT().CreateTopics(gomock.Any()).Return(&sarama.CreateTopicsResponse{ + TopicErrors: map[string]*sarama.TopicError{ + "a": &sarama.TopicError{ + Err: sarama.KError(0), + ErrMsg: &errMsg, + }, + }, + }, retErr) + err := tm.createTopic(topic, npar, rfactor, config) + test.AssertNotNil(t, err) + }) +} + +func TestTM_EnsureTopicExists(t *testing.T) { + t.Run("exists", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + rfactor int = 1 + config map[string]string = map[string]string{ + "a": "a", + } + ) + + bm.client.EXPECT().Partitions(topic).Return([]int32{0}, nil) + + err := tm.EnsureTopicExists(topic, npar, rfactor, config) + test.AssertNil(t, err) + }) + t.Run("create", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + rfactor int = 1 + config map[string]string = map[string]string{ + "a": "a", + } + ) + + bm.client.EXPECT().Partitions(topic).Return(nil, sarama.ErrUnknownTopicOrPartition) + bm.broker.EXPECT().CreateTopics(gomock.Any()).Return(nil, nil) + + err := tm.EnsureTopicExists(topic, npar, rfactor, config) + test.AssertNil(t, err) + }) + t.Run("fail", func(t *testing.T) { + tm, bm, ctrl := createTopicManager(t) + defer ctrl.Finish() + var ( + topic string = "some-topic" + npar int = 1 + rfactor int = 1 + config map[string]string = map[string]string{ + "a": "a", + } + retErr error = errors.New("some-error") + ) + + bm.client.EXPECT().Partitions(topic).Return([]int32{0}, retErr) + + err := tm.EnsureTopicExists(topic, npar, rfactor, config) + test.AssertNotNil(t, err) + }) +} diff --git a/view.go b/view.go index f4d9e892..4f2bbcde 100644 --- a/view.go +++ b/view.go @@ -6,12 +6,21 @@ import ( "fmt" "sync" - "github.com/lovoo/goka/kafka" + "github.com/Shopify/sarama" "github.com/lovoo/goka/logger" "github.com/lovoo/goka/multierr" "github.com/lovoo/goka/storage" ) +const ( + // ViewStateIdle - the view is not started yet + ViewStateIdle State = iota + // ViewStateCatchUp - the view is still catching up + ViewStateCatchUp + // ViewStateRunning - the view has caught up and is running + ViewStateRunning +) + // Getter functions return a value for a key or an error. If no value exists for the key, nil is returned without errors. type Getter func(string) (interface{}, error) @@ -20,9 +29,11 @@ type View struct { brokers []string topic string opts *voptions - partitions []*partition - consumer kafka.Consumer - terminated bool + log logger.Logger + partitions []*PartitionTable + consumer sarama.Consumer + tmgr TopicManager + state *Signal } // NewView creates a new View object from a group. @@ -30,9 +41,9 @@ func NewView(brokers []string, topic Table, codec Codec, options ...ViewOption) options = append( // default options comes first []ViewOption{ + WithViewClientID(fmt.Sprintf("goka-view-%s", topic)), WithViewLogger(logger.Default()), WithViewCallback(DefaultUpdate), - WithViewPartitionChannelSize(defaultPartitionChannelSize), WithViewStorageBuilder(storage.DefaultBuilder(DefaultViewStoragePath())), }, @@ -40,19 +51,31 @@ func NewView(brokers []string, topic Table, codec Codec, options ...ViewOption) options..., ) - // figure out how many partitions the group has opts := new(voptions) err := opts.applyOptions(topic, codec, options...) if err != nil { return nil, fmt.Errorf("Error applying user-defined options: %v", err) } + consumer, err := opts.builders.consumerSarama(brokers, opts.clientID) + if err != nil { + return nil, fmt.Errorf("Error creating sarama consumer for brokers %+v: %v", brokers, err) + } opts.tableCodec = codec + tmgr, err := opts.builders.topicmgr(brokers) + if err != nil { + return nil, fmt.Errorf("Error creating topic manager: %v", err) + } + v := &View{ - brokers: brokers, - topic: string(topic), - opts: opts, + brokers: brokers, + topic: string(topic), + opts: opts, + log: opts.log.Prefix(fmt.Sprintf("View %s", topic)), + consumer: consumer, + tmgr: tmgr, + state: NewSignal(ViewStateIdle, ViewStateCatchUp, ViewStateRunning).SetState(ViewStateIdle), } if err = v.createPartitions(brokers); err != nil { @@ -62,6 +85,11 @@ func NewView(brokers []string, topic Table, codec Codec, options ...ViewOption) return v, err } +// WaitRunning returns a channel that will be closed when the view enters the running state +func (v *View) WaitRunning() <-chan struct{} { + return v.state.WaitForState(ViewStateRunning) +} + func (v *View) createPartitions(brokers []string) (rerr error) { tm, err := v.opts.builders.topicmgr(brokers) if err != nil { @@ -86,115 +114,79 @@ func (v *View) createPartitions(brokers []string) (rerr error) { } } - v.opts.log.Printf("Table %s has %d partitions", v.topic, len(partitions)) - for _, p := range partitions { - st, err := v.opts.builders.storage(v.topic, p) - if err != nil { - // TODO(diogo): gracefully terminate all partitions - return fmt.Errorf("Error creating local storage for partition %d: %v", p, err) - } - - po := newPartition(v.opts.log, v.topic, nil, - &storageProxy{Storage: st, partition: p, update: v.opts.updateCallback}, - &proxy{p, nil}, - v.opts.partitionChannelSize, - ) - v.partitions = append(v.partitions, po) + for partID, p := range partitions { + v.partitions = append(v.partitions, newPartitionTable(v.topic, + p, + v.consumer, + v.tmgr, + v.opts.updateCallback, + v.opts.builders.storage, + v.log.Prefix(fmt.Sprintf("PartTable-%d", partID)), + )) } return nil } -// reinit (re)initializes the view and its partitions to connect to Kafka -func (v *View) reinit() error { - if v.terminated { - return fmt.Errorf("view: cannot reinitialize terminated view") - } +// Run starts consuming the view's topic. +func (v *View) Run(ctx context.Context) (rerr error) { + v.log.Debugf("starting") + defer v.log.Debugf("stopped") - consumer, err := v.opts.builders.consumer(v.brokers, "goka-view", v.opts.clientID) - if err != nil { - return fmt.Errorf("view: cannot create Kafka consumer: %v", err) - } - v.consumer = consumer + v.state.SetState(ViewStateCatchUp) + defer v.state.SetState(ViewStateIdle) - for i, p := range v.partitions { - p.reinit(&proxy{int32(i), v.consumer}) - } - return nil -} + // close the view after running + defer func() { + errs := new(multierr.Errors) + errs.Collect(rerr) + errs.Collect(v.close()) + rerr = errs.NilOrError() + }() -// Run starts consuming the view's topic. -func (v *View) Run(ctx context.Context) error { - v.opts.log.Printf("view [%s]: starting", v.Topic()) - defer v.opts.log.Printf("view [%s]: stopped", v.Topic()) + recoverErrg, recoverCtx := multierr.NewErrGroup(ctx) - if err := v.reinit(); err != nil { - return err + for _, partition := range v.partitions { + partition := partition + recoverErrg.Go(func() error { + return partition.SetupAndRecover(recoverCtx) + }) } - errg, ctx := multierr.NewErrGroup(ctx) - errg.Go(func() error { return v.run(ctx) }) - - for id, p := range v.partitions { - pid, par := int32(id), p - errg.Go(func() error { - v.opts.log.Printf("view [%s]: partition %d started", v.Topic(), pid) - defer v.opts.log.Printf("view [%s]: partition %d stopped", v.Topic(), pid) - if err := par.st.Open(); err != nil { - return fmt.Errorf("view [%s]: error opening storage partition %d: %v", v.Topic(), pid, err) - } - if err := par.startCatchup(ctx); err != nil { - return fmt.Errorf("view [%s]: error running partition %d: %v", v.Topic(), pid, err) - } - return nil - }) + err := recoverErrg.Wait().NilOrError() + if err != nil { + rerr = fmt.Errorf("Error recovering partitions for view %s: %v", v.Topic(), err) + return } + v.state.SetState(ViewStateRunning) - // wait for partition goroutines and shutdown - errs := errg.Wait() + catchupErrg, catchupCtx := multierr.NewErrGroup(ctx) - v.opts.log.Printf("view [%s]: closing consumer", v.Topic()) - if err := v.consumer.Close(); err != nil { - _ = errs.Collect(fmt.Errorf("view [%s]: failed closing consumer: %v", v.Topic(), err)) + for _, partition := range v.partitions { + partition := partition + catchupErrg.Go(func() error { + return partition.CatchupForever(catchupCtx, v.opts.restartable) + }) } - if !v.opts.restartable { - v.terminated = true - errs = errs.Merge(v.close()) + err = catchupErrg.Wait().NilOrError() + if err != nil { + rerr = fmt.Errorf("Error catching up partitions for view %s: %v", v.Topic(), err) } - - return errs.NilOrError() + return } // close closes all storage partitions -func (v *View) close() *multierr.Errors { - errs := new(multierr.Errors) +func (v *View) close() error { + errg, _ := multierr.NewErrGroup(context.Background()) for _, p := range v.partitions { - _ = errs.Collect(p.st.Close()) + p := p + errg.Go(func() error { + return p.Close() + }) } v.partitions = nil - return errs -} - -// Terminate closes storage partitions. It must be called only if the view is -// restartable (see WithViewRestartable() option). Once Terminate() is called, -// the view cannot be restarted anymore. -func (v *View) Terminate() error { - if !v.opts.restartable { - return nil - } - v.opts.log.Printf("View: closing") - - // do not allow any reinitialization - if v.terminated { - return nil - } - v.terminated = true - - if v.opts.restartable { - return v.close().NilOrError() - } - return nil + return errg.Wait().NilOrError() } func (v *View) hash(key string) (int32, error) { @@ -326,54 +318,10 @@ func (v *View) Evict(key string) error { return s.Delete(key) } -func (v *View) run(ctx context.Context) error { - for { - select { - case ev := <-v.consumer.Events(): - switch ev := ev.(type) { - case *kafka.Message: - partition := v.partitions[int(ev.Partition)] - select { - case partition.ch <- ev: - case <-ctx.Done(): - return nil - } - case *kafka.BOF: - partition := v.partitions[int(ev.Partition)] - select { - case partition.ch <- ev: - case <-ctx.Done(): - return nil - } - case *kafka.EOF: - partition := v.partitions[int(ev.Partition)] - select { - case partition.ch <- ev: - case <-ctx.Done(): - return nil - } - case *kafka.NOP: - partition := v.partitions[int(ev.Partition)] - select { - case partition.ch <- ev: - case <-ctx.Done(): - return nil - } - case *kafka.Error: - return fmt.Errorf("view: error from kafka consumer: %v", ev) - default: - return fmt.Errorf("view: cannot handle %T = %v", ev, ev) - } - case <-ctx.Done(): - return nil - } - } -} - // Recovered returns true when the view has caught up with events from kafka. func (v *View) Recovered() bool { for _, p := range v.partitions { - if !p.recovered() { + if !p.IsRecovered() { return false } } @@ -382,27 +330,33 @@ func (v *View) Recovered() bool { } // Stats returns a set of performance metrics of the view. -func (v *View) Stats() *ViewStats { - return v.statsWithContext(context.Background()) +func (v *View) Stats(ctx context.Context) *ViewStats { + return v.statsWithContext(ctx) } func (v *View) statsWithContext(ctx context.Context) *ViewStats { var ( m sync.Mutex - wg sync.WaitGroup stats = newViewStats() ) + errg, ctx := multierr.NewErrGroup(ctx) + + for _, partTable := range v.partitions { + partTable := partTable - wg.Add(len(v.partitions)) - for i, p := range v.partitions { - go func(pid int32, par *partition) { - s := par.fetchStats(ctx) + errg.Go(func() error { + tableStats := partTable.fetchStats(ctx) m.Lock() - stats.Partitions[pid] = s - m.Unlock() - wg.Done() - }(int32(i), p) + defer m.Unlock() + + stats.Partitions[partTable.partition] = tableStats + return nil + }) + } + + err := errg.Wait().NilOrError() + if err != nil { + v.log.Printf("Error retrieving stats: %v", err) } - wg.Wait() return stats } diff --git a/view_test.go b/view_test.go index 67dcc940..c91a96a8 100644 --- a/view_test.go +++ b/view_test.go @@ -2,23 +2,24 @@ package goka import ( "context" - "errors" + "fmt" "hash" + "strconv" "testing" "time" + "github.com/Shopify/sarama" + "github.com/golang/mock/gomock" "github.com/lovoo/goka/codec" - "github.com/lovoo/goka/kafka" + "github.com/lovoo/goka/internal/test" "github.com/lovoo/goka/logger" - "github.com/lovoo/goka/mock" "github.com/lovoo/goka/storage" - - "github.com/facebookgo/ensure" - "github.com/golang/mock/gomock" ) var ( - recoveredMessages int + viewTestRecoveredMessages int + viewTestGroup Group = "group-name" + viewTestTopic = tableName(viewTestGroup) ) // constHasher implements a hasher that will always return the specified @@ -26,6 +27,7 @@ var ( // tests. type constHasher struct { partition uint32 + returnErr bool } func (ch *constHasher) Sum(b []byte) []byte { @@ -44,17 +46,26 @@ func (ch *constHasher) Reset() {} func (ch *constHasher) Size() int { return 4 } -func (ch *constHasher) Write(p []byte) (n int, err error) { +func (ch *constHasher) Write(p []byte) (int, error) { + if ch.returnErr { + return 0, fmt.Errorf("constHasher write error") + } return len(p), nil } -// NewConstHasher creates a constant hasher that hashes any value to 0. -func NewConstHasher(part uint32) hash.Hash32 { +func (ch *constHasher) ReturnError() { + ch.returnErr = true +} + +// newConstHasher creates a constant hasher that hashes any value to 0. +func newConstHasher(part uint32) *constHasher { return &constHasher{partition: part} } -func createTestView(t *testing.T, consumer kafka.Consumer, sb storage.Builder, tm kafka.TopicManager) *View { - recoveredMessages = 0 +func createTestView(t *testing.T, consumer sarama.Consumer) (*View, *builderMock, *gomock.Controller) { + ctrl := gomock.NewController(t) + bm := newBuilderMock(ctrl) + viewTestRecoveredMessages = 0 opts := &voptions{ log: logger.Default(), tableCodec: new(codec.String), @@ -62,482 +73,671 @@ func createTestView(t *testing.T, consumer kafka.Consumer, sb storage.Builder, t if err := DefaultUpdate(s, partition, key, value); err != nil { return err } - recoveredMessages++ + viewTestRecoveredMessages++ return nil }, hasher: DefaultHasher(), } - opts.builders.storage = sb - opts.builders.topicmgr = func(brokers []string) (kafka.TopicManager, error) { - return tm, nil - } - opts.builders.consumer = func(brokers []string, topic, id string) (kafka.Consumer, error) { + opts.builders.storage = bm.getStorageBuilder() + opts.builders.topicmgr = bm.getTopicManagerBuilder() + opts.builders.consumerSarama = func(brokers []string, clientID string) (sarama.Consumer, error) { return consumer, nil } - reader := &View{topic: tableName(group), opts: opts} - return reader + view := &View{topic: viewTestTopic, opts: opts, log: opts.log} + return view, bm, ctrl } -func TestView_createPartitions(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - var ( - consumer = mock.NewMockConsumer(ctrl) - st = mock.NewMockStorage(ctrl) - sb = func(topic string, partition int32) (storage.Storage, error) { - return st, nil - } - tm = mock.NewMockTopicManager(ctrl) - ) - - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0, 1}, nil) - tm.EXPECT().Close() - v := createTestView(t, consumer, sb, tm) - - err := v.createPartitions(nil) - ensure.Nil(t, err) - - tm.EXPECT().Partitions(tableName(group)).Return(nil, errors.New("some error")) - tm.EXPECT().Close() - v = createTestView(t, consumer, sb, tm) - err = v.createPartitions(nil) - ensure.NotNil(t, err) - - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0, 4}, nil) - tm.EXPECT().Close() - v = createTestView(t, consumer, sb, tm) - err = v.createPartitions(nil) - ensure.NotNil(t, err) - - sb = func(topic string, partition int32) (storage.Storage, error) { - return nil, errors.New("some error") - } - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0, 1}, nil) - tm.EXPECT().Close() - v = createTestView(t, consumer, sb, tm) - err = v.createPartitions(nil) - ensure.NotNil(t, err) +func TestView_hash(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() -} + view.partitions = []*PartitionTable{ + &PartitionTable{}, + } -func TestView_HasGet(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() + h, err := view.hash("a") + test.AssertNil(t, err) + test.AssertTrue(t, h == 0) + }) + t.Run("fail_hash", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() - var ( - st = mock.NewMockStorage(ctrl) - sb = func(topic string, partition int32) (storage.Storage, error) { - return st, nil + view.partitions = []*PartitionTable{ + &PartitionTable{}, + } + view.opts.hasher = func() hash.Hash32 { + hasher := newConstHasher(0) + hasher.ReturnError() + return hasher } - consumer = mock.NewMockConsumer(ctrl) - tm = mock.NewMockTopicManager(ctrl) - v = createTestView(t, consumer, sb, tm) - ) - - gomock.InOrder( - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0, 1, 2}, nil), - tm.EXPECT().Close(), - st.EXPECT().Has("item1").Return(false, nil), - st.EXPECT().Get("item1").Return([]byte("item1-value"), nil), - ) - - err := v.createPartitions(nil) - ensure.Nil(t, err) - - hasItem, err := v.Has("item1") - ensure.Nil(t, err) - ensure.False(t, hasItem) - - value, err := v.Get("item1") - ensure.Nil(t, err) - ensure.DeepEqual(t, value.(string), "item1-value") + + _, err := view.hash("a") + test.AssertNotNil(t, err) + }) + t.Run("fail_no_partition", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + _, err := view.hash("a") + test.AssertNotNil(t, err) + }) } -func TestView_StartStop(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() +func TestView_find(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + key string = "some-key" + proxy = &storageProxy{} + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + } + view.opts.hasher = func() hash.Hash32 { + hasher := newConstHasher(0) + return hasher + } - var ( - st = mock.NewMockStorage(ctrl) - sb = func(topic string, partition int32) (storage.Storage, error) { - return st, nil + st, err := view.find(key) + test.AssertNil(t, err) + test.AssertEqual(t, st, proxy) + }) + t.Run("fail", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + view.partitions = []*PartitionTable{ + &PartitionTable{}, + } + view.opts.hasher = func() hash.Hash32 { + hasher := newConstHasher(0) + hasher.ReturnError() + return hasher } - consumer = mock.NewMockConsumer(ctrl) - tm = mock.NewMockTopicManager(ctrl) - v = createTestView(t, consumer, sb, tm) - initial = make(chan bool) - final = make(chan bool) - ch = make(chan kafka.Event) - chClose = func() { close(ch) } - initialClose = func() { close(initial) } - - offset = int64(123) - par = int32(0) - ) - - gomock.InOrder( - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0}, nil), - tm.EXPECT().Close(), - consumer.EXPECT().Events().Do(initialClose).Return(ch), - ) - gomock.InOrder( - st.EXPECT().Open(), - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil), - consumer.EXPECT().AddPartition(tableName(group), int32(par), int64(offset)), - ) - gomock.InOrder( - consumer.EXPECT().RemovePartition(tableName(group), int32(par)), - consumer.EXPECT().Close().Do(chClose).Return(nil), - st.EXPECT().Close(), - ) - - err := v.createPartitions(nil) - ensure.Nil(t, err) - - ctx, cancel := context.WithCancel(context.Background()) - go func() { - errs := v.Run(ctx) - ensure.Nil(t, errs) - close(final) - }() - - err = doTimed(t, func() { - <-initial - cancel() - <-final + + _, err := view.find("a") + test.AssertNotNil(t, err) }) - ensure.Nil(t, err) } -func TestView_StartStopWithError(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() +func TestView_Get(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + proxy = &storageProxy{ + Storage: bm.mst, + partition: 0, + update: func(s storage.Storage, partition int32, key string, value []byte) error { + return nil + }, + } + key string = "some-key" + value int64 = 3 + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + } + view.opts.tableCodec = &codec.Int64{} + + bm.mst.EXPECT().Get(key).Return([]byte(strconv.FormatInt(value, 10)), nil) - var ( - st = mock.NewMockStorage(ctrl) - sb = func(topic string, partition int32) (storage.Storage, error) { - return st, nil + ret, err := view.Get(key) + test.AssertNil(t, err) + test.AssertTrue(t, ret == value) + }) + t.Run("succeed_nil", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + proxy = &storageProxy{ + Storage: bm.mst, + partition: 0, + update: func(s storage.Storage, partition int32, key string, value []byte) error { + return nil + }, + } + key string = "some-key" + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, } - consumer = mock.NewMockConsumer(ctrl) - tm = mock.NewMockTopicManager(ctrl) - v = createTestView(t, consumer, sb, tm) - final = make(chan bool) - ch = make(chan kafka.Event) - ) - - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0}, nil) - tm.EXPECT().Close() - err := v.createPartitions(nil) - ensure.Nil(t, err) - - consumer.EXPECT().Events().Return(ch) - st.EXPECT().Open() - st.EXPECT().GetOffset(int64(-2)).Return(int64(0), errors.New("some error1")) - st.EXPECT().Close() - consumer.EXPECT().Close().Return(errors.New("some error2")).Do(func() { close(ch) }) - - go func() { - viewErrs := v.Run(context.Background()) - ensure.StringContains(t, viewErrs.Error(), "error1") - ensure.StringContains(t, viewErrs.Error(), "error2") - close(final) - }() - - err = doTimed(t, func() { - <-final + view.opts.tableCodec = &codec.Int64{} + + bm.mst.EXPECT().Get(key).Return(nil, nil) + + ret, err := view.Get(key) + test.AssertNil(t, err) + test.AssertTrue(t, ret == nil) + }) + t.Run("fail_get", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + proxy = &storageProxy{ + Storage: bm.mst, + partition: 0, + update: func(s storage.Storage, partition int32, key string, value []byte) error { + return nil + }, + } + key string = "some-key" + errRet error = fmt.Errorf("get failed") + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + } + view.opts.tableCodec = &codec.Int64{} + + bm.mst.EXPECT().Get(key).Return(nil, errRet) + + _, err := view.Get(key) + test.AssertNotNil(t, err) }) - ensure.Nil(t, err) } -func TestView_RestartNonRestartable(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() +func TestView_Has(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() - var ( - st = mock.NewMockStorage(ctrl) - sb = func(topic string, partition int32) (storage.Storage, error) { - return st, nil + var ( + proxy = &storageProxy{ + Storage: bm.mst, + } + key string = "some-key" + has bool = true + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + } + view.opts.hasher = func() hash.Hash32 { + hasher := newConstHasher(0) + return hasher } - consumer = mock.NewMockConsumer(ctrl) - tm = mock.NewMockTopicManager(ctrl) - v = createTestView(t, consumer, sb, tm) - initial = make(chan bool) - final = make(chan bool) - ch = make(chan kafka.Event) - chClose = func() { close(ch) } - initialClose = func() { close(initial) } - - offset = int64(123) - par = int32(0) - ) - v.opts.restartable = false - - gomock.InOrder( - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0}, nil), - tm.EXPECT().Close(), - consumer.EXPECT().Events().Do(initialClose).Return(ch), - ) - gomock.InOrder( - st.EXPECT().Open(), - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil), - consumer.EXPECT().AddPartition(tableName(group), int32(par), int64(offset)), - ) - gomock.InOrder( - consumer.EXPECT().RemovePartition(tableName(group), int32(par)), - consumer.EXPECT().Close().Do(chClose).Return(nil), - st.EXPECT().Close(), - ) - - err := v.createPartitions(nil) - ensure.Nil(t, err) - - ctx, cancel := context.WithCancel(context.Background()) - go func() { - errs := v.Run(ctx) - ensure.Nil(t, errs) - close(final) - }() - - err = doTimed(t, func() { - <-initial - cancel() - <-final + + bm.mst.EXPECT().Has(key).Return(has, nil) + + ret, err := view.Has(key) + test.AssertNil(t, err) + test.AssertEqual(t, ret, has) }) - ensure.Nil(t, err) + t.Run("succeed_false", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() - // restart view - final = make(chan bool) + var ( + proxy = &storageProxy{ + Storage: bm.mst, + } + key string = "some-key" + has bool = false + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + } + view.opts.hasher = func() hash.Hash32 { + hasher := newConstHasher(0) + return hasher + } - go func() { - err = v.Run(context.Background()) - ensure.NotNil(t, err) - ensure.StringContains(t, err.Error(), "terminated") - close(final) - }() + bm.mst.EXPECT().Has(key).Return(has, nil) - err = doTimed(t, func() { - <-final + ret, err := view.Has(key) + test.AssertNil(t, err) + test.AssertEqual(t, ret, has) }) - ensure.Nil(t, err) + t.Run("fail_err", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + key string = "some-key" + has bool = false + ) + view.partitions = []*PartitionTable{ + &PartitionTable{}, + } + view.opts.hasher = func() hash.Hash32 { + hasher := newConstHasher(0) + hasher.ReturnError() + return hasher + } - err = v.Terminate() // silent - ensure.Nil(t, err) + ret, err := view.Has(key) + test.AssertNotNil(t, err) + test.AssertTrue(t, ret == has) + }) } -func TestView_Restart(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() +func TestView_Evict(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + key string = "some-key" + proxy = &storageProxy{ + Storage: bm.mst, + } + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + } + view.opts.hasher = func() hash.Hash32 { + hasher := newConstHasher(0) + return hasher + } + bm.mst.EXPECT().Delete(key).Return(nil) + + err := view.Evict(key) + test.AssertNil(t, err) + }) +} - var ( - st = mock.NewMockStorage(ctrl) - sb = func(topic string, partition int32) (storage.Storage, error) { - return st, nil +func TestView_Recovered(t *testing.T) { + t.Run("true", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + hasRecovered = true + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + state: NewSignal(State(PartitionRunning)).SetState(State(PartitionRunning)), + }, } - consumer = mock.NewMockConsumer(ctrl) - tm = mock.NewMockTopicManager(ctrl) - v = createTestView(t, consumer, sb, tm) - initial = make(chan bool) - final = make(chan bool) - ch = make(chan kafka.Event) - chClose = func() { close(ch) } - initialClose = func() { close(initial) } - - offset = int64(123) - par = int32(0) - ) - v.opts.restartable = true - - gomock.InOrder( - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0}, nil), - tm.EXPECT().Close(), - consumer.EXPECT().Events().Do(initialClose).Return(ch), - ) - gomock.InOrder( - st.EXPECT().Open(), - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil), - consumer.EXPECT().AddPartition(tableName(group), int32(par), int64(offset)), - ) - gomock.InOrder( - consumer.EXPECT().RemovePartition(tableName(group), int32(par)), - consumer.EXPECT().Close().Do(chClose).Return(nil), - ) - - err := v.createPartitions(nil) - ensure.Nil(t, err) - - ctx, cancel := context.WithCancel(context.Background()) - go func() { - errs := v.Run(ctx) - ensure.Nil(t, errs) - close(final) - }() - - err = doTimed(t, func() { - <-initial - cancel() - <-final + ret := view.Recovered() + test.AssertTrue(t, ret == hasRecovered) }) - ensure.Nil(t, err) - - // restart view - final = make(chan bool) - initial = make(chan bool, 3) - initialPush := func() { initial <- true } - ch = make(chan kafka.Event) - chClose = func() { close(ch) } - - // st.Open is not called because of openOnce in the storageProxy - st.EXPECT().GetOffset(int64(-2)).Return(int64(123), nil) - consumer.EXPECT().AddPartition(tableName(group), int32(0), int64(offset)) - consumer.EXPECT().Events().Return(ch) - consumer.EXPECT().RemovePartition(tableName(group), int32(0)) - consumer.EXPECT().Close().Do(chClose).Return(nil) - - _ = initialPush - ctx, cancel = context.WithCancel(context.Background()) - go func() { - err = v.Run(ctx) - ensure.Nil(t, err) - close(final) - }() - time.Sleep(2 * time.Second) - - err = doTimed(t, func() { - cancel() - <-final + t.Run("true", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + hasRecovered = false + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + state: NewSignal(State(PartitionRunning), State(PartitionRecovering)).SetState(State(PartitionRecovering)), + }, + &PartitionTable{ + state: NewSignal(State(PartitionRunning)).SetState(State(PartitionRunning)), + }, + } + ret := view.Recovered() + test.AssertTrue(t, ret == hasRecovered) }) - ensure.Nil(t, err) - - st.EXPECT().Close() - err = v.Terminate() - ensure.Nil(t, err) } -func TestView_GetErrors(t *testing.T) { - v := &View{opts: &voptions{hasher: DefaultHasher()}} - _, err := v.Get("hey") - ensure.NotNil(t, err) +func TestView_Topic(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() - _, err = v.Has("hey") - ensure.NotNil(t, err) + ret := view.Topic() + test.AssertTrue(t, ret == viewTestTopic) + }) +} - ctrl := gomock.NewController(t) - defer ctrl.Finish() +func TestView_close(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() - var ( - st = mock.NewMockStorage(ctrl) - sb = func(topic string, partition int32) (storage.Storage, error) { - return st, nil + var ( + proxy = &storageProxy{ + Storage: bm.mst, + } + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + &PartitionTable{ + st: proxy, + }, + &PartitionTable{ + st: proxy, + }, } - consumer = mock.NewMockConsumer(ctrl) - tm = mock.NewMockTopicManager(ctrl) - ) + bm.mst.EXPECT().Close().Return(nil).AnyTimes() - v = createTestView(t, consumer, sb, tm) + err := view.close() + test.AssertNil(t, err) + test.AssertTrue(t, len(view.partitions) == 0) + }) + t.Run("fail", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0}, nil) - tm.EXPECT().Close() - err = v.createPartitions(nil) - ensure.Nil(t, err) + var ( + proxy = &storageProxy{ + Storage: bm.mst, + } + retErr error = fmt.Errorf("some-error") + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + &PartitionTable{ + st: proxy, + }, + &PartitionTable{ + st: proxy, + }, + } + bm.mst.EXPECT().Close().Return(retErr).AnyTimes() - st.EXPECT().Get("hey").Return(nil, errors.New("some error")) - _, err = v.Get("hey") - ensure.NotNil(t, err) + err := view.close() + test.AssertNotNil(t, err) + test.AssertTrue(t, len(view.partitions) == 0) + }) } -func TestNewView(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - consumer = mock.NewMockConsumer(ctrl) - tm = mock.NewMockTopicManager(ctrl) - ) - _, err := NewView(nil, GroupTable(group), new(codec.Bytes), WithViewConsumerBuilder(createConsumerBuilder(nil))) - ensure.NotNil(t, err) - - gomock.InOrder( - tm.EXPECT().Partitions(tableName(group)).Return(nil, errors.New("some error")), - tm.EXPECT().Close(), - ) - _, err = NewView(nil, GroupTable(group), new(codec.Bytes), - WithViewConsumerBuilder(createConsumerBuilder(consumer)), - WithViewTopicManagerBuilder(createTopicManagerBuilder(tm))) - ensure.NotNil(t, err) - - gomock.InOrder( - tm.EXPECT().Partitions(tableName(group)).Return([]int32{0, 1, 2}, nil), - tm.EXPECT().Close(), - ) - v, err := NewView(nil, GroupTable(group), new(codec.Bytes), - WithViewConsumerBuilder(createConsumerBuilder(consumer)), - WithViewTopicManagerBuilder(createTopicManagerBuilder(tm))) - ensure.Nil(t, err) - ensure.DeepEqual(t, v.topic, tableName(group)) - ensure.DeepEqual(t, v.consumer, nil) // is set first upon start - ensure.True(t, len(v.partitions) == 3) -} +func TestView_Terminate(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() -func TestView_Evict(t *testing.T) { - key := "some-key" - val := "some-val" + var ( + proxy = &storageProxy{ + Storage: bm.mst, + } + isRestartable = true + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + &PartitionTable{ + st: proxy, + }, + &PartitionTable{ + st: proxy, + }, + } + view.opts.restartable = isRestartable + bm.mst.EXPECT().Close().Return(nil).AnyTimes() + + ret := view.close() + test.AssertNil(t, ret) + test.AssertTrue(t, len(view.partitions) == 0) + }) - st := storage.NewMemory() - err := st.Set(key, []byte(val)) - ensure.Nil(t, err) + t.Run("succeed_twice", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() - v := &View{ - partitions: []*partition{ - {st: &storageProxy{partition: 0, Storage: st}}, - }, - opts: &voptions{ - hasher: func() hash.Hash32 { - return NewConstHasher(0) + var ( + proxy = &storageProxy{ + Storage: bm.mst, + } + isRestartable = true + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, }, - tableCodec: new(codec.String), - }, - } + &PartitionTable{ + st: proxy, + }, + &PartitionTable{ + st: proxy, + }, + } + view.opts.restartable = isRestartable + bm.mst.EXPECT().Close().Return(nil).AnyTimes() + + ret := view.close() + test.AssertNil(t, ret) + test.AssertTrue(t, len(view.partitions) == 0) + ret = view.close() + test.AssertNil(t, ret) + test.AssertTrue(t, len(view.partitions) == 0) + }) - vinf, err := v.Get(key) - ensure.Nil(t, err) - ensure.DeepEqual(t, vinf, val) + t.Run("fail", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() - err = v.Evict(key) - ensure.Nil(t, err) + var ( + proxy = &storageProxy{ + Storage: bm.mst, + } + retErr error = fmt.Errorf("some-error") + isRestartable = true + ) + view.partitions = []*PartitionTable{ + &PartitionTable{ + st: proxy, + }, + &PartitionTable{ + st: proxy, + }, + &PartitionTable{ + st: proxy, + }, + } + view.opts.restartable = isRestartable + bm.mst.EXPECT().Close().Return(retErr).AnyTimes() - vinf, err = v.Get(key) - ensure.Nil(t, err) - ensure.Nil(t, vinf) + ret := view.close() + test.AssertNotNil(t, ret) + test.AssertTrue(t, len(view.partitions) == 0) + }) } -func doTimed(t *testing.T, do func()) error { - ch := make(chan bool) - go func() { - do() - close(ch) - }() - - select { - case <-time.After(2 * time.Second): - t.Fail() - return errors.New("function took too long to complete") - case <-ch: - } +func TestView_Run(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + oldest int64 + newest int64 = 10 + // local int64 = oldest + consumer = defaultSaramaAutoConsumerMock(t) + partition int32 + count int64 + updateCB UpdateCallback = func(s storage.Storage, partition int32, key string, value []byte) error { + count++ + return nil + } + ) + bm.useMemoryStorage() + + pt := newPartitionTable( + viewTestTopic, + partition, + consumer, + bm.tmgr, + updateCB, + bm.getStorageBuilder(), + logger.Default(), + ) + + pt.consumer = consumer + view.partitions = []*PartitionTable{pt} + view.state = NewSignal(State(ViewStateCatchUp), State(ViewStateRunning), State(ViewStateIdle)).SetState(State(ViewStateIdle)) + + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(oldest, nil).AnyTimes() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(newest, nil).AnyTimes() + partConsumer := consumer.ExpectConsumePartition(viewTestTopic, partition, AnyOffset) + for i := 0; i < 10; i++ { + partConsumer.YieldMessage(&sarama.ConsumerMessage{}) + } - return nil + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + go func() { + for { + select { + case <-ctx.Done(): + return + default: + } + if count == 10 { + time.Sleep(time.Millisecond * 10) + cancel() + return + } + } + }() + + ret := view.Run(ctx) + test.AssertNil(t, ret) + }) + t.Run("fail", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + partition int32 + consumer = defaultSaramaAutoConsumerMock(t) + updateCB UpdateCallback + retErr = fmt.Errorf("run error") + ) + bm.useMemoryStorage() + + pt := newPartitionTable( + viewTestTopic, + partition, + consumer, + bm.tmgr, + updateCB, + bm.getStorageBuilder(), + logger.Default(), + ) + + pt.consumer = consumer + view.partitions = []*PartitionTable{pt} + view.state = NewSignal(State(ViewStateCatchUp), State(ViewStateRunning), State(ViewStateIdle)).SetState(State(ViewStateIdle)) + + bm.mst.EXPECT().GetOffset(gomock.Any()).Return(int64(0), retErr).AnyTimes() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetNewest).Return(sarama.OffsetNewest, retErr).AnyTimes() + bm.tmgr.EXPECT().GetOffset(pt.topic, pt.partition, sarama.OffsetOldest).Return(sarama.OffsetOldest, retErr).AnyTimes() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + + ret := view.Run(ctx) + test.AssertNotNil(t, ret) + }) } -func ExampleView_simple() { - var ( - brokers = []string{"localhost:9092"} - group Group = "group-name" - ) - v, err := NewView(brokers, GroupTable(group), nil) - if err != nil { - panic(err) - } - if err = v.Run(context.Background()); err != nil { - panic(err) - } +func TestView_createPartitions(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + partition int32 + ) + bm.tmgr.EXPECT().Partitions(viewTestTopic).Return([]int32{partition}, nil) + bm.tmgr.EXPECT().Close() + + ret := view.createPartitions([]string{""}) + test.AssertNil(t, ret) + test.AssertTrue(t, len(view.partitions) == 1) + }) + t.Run("fail_tmgr", func(t *testing.T) { + view, bm, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + var ( + retErr error = fmt.Errorf("tmgr-partition-error") + ) + bm.tmgr.EXPECT().Partitions(viewTestTopic).Return(nil, retErr) + bm.tmgr.EXPECT().Close() + + ret := view.createPartitions([]string{""}) + test.AssertNotNil(t, ret) + test.AssertTrue(t, len(view.partitions) == 0) + }) +} + +func TestView_WaitRunning(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + view, _, ctrl := createTestView(t, NewMockAutoConsumer(t, DefaultConfig())) + defer ctrl.Finish() + + view.state = NewSignal(State(ViewStateCatchUp), State(ViewStateRunning), State(ViewStateIdle)).SetState(State(ViewStateRunning)) + + var isRunning bool + select { + case <-view.WaitRunning(): + isRunning = true + case <-time.After(time.Second): + } + + test.AssertTrue(t, isRunning == true) + }) +} + +func TestView_NewView(t *testing.T) { + t.Run("succeed", func(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + bm := newBuilderMock(ctrl) + + var ( + partition int32 + ) + bm.tmgr.EXPECT().Partitions(viewTestTopic).Return([]int32{partition}, nil).AnyTimes() + bm.tmgr.EXPECT().Close().AnyTimes() + + view, err := NewView([]string{""}, Table(viewTestTopic), &codec.Int64{}, []ViewOption{ + WithViewTopicManagerBuilder(bm.getTopicManagerBuilder()), + WithViewConsumerSaramaBuilder(func(brokers []string, clientID string) (sarama.Consumer, error) { + return NewMockAutoConsumer(t, DefaultConfig()), nil + }), + }...) + test.AssertNil(t, err) + test.AssertNotNil(t, view) + }) + t.Run("succeed", func(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + bm := newBuilderMock(ctrl) + + var ( + retErr error = fmt.Errorf("tmgr-error") + ) + bm.tmgr.EXPECT().Partitions(viewTestTopic).Return(nil, retErr).AnyTimes() + bm.tmgr.EXPECT().Close().AnyTimes() + + view, err := NewView([]string{""}, Table(viewTestTopic), &codec.Int64{}, []ViewOption{ + WithViewTopicManagerBuilder(bm.getTopicManagerBuilder()), + WithViewConsumerSaramaBuilder(func(brokers []string, clientID string) (sarama.Consumer, error) { + return NewMockAutoConsumer(t, DefaultConfig()), nil + }), + }...) + test.AssertNotNil(t, err) + test.AssertNil(t, view) + }) } diff --git a/web/monitor/monitoring.go b/web/monitor/monitoring.go index 1141f773..56014a86 100644 --- a/web/monitor/monitoring.go +++ b/web/monitor/monitoring.go @@ -1,10 +1,12 @@ package monitor import ( + "context" "encoding/json" "fmt" "strconv" "sync" + "time" "github.com/lovoo/goka" "github.com/lovoo/goka/logger" @@ -116,7 +118,9 @@ func (s *Server) renderData(w http.ResponseWriter, r *http.Request) { http.NotFound(w, r) return } - stats = s.views[idx].Stats() + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + stats = s.views[idx].Stats(ctx) default: w.Write([]byte("Invalid render type")) http.NotFound(w, r) @@ -128,12 +132,13 @@ func (s *Server) renderData(w http.ResponseWriter, r *http.Request) { w.Write([]byte(err.Error())) return } + w.Write(marshalled) } // renders the processor page func (s *Server) renderProcessor(w http.ResponseWriter, r *http.Request) { - tmpl, err := templates.LoadTemplates(append(baseTemplates, "web/templates/monitor/details.go.html")...) + tmpl, err := templates.LoadTemplates(append(baseTemplates, "web/templates/monitor/details_processor.go.html")...) if err != nil { http.Error(w, err.Error(), http.StatusInternalServerError) return @@ -163,7 +168,7 @@ func (s *Server) renderProcessor(w http.ResponseWriter, r *http.Request) { // renders the processor page func (s *Server) renderView(w http.ResponseWriter, r *http.Request) { - tmpl, err := templates.LoadTemplates(append(baseTemplates, "web/templates/monitor/details.go.html")...) + tmpl, err := templates.LoadTemplates(append(baseTemplates, "web/templates/monitor/details_view.go.html")...) if err != nil { http.Error(w, err.Error(), http.StatusInternalServerError) return diff --git a/web/templates/bindata.go b/web/templates/bindata.go index a03ae41e..57ffda50 100644 --- a/web/templates/bindata.go +++ b/web/templates/bindata.go @@ -1,17 +1,18 @@ -// Code generated by go-bindata. +// Code generated by go-bindata. DO NOT EDIT. // sources: // web/templates/common/base.go.html // web/templates/common/head.go.html // web/templates/common/menu.go.html -// web/templates/monitor/details.go.html +// web/templates/monitor/details_processor.go.html +// web/templates/monitor/details_view.go.html // web/templates/monitor/index.go.html // web/templates/monitor/menu.go.html // web/templates/query/index.go.html // web/templates/index/index.go.html -// DO NOT EDIT! package templates + import ( "bytes" "compress/gzip" @@ -44,16 +45,23 @@ func bindataRead(data []byte, name string) ([]byte, error) { return buf.Bytes(), nil } + type asset struct { bytes []byte - info os.FileInfo + info fileInfoEx +} + +type fileInfoEx interface { + os.FileInfo + MD5Checksum() string } type bindataFileInfo struct { - name string - size int64 - mode os.FileMode - modTime time.Time + name string + size int64 + mode os.FileMode + modTime time.Time + md5checksum string } func (fi bindataFileInfo) Name() string { @@ -68,6 +76,9 @@ func (fi bindataFileInfo) Mode() os.FileMode { func (fi bindataFileInfo) ModTime() time.Time { return fi.modTime } +func (fi bindataFileInfo) MD5Checksum() string { + return fi.md5checksum +} func (fi bindataFileInfo) IsDir() bool { return false } @@ -75,169 +86,552 @@ func (fi bindataFileInfo) Sys() interface{} { return nil } -var _webTemplatesCommonBaseGoHtml = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xaa\xae\x4e\x49\x4d\xcb\xcc\x4b\x55\x50\x4a\x4a\x2c\x4e\x55\xaa\xad\xe5\xb2\x51\x74\xf1\x77\x0e\x89\x0c\x70\x55\xc8\x28\xc9\xcd\xb1\xe3\xaa\xae\x2e\x49\xcd\x2d\xc8\x49\x2c\x49\x55\x50\xca\x48\x4d\x4c\x51\x52\xd0\x03\xa9\x82\x48\xda\x24\xe5\xa7\x54\xa2\xaa\xc9\x4d\xcd\x2b\x85\xa8\x41\x16\x4d\xce\xcf\x2b\x49\xcd\x2b\x81\x6a\xd6\x87\x68\xb3\xd1\x87\x59\x91\x9a\x97\x52\x5b\xcb\x05\x08\x00\x00\xff\xff\xfd\x8f\xc0\x67\x8d\x00\x00\x00") +var _bindataWebTemplatesCommonBasegohtml = []byte( + "\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xaa\xae\x4e\x49\x4d\xcb\xcc\x4b\x55\x50\x4a\x4a\x2c\x4e\x55\xaa\xad\xe5" + + "\xb2\x51\x74\xf1\x77\x0e\x89\x0c\x70\x55\xc8\x28\xc9\xcd\xb1\xe3\xaa\xae\x2e\x49\xcd\x2d\xc8\x49\x2c\x49\x55\x50" + + "\xca\x48\x4d\x4c\x51\x52\xd0\x03\xa9\x82\x48\xda\x24\xe5\xa7\x54\xa2\xaa\xc9\x4d\xcd\x2b\x85\xa8\x41\x16\x4d\xce" + + "\xcf\x2b\x49\xcd\x2b\x81\x6a\xd6\x87\x68\xb3\xd1\x87\x59\x91\x9a\x97\x52\x5b\xcb\x05\x08\x00\x00\xff\xff\xfd\x8f" + + "\xc0\x67\x8d\x00\x00\x00") -func webTemplatesCommonBaseGoHtmlBytes() ([]byte, error) { +func bindataWebTemplatesCommonBasegohtmlBytes() ([]byte, error) { return bindataRead( - _webTemplatesCommonBaseGoHtml, + _bindataWebTemplatesCommonBasegohtml, "web/templates/common/base.go.html", ) } -func webTemplatesCommonBaseGoHtml() (*asset, error) { - bytes, err := webTemplatesCommonBaseGoHtmlBytes() + + +func bindataWebTemplatesCommonBasegohtml() (*asset, error) { + bytes, err := bindataWebTemplatesCommonBasegohtmlBytes() if err != nil { return nil, err } - info := bindataFileInfo{name: "web/templates/common/base.go.html", size: 141, mode: os.FileMode(436), modTime: time.Unix(1548705120, 0)} + info := bindataFileInfo{ + name: "web/templates/common/base.go.html", + size: 141, + md5checksum: "", + mode: os.FileMode(420), + modTime: time.Unix(1559056733, 0), + } + a := &asset{bytes: bytes, info: info} + return a, nil } -var _webTemplatesCommonHeadGoHtml = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xb4\x54\x4d\x73\xf2\x36\x18\xbc\xe7\x57\x78\x74\xe1\x90\x5a\xc2\x98\x16\x92\xc1\xe9\xa4\xa4\xe5\x23\x5f\x04\x12\x08\xbd\x74\x84\xf4\xd8\x96\x23\x4b\x8e\x25\x03\x2e\xe5\xbf\x77\x1c\x4a\xc2\xa4\x79\x3f\x72\x78\x6f\x7e\xd6\xab\xf5\xee\xfa\x19\x6d\x36\x1c\x42\xa1\xc0\x41\x31\x50\x8e\xb6\xdb\xa3\x4e\xf5\x70\x76\xe4\x38\x8e\xd3\x49\xc1\x52\x87\xc5\x34\x37\x60\x03\x54\xd8\xd0\x6d\xa3\xc3\x57\xb1\xb5\x99\x0b\xcf\x85\x58\x06\x68\xed\x16\xd4\x65\x3a\xcd\xa8\x15\x0b\x09\xc8\x61\x5a\x59\x50\x36\x40\x02\x02\xe0\x11\xec\x4f\x5a\x61\x25\x9c\x6d\x36\x38\xa3\x11\xfc\xf5\x32\x6d\xb7\x1d\xb2\x83\x0f\xc4\x15\x4d\x21\x40\x1c\x0c\xcb\x45\x66\x85\x56\x07\x92\xe8\xff\xc4\xa5\x80\x55\xa6\x73\x7b\xc0\x5a\x09\x6e\xe3\x80\xc3\x52\x30\x70\x5f\x86\x9f\x1c\xa1\x84\x15\x54\xba\x86\x51\x09\x81\xb7\x17\x92\x42\x3d\x39\x71\x0e\x61\x50\xab\x42\x99\x53\x42\x42\xad\xac\xc1\x91\xd6\x91\x04\x9a\x09\x83\x99\x4e\x09\x33\xe6\xd7\x90\xa6\x42\x96\xc1\x6d\x06\xea\x78\x42\x95\x39\xee\x6a\xc5\x41\x19\xe0\xa7\x7e\xbd\xfe\xcf\x2b\x5e\x73\x72\x90\x41\xcd\xd8\x52\x82\x89\x01\x6c\xcd\xb1\x65\x06\x41\xcd\xc2\xda\x56\x4a\xb5\xc3\x8f\x57\x5c\xf4\xc6\x45\x3b\x37\x68\xef\x26\xa5\x6b\xc6\x15\x5e\x68\x6d\x8d\xcd\x69\x56\x0d\x95\xa1\x57\x80\xf8\xd8\xc7\xad\x4a\xf6\x0d\xc3\xa9\x50\x98\x19\x83\x1c\xa1\x2c\x44\xb9\xb0\x65\x80\x4c\x4c\xfd\x76\xd3\xfd\x6d\x3a\x17\x62\x32\xf8\x03\x2e\x3d\xde\x4b\x87\xe3\xf3\xa7\x92\x15\xfd\xf3\xfe\x38\xf2\x1b\xb7\xe9\x03\x5b\xad\x5a\x5a\xf9\xe3\x39\x8f\x9a\x53\x7a\x3c\x4a\x27\xf7\xe6\x6f\x72\xf9\x4b\x7b\xb9\xe0\xbf\x27\x71\xb3\x40\x0e\xcb\xb5\x31\x3a\x17\x91\x50\x01\xa2\x4a\xab\x32\xd5\x85\x41\x3f\x3a\x94\x6b\x63\x48\xe1\x6b\xd1\xf2\x7e\xa9\x6f\x3c\x31\x36\xd3\xc7\x69\x53\x5d\xd4\x87\x85\x95\xaa\x47\x8d\xec\x0e\x8b\x6e\xab\x58\x25\xbc\x98\x9d\x4c\xa6\xf9\xd5\x72\x3c\xd7\x7a\x94\x35\x16\xb3\x79\x94\x46\xc3\xbb\xc1\xe3\x4a\x92\x49\xf6\xad\x68\xbb\x8d\x74\x4c\xce\x02\x44\x08\x4d\xe8\xfa\xfd\x9a\x54\x18\x91\x62\x61\x48\xf2\x5c\x40\x5e\x12\x0f\x7b\x1e\xae\xff\x37\xbd\x78\x4f\x0c\x3a\xeb\x90\x9d\xd4\x07\xba\x9f\xad\x28\x79\xff\xdb\x93\x0f\xab\xb9\x67\x3f\x0f\xee\xc4\xa2\xde\x68\x3d\x2f\xcb\x64\x72\x1d\xf6\x93\xdb\x6b\x7a\xf5\x14\x16\xb3\xe9\xfa\xcf\xf5\xc3\x48\x75\x87\xe7\x2d\xd9\x48\xbb\xb3\x9b\x41\xd6\x3b\x49\x7b\xdd\x8b\xf6\xaa\x77\x33\x60\xa3\x8b\xd6\xfd\x9a\x7e\xb9\x9a\xef\xc8\xc2\xb8\x4a\x0c\x66\x52\x17\x3c\x94\x34\x87\x77\x55\x49\xcd\xa9\x89\x71\x62\x48\x13\x7b\x2d\xec\xed\x81\x4f\xb4\xc5\xfd\xc4\x60\x9d\x47\x84\xfb\x78\xd9\xfc\xe0\x64\x87\xec\xae\xb7\xcd\x06\x14\xdf\x6e\x8f\xfe\x0d\x00\x00\xff\xff\x25\x6d\x7e\xc6\x00\x05\x00\x00") - -func webTemplatesCommonHeadGoHtmlBytes() ([]byte, error) { +var _bindataWebTemplatesCommonHeadgohtml = []byte( + "\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xb4\x54\x4d\x73\xf2\x36\x18\xbc\xe7\x57\x78\x74\xe1\x90\x5a\xc2\x98\x16" + + "\x92\xc1\xe9\xa4\xa4\xe5\x23\x5f\x04\x12\x08\xbd\x74\x84\xf4\xd8\x96\x23\x4b\x8e\x25\x03\x2e\xe5\xbf\x77\x1c\x4a" + + "\xc2\xa4\x79\x3f\x72\x78\x6f\x7e\xd6\xab\xf5\xee\xfa\x19\x6d\x36\x1c\x42\xa1\xc0\x41\x31\x50\x8e\xb6\xdb\xa3\x4e" + + "\xf5\x70\x76\xe4\x38\x8e\xd3\x49\xc1\x52\x87\xc5\x34\x37\x60\x03\x54\xd8\xd0\x6d\xa3\xc3\x57\xb1\xb5\x99\x0b\xcf" + + "\x85\x58\x06\x68\xed\x16\xd4\x65\x3a\xcd\xa8\x15\x0b\x09\xc8\x61\x5a\x59\x50\x36\x40\x02\x02\xe0\x11\xec\x4f\x5a" + + "\x61\x25\x9c\x6d\x36\x38\xa3\x11\xfc\xf5\x32\x6d\xb7\x1d\xb2\x83\x0f\xc4\x15\x4d\x21\x40\x1c\x0c\xcb\x45\x66\x85" + + "\x56\x07\x92\xe8\xff\xc4\xa5\x80\x55\xa6\x73\x7b\xc0\x5a\x09\x6e\xe3\x80\xc3\x52\x30\x70\x5f\x86\x9f\x1c\xa1\x84" + + "\x15\x54\xba\x86\x51\x09\x81\xb7\x17\x92\x42\x3d\x39\x71\x0e\x61\x50\xab\x42\x99\x53\x42\x42\xad\xac\xc1\x91\xd6" + + "\x91\x04\x9a\x09\x83\x99\x4e\x09\x33\xe6\xd7\x90\xa6\x42\x96\xc1\x6d\x06\xea\x78\x42\x95\x39\xee\x6a\xc5\x41\x19" + + "\xe0\xa7\x7e\xbd\xfe\xcf\x2b\x5e\x73\x72\x90\x41\xcd\xd8\x52\x82\x89\x01\x6c\xcd\xb1\x65\x06\x41\xcd\xc2\xda\x56" + + "\x4a\xb5\xc3\x8f\x57\x5c\xf4\xc6\x45\x3b\x37\x68\xef\x26\xa5\x6b\xc6\x15\x5e\x68\x6d\x8d\xcd\x69\x56\x0d\x95\xa1" + + "\x57\x80\xf8\xd8\xc7\xad\x4a\xf6\x0d\xc3\xa9\x50\x98\x19\x83\x1c\xa1\x2c\x44\xb9\xb0\x65\x80\x4c\x4c\xfd\x76\xd3" + + "\xfd\x6d\x3a\x17\x62\x32\xf8\x03\x2e\x3d\xde\x4b\x87\xe3\xf3\xa7\x92\x15\xfd\xf3\xfe\x38\xf2\x1b\xb7\xe9\x03\x5b" + + "\xad\x5a\x5a\xf9\xe3\x39\x8f\x9a\x53\x7a\x3c\x4a\x27\xf7\xe6\x6f\x72\xf9\x4b\x7b\xb9\xe0\xbf\x27\x71\xb3\x40\x0e" + + "\xcb\xb5\x31\x3a\x17\x91\x50\x01\xa2\x4a\xab\x32\xd5\x85\x41\x3f\x3a\x94\x6b\x63\x48\xe1\x6b\xd1\xf2\x7e\xa9\x6f" + + "\x3c\x31\x36\xd3\xc7\x69\x53\x5d\xd4\x87\x85\x95\xaa\x47\x8d\xec\x0e\x8b\x6e\xab\x58\x25\xbc\x98\x9d\x4c\xa6\xf9" + + "\xd5\x72\x3c\xd7\x7a\x94\x35\x16\xb3\x79\x94\x46\xc3\xbb\xc1\xe3\x4a\x92\x49\xf6\xad\x68\xbb\x8d\x74\x4c\xce\x02" + + "\x44\x08\x4d\xe8\xfa\xfd\x9a\x54\x18\x91\x62\x61\x48\xf2\x5c\x40\x5e\x12\x0f\x7b\x1e\xae\xff\x37\xbd\x78\x4f\x0c" + + "\x3a\xeb\x90\x9d\xd4\x07\xba\x9f\xad\x28\x79\xff\xdb\x93\x0f\xab\xb9\x67\x3f\x0f\xee\xc4\xa2\xde\x68\x3d\x2f\xcb" + + "\x64\x72\x1d\xf6\x93\xdb\x6b\x7a\xf5\x14\x16\xb3\xe9\xfa\xcf\xf5\xc3\x48\x75\x87\xe7\x2d\xd9\x48\xbb\xb3\x9b\x41" + + "\xd6\x3b\x49\x7b\xdd\x8b\xf6\xaa\x77\x33\x60\xa3\x8b\xd6\xfd\x9a\x7e\xb9\x9a\xef\xc8\xc2\xb8\x4a\x0c\x66\x52\x17" + + "\x3c\x94\x34\x87\x77\x55\x49\xcd\xa9\x89\x71\x62\x48\x13\x7b\x2d\xec\xed\x81\x4f\xb4\xc5\xfd\xc4\x60\x9d\x47\x84" + + "\xfb\x78\xd9\xfc\xe0\x64\x87\xec\xae\xb7\xcd\x06\x14\xdf\x6e\x8f\xfe\x0d\x00\x00\xff\xff\x25\x6d\x7e\xc6\x00\x05" + + "\x00\x00") + +func bindataWebTemplatesCommonHeadgohtmlBytes() ([]byte, error) { return bindataRead( - _webTemplatesCommonHeadGoHtml, + _bindataWebTemplatesCommonHeadgohtml, "web/templates/common/head.go.html", ) } -func webTemplatesCommonHeadGoHtml() (*asset, error) { - bytes, err := webTemplatesCommonHeadGoHtmlBytes() + + +func bindataWebTemplatesCommonHeadgohtml() (*asset, error) { + bytes, err := bindataWebTemplatesCommonHeadgohtmlBytes() if err != nil { return nil, err } - info := bindataFileInfo{name: "web/templates/common/head.go.html", size: 1280, mode: os.FileMode(436), modTime: time.Unix(1548705120, 0)} + info := bindataFileInfo{ + name: "web/templates/common/head.go.html", + size: 1280, + md5checksum: "", + mode: os.FileMode(420), + modTime: time.Unix(1559056733, 0), + } + a := &asset{bytes: bytes, info: info} + return a, nil } -var _webTemplatesCommonMenuGoHtml = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\x64\x90\x41\x4e\xc4\x30\x0c\x45\xf7\x3d\x85\x65\x36\xb0\x60\x7a\x81\xb4\x0b\x38\x08\x72\x13\x57\x8d\x6a\x1c\x94\x36\x91\x50\x94\xbb\x23\x98\x36\xc0\xcc\x2a\xb1\xfe\x8b\xde\x8f\x4b\x71\x3c\x7b\x65\xc0\x77\xd6\x84\xb5\x76\x46\x29\x83\x15\xda\xb6\x01\x95\xf2\x44\x11\xae\xc7\xb3\xe3\x99\x92\xec\x38\x76\x00\xc6\xf9\x46\xd9\xa0\x3b\x79\xe5\xf8\x93\xfc\xcf\x8e\xa7\x0b\x93\x6b\x39\x80\xa1\x9b\x7c\x8a\xa4\x0e\x61\x89\x3c\x0f\xf8\x80\x10\xf4\x55\xbc\x5d\x07\x5c\xfc\xb6\x87\xf8\x79\x99\xc8\xae\x8f\x4f\x38\xbe\x90\x5d\x4d\x4f\x87\xa8\x77\x3e\xdf\x3b\x6d\x10\xa1\x8f\x8d\xcf\xde\xe7\xfc\xab\x4f\xf2\xc7\x7f\x62\x4a\xb9\x11\xa5\x4c\x12\xec\x7a\xdd\xca\xdb\xf7\x07\x59\x77\x84\x4b\xad\x0d\x60\x75\x6d\x32\x7d\x92\x9b\x4a\xc7\xc5\xf4\x4a\x79\xec\x4a\x01\x56\x07\xb5\x76\x5f\x01\x00\x00\xff\xff\x42\x5b\x78\xaf\x71\x01\x00\x00") +var _bindataWebTemplatesCommonMenugohtml = []byte( + "\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\x64\x90\x41\x4e\xc4\x30\x0c\x45\xf7\x3d\x85\x65\x36\xb0\x60\x7a\x81\xb4" + + "\x0b\x38\x08\x72\x13\x57\x8d\x6a\x1c\x94\x36\x91\x50\x94\xbb\x23\x98\x36\xc0\xcc\x2a\xb1\xfe\x8b\xde\x8f\x4b\x71" + + "\x3c\x7b\x65\xc0\x77\xd6\x84\xb5\x76\x46\x29\x83\x15\xda\xb6\x01\x95\xf2\x44\x11\xae\xc7\xb3\xe3\x99\x92\xec\x38" + + "\x76\x00\xc6\xf9\x46\xd9\xa0\x3b\x79\xe5\xf8\x93\xfc\xcf\x8e\xa7\x0b\x93\x6b\x39\x80\xa1\x9b\x7c\x8a\xa4\x0e\x61" + + "\x89\x3c\x0f\xf8\x80\x10\xf4\x55\xbc\x5d\x07\x5c\xfc\xb6\x87\xf8\x79\x99\xc8\xae\x8f\x4f\x38\xbe\x90\x5d\x4d\x4f" + + "\x87\xa8\x77\x3e\xdf\x3b\x6d\x10\xa1\x8f\x8d\xcf\xde\xe7\xfc\xab\x4f\xf2\xc7\x7f\x62\x4a\xb9\x11\xa5\x4c\x12\xec" + + "\x7a\xdd\xca\xdb\xf7\x07\x59\x77\x84\x4b\xad\x0d\x60\x75\x6d\x32\x7d\x92\x9b\x4a\xc7\xc5\xf4\x4a\x79\xec\x4a\x01" + + "\x56\x07\xb5\x76\x5f\x01\x00\x00\xff\xff\x42\x5b\x78\xaf\x71\x01\x00\x00") -func webTemplatesCommonMenuGoHtmlBytes() ([]byte, error) { +func bindataWebTemplatesCommonMenugohtmlBytes() ([]byte, error) { return bindataRead( - _webTemplatesCommonMenuGoHtml, + _bindataWebTemplatesCommonMenugohtml, "web/templates/common/menu.go.html", ) } -func webTemplatesCommonMenuGoHtml() (*asset, error) { - bytes, err := webTemplatesCommonMenuGoHtmlBytes() + + +func bindataWebTemplatesCommonMenugohtml() (*asset, error) { + bytes, err := bindataWebTemplatesCommonMenugohtmlBytes() if err != nil { return nil, err } - info := bindataFileInfo{name: "web/templates/common/menu.go.html", size: 369, mode: os.FileMode(436), modTime: time.Unix(1548705120, 0)} + info := bindataFileInfo{ + name: "web/templates/common/menu.go.html", + size: 369, + md5checksum: "", + mode: os.FileMode(420), + modTime: time.Unix(1559056733, 0), + } + a := &asset{bytes: bytes, info: info} + return a, nil } -var _webTemplatesMonitorDetailsGoHtml = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xe4\x59\xdf\x6f\xdb\x38\xf2\x7f\xf7\x5f\x31\x5f\x7d\x1f\x2a\xa1\xb6\x6c\x27\x6f\x75\x6c\xe0\xae\xc5\x5d\xbb\xd7\x76\x8b\x6b\xb1\x7d\xc8\x05\x05\x2d\x8e\x6d\xa6\x12\xa9\x23\x69\x3b\x5e\x43\xff\xfb\x81\xa4\x24\x93\x8a\x1d\x67\x7b\x09\xb0\xc5\xe5\x21\xb0\x39\x3f\x39\x9c\xf9\xcc\x90\xde\xef\x29\x2e\x18\x47\x88\x32\xc1\x35\x72\x1d\x55\x55\xef\x8a\xb2\x0d\x64\x39\x51\x6a\x1a\x49\xb1\x8d\x66\x3d\x00\x7f\xcd\xb0\x12\xc6\x51\x5a\x4a\x97\x96\x0f\x0a\x3a\x18\x5f\x34\xb4\xd5\x78\xb6\xdf\xa7\x9a\xe9\x1c\xab\xea\x6a\xb8\x1a\xdf\x97\x29\x09\xc7\x1c\xec\xff\x01\xc5\x05\x59\xe7\xba\x96\x3e\xcd\xb7\x42\x42\x19\x5f\xb6\x7c\xc6\xd2\xe5\xec\x0b\x99\xe7\x08\x4a\x13\xcd\x94\x66\x99\xba\x1a\xae\x2e\x5b\x4d\x43\xca\x36\x27\xd5\x0e\xe6\x82\xee\x7c\x6d\xda\xaa\xaa\x59\xdc\x17\xfb\x7f\xa0\xb4\x64\x25\x52\x8f\xd7\x70\x1b\x7f\xfc\x15\xb3\x26\xc3\x05\xcb\x06\x36\x12\xc6\xa8\xd4\x4c\x33\xc1\x81\xaf\x8b\x39\xca\x08\xa4\xd8\xaa\x92\xf0\x69\x74\x11\xcd\x3e\x35\xd4\xab\xa1\x5e\x3d\xa0\xe5\xb3\x26\x7a\xad\x40\x2c\x40\xaf\x10\x4a\x29\x32\x54\x4a\x74\x94\x19\x26\x3c\xa3\xe8\xa3\x75\xc2\x28\x2a\x50\x29\xb2\x44\x05\x39\x59\x2e\x19\x5f\xc2\x1c\x57\x8c\x53\x78\xfb\xf5\x43\xa8\xf6\xd7\xc5\x42\xa1\x1e\xbc\x27\xcb\x33\xba\xdf\xb2\xe5\x0a\xb6\x44\xa3\x84\x82\xc8\xef\x10\x0b\x2b\xd9\x78\xcd\xf1\x4e\x37\x56\x61\x8e\xc6\x64\x26\xb8\x5a\x17\x48\x93\xd0\xe2\xdb\xaf\x1f\x4e\x99\xf2\xd9\xbe\x4a\xa6\x71\xf0\xcf\x07\x36\x9d\x89\xdc\x71\x5f\x46\xb3\x77\xbc\x5c\xeb\xb3\x8c\x66\xbf\x6b\x7d\x94\xf3\x6a\xd8\x3d\xe8\xe3\x27\x3f\x93\xa7\x3d\x9a\xcd\x77\x1a\xd5\x29\x22\xc5\x9c\xec\x4e\x11\x7f\x40\x6b\xd7\x63\xc3\xd1\x49\xdf\x2b\x6d\xea\x01\x18\xf5\x32\xf5\x37\x86\xdb\xa8\x23\x67\xb8\xbc\xa2\x19\xda\x12\x39\x52\x71\xf5\xc7\xde\x7e\xcf\x16\x80\xff\x86\x54\x22\xa7\x28\xbf\xec\x4a\x84\xe8\x90\xb7\x55\xf5\x8c\xe0\xf0\x5a\x0c\x7e\x11\x8c\x23\x05\x0b\x13\x3f\x13\x38\x3c\x1a\x10\xbe\x88\x92\x65\x75\x15\xdd\xda\xcd\x46\x0e\x14\x7f\x10\x48\x9e\x10\x3c\xfe\x5c\x80\x51\x9b\x32\x28\x11\x38\xbe\x95\x4c\x6b\xe4\xa0\x05\xe4\x22\x23\xb9\x6d\x26\x08\x25\x4a\x50\x98\x09\x4e\x1f\xc6\x97\x3f\x56\x5a\xe6\x84\x9e\xa0\xaa\xf6\x7b\xe4\xb4\xaa\x7a\x6e\x4d\x65\x92\x95\x1a\xf4\xae\xc4\x69\xa4\xf1\x4e\x0f\x6f\xc9\x86\xb8\xd5\x68\xd6\xab\xb5\x6c\x88\x84\x9c\x28\xdd\x26\x16\x4c\x81\x5e\xa6\x76\xcf\x71\x32\xe9\x70\x99\xba\x31\x99\xa0\x4e\x72\x29\x9b\x40\x1f\x48\x09\x53\xd8\x8f\x5e\x41\x24\x31\x13\x1b\x94\xa6\x0c\xfb\x30\x7e\x15\x95\x12\x4b\x52\x7f\xbd\x30\xf4\x35\xe7\xe6\x5b\x35\xf1\x5d\x72\xb0\xf0\x06\x35\x61\xb9\x31\xb6\x58\xf3\xcc\x78\x17\xb7\x25\xa1\x92\x7d\xaf\x0d\x8d\x11\x59\x97\x94\x68\x6c\xf7\xf1\xc9\xe2\x80\x27\x49\x89\x26\xc9\xde\x0b\xb0\x11\x6a\xd5\xbd\xa3\x66\x4b\x44\x93\xeb\xd1\xcd\xa4\xc3\xa4\x9a\x1d\x1b\xf2\xf8\x66\xd2\x3b\x42\x5f\x1b\x86\x76\xf3\xd7\x56\x24\xb5\x15\x97\xba\x9a\x0a\xc5\xd8\x22\xee\xb0\xe4\x39\xd2\xc0\x3d\x38\xe8\x8d\x94\xa3\x47\xbe\x67\x55\xc7\x0b\x57\x1c\xef\xc9\xb2\x76\xa4\xd1\xfd\x76\x5b\xc0\x20\x58\x71\x05\x08\x03\x18\x4f\x7a\xbe\x53\xc3\x21\x30\xce\x34\x23\x39\xfb\x1d\x4d\x89\x15\x30\xdf\x41\x0d\xb6\x1d\x6b\xa6\x3e\xd0\x56\xcd\x14\x46\xdd\x80\x31\xd3\x4b\x1f\x24\xfe\xd5\x74\xa4\x63\x54\x61\xbb\xeb\x29\x59\x47\x3d\x29\x6c\x55\xbf\x31\x5d\x12\xa6\x30\x18\x4f\xc2\xbd\x29\x4d\xa4\x86\x2d\xd3\x2b\x20\x90\xad\x88\x6d\x01\xa2\x44\x49\x6c\xde\x0b\x6e\x51\xc5\xce\x01\x06\x0a\x5c\xc4\x8e\x59\x78\x6d\x64\x61\x0a\xdf\x52\xab\xa5\x3e\x49\x2b\x98\x74\x6c\xb2\x05\x6c\x11\xa8\x80\x15\xd9\x20\x6c\x48\xbe\x46\x65\x30\x65\x89\xda\x5a\x23\x9b\x25\x2c\xa4\x28\xfa\x90\xa3\x7e\xa1\x40\x93\xef\x08\x4c\xa7\xc7\xb4\x68\xc9\x8c\xc3\x1c\x08\x07\x2c\x4a\xbd\x83\x9c\x29\xdd\x07\xa6\x61\x2b\xd6\x39\x85\x4c\xa2\x89\x1b\x81\x8f\xe4\x63\x98\x6b\x07\xb7\x53\xc5\x7e\xc7\x38\x49\xad\x2b\x71\x02\x33\x18\x75\xb2\x2e\x08\xa2\x27\xb8\xc8\x89\xfe\x40\xca\x38\xb2\xb4\x28\x49\x0b\x24\xdc\xd3\x34\x84\xf1\xc8\xfe\x85\x49\xda\x49\x30\xb7\xcd\x6c\x85\xd9\xf7\x7a\x5b\x36\x32\x04\x4a\x89\x1b\x26\xd6\xca\x56\x99\x49\x4f\xc2\x29\x28\x17\xa6\xc2\x84\x4c\xaf\x90\xc9\x50\x59\x26\xa4\xc4\x4c\xd7\x71\xed\x1c\x95\xc1\xab\x06\xab\x02\x84\x4b\x97\xa8\x63\xbd\x62\x2a\xe9\x96\x64\x2b\xd2\x09\x89\x35\x55\xee\xec\x89\x39\x30\xd0\x02\xc8\x46\x30\x0a\x85\xa0\x6c\xb1\x33\xbd\x87\x69\x13\xad\x9c\x64\x18\xc8\x1a\x57\xb2\xb5\x6c\x3c\xf9\xe5\xf3\xaf\x1f\xd3\x92\x48\x85\xb1\xfd\x68\x06\x05\xbe\x64\x8b\x9d\xcb\xa2\x24\x48\x20\x6b\x99\xb2\xc5\xc2\xd8\x6b\x03\xe4\x3c\x60\xbc\x6e\x44\xea\x9e\x39\xcd\x0a\x7c\x63\xa4\xa6\x10\x73\xdc\xc2\x1b\xa2\x31\x6e\x7c\x48\x3f\x8a\x6d\x02\x03\x68\x09\xed\xa6\x2d\xa5\x39\xc7\x74\x14\x3a\xe2\xd7\xfb\x41\x57\x07\x4f\x0e\xaa\x7c\x82\x51\xd9\xb8\x14\x44\x1c\x5a\x70\x76\x27\x68\x18\x62\x31\xbf\xfd\xcd\x7c\xe9\x83\x92\x99\xfd\x94\xc0\xbe\xd3\xb8\x25\xea\xb5\xe4\xf7\x96\x01\x5e\x8b\x35\xd7\xaf\xa0\xd1\x91\xda\xef\x06\xfd\x6a\x55\x6e\xa1\x7f\x4f\xce\x22\x8a\x27\xe7\x10\xc6\x93\xb3\x0b\x5d\xb9\x2a\x0c\x51\xd5\xbb\x77\x12\x4b\x29\xd6\x25\x52\x87\x29\x06\x30\x0a\x94\x4b\xfc\xca\xf4\xea\x10\x43\x4b\xec\x7b\xb1\xab\x17\xda\x90\x74\x52\xc2\x07\xd7\x06\x81\x7c\x3b\xc9\xa1\x52\xed\x6e\xa3\x24\x55\xeb\x22\x28\xd4\xc3\x69\xdc\x53\xdc\x60\xeb\x19\xcd\x96\xed\x41\xcd\xa7\x62\xe1\xae\x4f\xa7\x82\xe1\xa8\x7e\x34\x9a\x95\x53\xe1\x08\x1a\x46\xc7\x6b\x27\xfb\xc3\x01\x09\xbb\xcd\x39\xdd\xe7\x43\xe2\xa7\xca\xe1\x73\x88\x4d\xaa\xc6\xa6\xbe\x2b\xf3\x10\xa2\xea\xbc\x7f\x71\xa5\xe9\xec\xc5\x4b\x6f\x76\x79\xf9\xe2\x6a\xa8\xe9\xec\x5f\xfc\xc5\xcb\x60\x07\x35\xa7\x1b\x24\xce\x30\xb5\xf3\xc3\x23\x94\x79\xb3\xc5\x19\xee\x16\x37\x52\x2d\xfe\xc6\xee\x90\xc6\x17\xc9\x19\x91\x36\xc1\xff\xa0\x88\x3d\x80\x56\x66\xf4\x28\x19\xdb\xcb\x02\x3b\x50\xa8\x87\xc3\xd4\xe6\xdb\xe3\xdd\xf3\x12\xe9\xa8\x7f\x87\xd4\xa8\xbc\xf3\x7e\xdc\x1d\xd9\xfd\xb5\x05\x06\xd3\xc3\x4c\xab\xd2\xbf\x9b\xa5\x89\xa7\x11\x73\x85\x8f\x11\x6c\x13\x52\x05\xd2\x87\xeb\x05\xb8\xe6\x64\x0d\xb8\x7b\x74\xef\x9e\xca\xf7\x4c\xb9\x42\xd7\xe2\x13\x61\x52\xb9\xd2\x49\x52\x25\xa4\x8e\xdb\xb9\x9c\xf4\xe7\xc9\xbe\xce\x6c\xdb\x18\xdf\x71\x1d\x9b\x31\xdc\x74\xa9\x76\x61\x6e\x16\x26\x95\x57\xfc\xc3\x61\x3d\xf2\xf7\x01\xb9\xb9\x23\x9a\x81\x41\x62\x21\x36\x68\x67\x88\xc0\x1f\xea\x2e\x2c\x0a\x73\xcc\x74\x1c\xfd\x7f\xf8\x98\x91\xd4\x84\xbf\xe4\x79\x1c\xa5\x2d\x6d\x2e\xee\xa2\x24\x35\xba\xe2\x76\x3f\x7d\xef\x42\x91\xec\x9b\x8a\xa4\xe6\xd6\x00\xbe\x77\x34\x5d\xe9\x22\x8f\x8f\x5d\x4a\x7c\xae\xe7\x77\x29\xb5\xc1\x89\x93\x94\x94\x25\x72\x1a\x47\x5a\x46\x49\x6a\x9f\x2c\x90\xc6\x51\xa0\xb9\x0f\x5a\xae\x31\x79\x9c\xeb\x29\xde\x31\x1d\x27\xa9\x0b\x79\x9c\x3c\xf6\x51\x07\x82\xfb\x9a\xb9\x51\x3e\xd7\x55\xed\xf2\x28\xf9\xa7\xbd\xa9\x85\x4a\x3a\x17\xb0\x3f\xcb\x94\xdd\xbe\x10\xfc\x6f\x4e\xd9\xea\xa9\x46\xec\xa3\x59\xf0\x98\xf9\xfa\x78\xda\x85\x47\xf3\x2c\x43\x46\xfd\x3a\xf2\xb3\x8e\x22\x41\x0f\xf6\x12\xe3\xb5\xa8\x1f\x8b\x0f\x2d\x31\x80\xb1\x5b\xc1\x78\xdb\xe8\x9a\x61\xf0\xd0\xf2\xbc\x9e\xea\xf4\x24\x1e\x5c\x6f\xee\x5f\x6f\xea\xe8\x7f\x4b\x0b\x52\x3a\xfa\xf5\xf8\xc6\x13\x61\x9c\xa3\xac\x2f\x48\xdf\x71\xd7\xbd\x1b\xd9\x37\x0a\xfa\x0f\x34\x77\x77\x93\x7e\x9f\x6d\xe2\xd7\x8a\x46\x37\x49\x9a\x09\x9e\x11\x1d\x1b\xd1\x30\xfb\x6a\xc3\xd7\x6d\xc7\x6d\x65\xac\xa1\xbe\x53\xdb\x87\x83\x03\x01\xba\xfa\xbd\xaf\xba\xdf\xe2\x61\x1e\x7a\xca\x16\xb6\xc5\xc3\xff\x4d\xc1\x76\xf6\xce\x36\x6a\x67\x2c\xcb\xc0\x72\x9c\x4a\xec\x96\x73\x7c\xe3\x9e\x23\xf1\xb5\x28\x4a\x22\x31\x9e\x5f\x8f\x6f\x02\xa7\x26\xbd\xa7\x99\x1d\xda\xd7\xda\xf3\x3d\xba\x49\x8e\x53\x2d\xfa\xe2\xa1\xa9\xa1\x6d\x8d\x27\x06\x86\x27\xf7\xe3\xbf\x1c\x15\x8e\xfa\xdb\x9d\x12\x26\xbd\xe6\x99\xda\x1c\xc2\x52\x80\xc6\xa2\xcc\x0d\xe8\x21\xa7\xa6\x57\x19\xf4\xcf\x44\x51\x0a\x8e\x5c\x03\x53\xc0\xc5\xe1\x77\x88\xe6\x04\x0f\x63\xf2\x96\x71\x2a\xb6\x06\xd1\xde\x19\xe7\x37\x24\x3f\xa4\x9d\x9f\x55\xf4\x32\xbd\x55\x82\xc7\xd1\x7e\x9f\xce\x89\xc2\x6f\x25\xd1\xab\xaa\x1a\x9a\xf0\x0c\xf7\x7b\x6f\x64\xa9\x2a\xf3\x7d\x43\xa4\x4a\x19\xbd\xab\xaa\xa8\x1f\xbe\x46\xb7\xbb\xab\xfa\x70\x31\x1a\x8d\x0e\x79\x65\x1a\x18\xc9\x73\xd7\xa3\xec\x3b\x6a\xbe\xeb\x3d\x83\xf9\xfa\xcd\xdf\x3d\xe5\xdb\x1f\xe5\xdd\xfb\x7f\xe7\x67\x80\xff\x04\x00\x00\xff\xff\x86\xe7\xae\xf6\xd4\x1f\x00\x00") - -func webTemplatesMonitorDetailsGoHtmlBytes() ([]byte, error) { +var _bindataWebTemplatesMonitorDetailsprocessorgohtml = []byte( + "\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xec\x3c\x5b\x73\xdb\x36\xb3\xef\xf9\x15\x5b\x9e\x07\x91\x8d\x4c\xfa\xd2" + + "\x27\xdb\xd2\x19\xa7\xe9\x39\x4d\x27\xb7\x69\x3d\xa7\x0f\x3e\x1e\x0f\x44\x42\x12\x62\x0a\xe0\x00\xa0\x64\x55\xa3" + + "\xff\xfe\x0d\x00\x5e\x40\x90\xd4\x25\xc9\xd7\x26\xdf\x58\x0f\x8d\x85\x5d\x2c\x16\xd8\xfb\x02\xea\x66\x93\xe0\x29" + + "\xa1\x18\xbc\x98\x51\x89\xa9\xf4\xb6\xdb\x17\xd7\x09\x59\x42\x9c\x22\x21\x46\x1e\x67\x2b\x6f\xfc\x02\xc0\x1e\x53" + + "\xa8\x88\x50\xcc\x35\xc4\x85\xa5\x27\x8b\xe4\xe4\xec\xbc\x84\xcd\xcf\xc6\x9b\x4d\x28\x89\x4c\xf1\x76\x7b\x1d\xcd" + + "\xcf\xc6\x2f\x5a\x93\x32\x44\x71\x0a\xfa\xbf\x27\x09\x9e\xa2\x3c\x95\xc5\xf4\x7e\xbc\x39\x46\x09\xa1\xb3\x0a\x4f" + + "\x2d\x75\x31\xfe\x88\xb8\x24\x92\x30\x0a\xb7\x68\x92\x62\xf0\xff\x90\x48\xe2\xe0\x3a\x9a\x5f\x54\x04\xa3\x84\x2c" + + "\x0b\x26\x3a\xc8\x9f\x4c\x58\xb2\xb6\xa9\x4a\x4d\xa8\x40\x31\x5f\xf4\x7f\x4f\x84\xe4\x24\xc3\x89\x85\xab\xb0\x15" + + "\x5f\xf6\x88\x1a\xe3\xcd\x01\x8d\x06\xfa\x48\x46\x5e\xc5\xb1\x07\x9c\xad\x44\x86\xe8\xc8\x3b\xf7\xea\x8d\x5c\x47" + + "\x72\xbe\x63\xba\xda\x5f\x2e\x80\x4d\x41\xce\x31\x64\x9c\xc5\x58\x08\xc6\x9b\xc4\xf4\x21\x1c\x40\x48\x40\x92\x73" + + "\x42\x67\xc0\x71\xcc\x96\x98\xaf\x3d\x88\x59\x6a\xc8\x5c\x78\xe3\xdf\x8b\xd1\xa3\x28\xe5\x14\x24\x59\x60\x8b\xd2" + + "\xb9\x37\xfe\x93\x13\x89\x45\x9b\xce\x75\xe4\x9e\xd5\xee\xc3\xbb\x25\x0b\xac\xf6\x9e\x21\x21\x70\x52\xb1\x0d\x82" + + "\xc1\x14\x71\x6f\xac\xe0\x7b\xb8\x7d\x9f\x2f\x26\x98\x2b\x22\x0b\x2c\x04\x9a\x61\x51\x92\xc1\x09\x64\x98\x83\xc0" + + "\x31\xa3\x89\x37\xfe\x7d\xff\x19\x76\xd0\x4a\xd1\x6c\xa6\xce\x61\x82\xe7\x84\x26\xf0\xeb\x9f\xef\xbc\xf1\x5b\x34" + + "\x3b\x9e\x50\x9e\x25\x48\x7e\x09\x4b\x93\xb5\xec\x21\xf3\x6a\x7d\x90\x30\x14\x86\xa3\xdc\xd7\x52\x59\x0b\x90\xa4" + + "\xb0\x0c\x25\x7a\x22\x24\x89\x85\xe7\xcc\x54\x78\x96\x51\x45\x1a\xdd\x31\x49\x70\xac\xf3\xdf\xe3\x22\x7e\x63\x84" + + "\x8a\xe8\x2d\x63\x8f\x79\x26\x3a\x1c\x43\x0f\xcd\xbf\xdb\x2f\xdc\xae\x33\xad\xda\x4d\x4b\x56\xa3\x7b\x24\x7e\xcb" + + "\x32\x12\x3b\xb3\xd4\xd0\x9e\x69\x5f\xd5\x0f\x69\xe7\xfb\x2d\xfb\xa0\x9f\xbc\xf1\x1b\x1a\xb3\x05\xa1\x1d\xa6\xf8" + + "\xec\x85\xbe\x21\x2f\xd4\xa0\x43\x2b\x3a\x6c\x3a\x15\x58\x8a\xa1\x56\xb7\x25\xc1\x2b\x20\xd5\x36\xbd\xf1\x07\x0d" + + "\x85\xfd\x5b\x54\x6a\x01\x09\x4e\xd1\xba\x87\x92\x96\x6d\x27\x9d\xe3\xdc\xe3\x27\x46\xa8\x56\xca\x6f\xd7\x31\xbe" + + "\xa1\x59\x2e\xe1\x96\xa3\xe9\x54\x79\x8b\xda\x31\x02\x5c\xe7\xa9\xde\x05\x51\x28\xda\x9b\xd8\xfb\xb8\x8e\xf2\xb4" + + "\x4e\xa8\xbe\x2d\x2f\xfa\x79\xce\xf0\x2d\x12\x12\x8c\x12\x35\x27\x17\x8a\xe5\x2f\x08\x8d\xd0\x72\x16\x2d\xd0\x53" + + "\xb0\x87\x96\x32\x0f\xa5\xb1\x78\x89\xa9\x14\xcd\x3c\xe8\x17\x35\x06\x07\x18\x50\x49\x44\x9b\x4f\x93\x86\xb2\x9b" + + "\x43\x48\xbc\x45\x33\x98\x32\x5e\x58\x0e\x20\x9a\xb8\x4e\xf1\x42\x3b\x05\xf0\x0d\x46\xa4\xa0\x1d\x9b\x3b\xd6\x3b" + + "\x9a\x3d\x72\xb5\x01\x42\x41\x32\x89\x52\x25\x04\x89\xd2\x3d\x0c\x5b\x13\x95\x9f\xc8\xaa\x00\x35\xfe\x88\x39\x1c" + + "\x1a\x98\xf4\xf1\x7c\xc6\xea\xf5\xbc\x2f\x58\xfc\x1d\xa1\xca\x79\x78\xe3\x77\x64\x1f\xea\xcd\x72\x66\x50\x6f\x96" + + "\xfb\x9c\xd6\x3b\xf4\x54\x50\x45\x4f\x5f\xea\x97\xb4\x45\x7f\x0f\x69\xdb\x87\x5c\xce\x98\x8a\x61\xcf\x0e\xea\x9f" + + "\x75\x2a\xcf\xf6\xbf\x73\xf1\xe3\xcc\x8f\xe5\xf2\xeb\xdb\x9f\x88\x39\xc9\x24\xc8\x75\xa6\x32\x1c\xfc\x24\xa3\x4f" + + "\x68\x89\xcc\xa8\x57\xd9\xc1\x12\x71\x48\x91\x90\x6f\x4a\x0f\x20\x60\x04\xc9\x45\x98\xb2\x18\xa5\x7e\x70\xe5\xa0" + + "\x7d\xa8\x38\xdd\x8d\x77\x5b\xd6\x81\xbb\xd1\x7e\x2b\xf3\xa1\x5e\x2c\xa1\x2b\x8a\x77\x28\x83\x11\x6c\x4e\x2f\x3d" + + "\x21\x59\xa6\xec\x6d\x08\x67\x97\x1e\xa1\x44\x12\x94\x92\xbf\x94\xa3\x18\xc2\xf9\x25\x78\x45\xbe\x6c\x06\x2e\x2e" + + "\xbd\x8c\xe3\x0c\x15\x5f\x7f\x52\xf0\x9c\x52\xf5\x6d\x7b\x65\x9f\x80\xc0\xb1\xb8\x65\xef\x11\x65\x30\x82\xb3\xd3" + + "\xf2\xd3\xc0\xe1\x98\x26\x98\xbf\xc6\x12\x91\x54\xf1\x3b\xcd\x69\xac\x84\xef\x57\x5a\x21\x82\xcd\x8b\x4a\x4a\x64" + + "\x6a\x01\x60\x34\x02\x9a\xa7\x69\xb0\xb1\x04\xcb\xb1\xcc\x39\xbd\xaa\x46\xb6\xf5\x64\xb5\x9e\xc9\x8e\x6f\x9b\x05" + + "\xb5\xbd\xae\x3a\x9a\xc6\x92\xf6\x89\xc1\xa8\x3e\xba\x3b\x8d\x19\x9a\xe2\xec\xfe\xca\x9e\x40\xa6\x7e\x05\x4c\x53" + + "\x9c\x34\x18\x84\x9a\x96\x27\x0c\xdc\xbb\xb2\xe0\x5b\x67\x65\x8e\xe3\x5f\x68\xa2\x73\x65\xb3\xba\x08\xcb\x42\xad" + + "\xfa\x43\x43\x7f\x18\x81\xe7\xc1\x7f\xef\xc4\xb9\x2c\xa0\xef\xd9\xea\xaa\xbd\x4e\x8d\x37\x02\x9f\xe2\x15\xbc\x46" + + "\x12\xfb\x35\x03\x01\x9c\x40\x35\xec\x2c\xf3\x87\x44\x5c\x6a\xa4\x00\x22\x2d\xed\xf0\xb4\x6f\x09\xed\x1c\x47\xd0" + + "\x0b\x57\x99\x52\x6b\xaf\xbf\xae\x16\x70\xe2\x0e\x16\x39\xe3\x95\x2b\xae\x15\x27\x12\xf7\x2d\xa3\x81\xba\x24\x32" + + "\x50\x0b\x1c\x45\x90\x62\x39\x10\x10\xcf\x71\xfc\x08\x64\x0a\x2b\x0c\x73\xb4\xc4\x80\x20\xe3\x78\x49\x58\x2e\x20" + + "\x41\x12\x95\x69\x9e\xfa\x57\xce\xf1\x02\x24\x53\xff\x12\xde\x24\x16\x33\xce\x71\x2c\x61\x89\xd2\x1c\x0b\x87\x0f" + + "\x65\xab\xa5\x9d\x36\xcd\x3b\x9c\x61\xe9\xcb\x39\x11\x81\xab\x57\xd5\x1c\x47\xa7\xa2\x08\x12\x32\x9d\x2a\x36\x2a" + + "\x3e\xf5\x59\x29\xd7\x6c\xca\x40\xd1\x98\xa0\x18\x50\x59\xe8\x6b\x35\x6b\x04\x7e\xa5\x17\x70\x52\x33\xa6\xbe\xdb" + + "\xe2\x6c\x50\x28\xb4\x3c\xd7\x76\x68\xbb\x08\x87\x35\x70\xe5\xee\xaa\x4e\x21\x44\x7b\x5d\x07\x16\x44\x25\xab\x57" + + "\x0d\xd2\xdb\x26\x47\xb6\xd4\x8b\x45\x4c\x37\x34\xfc\x99\xe5\xb4\xb9\x82\x0d\xe8\x23\xdf\x50\x94\x26\x41\x33\xda" + + "\x41\x50\x03\xba\x09\xda\x96\xed\xc8\x5b\x14\xf2\x1e\x1a\xa1\x35\xc5\x6e\x5c\x1a\x0c\xae\x65\x32\x1e\xbc\x34\x6c" + + "\x54\x5e\x30\x94\xec\x7f\xc8\x13\x4e\xfc\xd3\xe0\xe5\xe0\x3a\x92\xc9\xf8\xff\xe9\xe0\x65\x63\x1b\xd6\xc4\x5c\xec" + + "\x41\xb2\xbd\xc0\xcb\x81\x38\x08\x57\x9d\xf8\xe1\x6c\x58\x46\xbe\x07\xb3\x12\xe7\xe1\xc4\x6b\x89\x75\xce\xb1\xc2" + + "\xc2\x55\x33\x2e\xcc\x38\xcb\xb3\x9f\xe7\x88\x50\x18\xc1\x43\x18\xab\xbf\xac\x50\x13\xfe\xaf\x82\x07\x57\x8d\x39" + + "\x15\xf8\x2d\x11\x12\x46\x16\x0d\x47\xff\xc3\x29\x49\x25\xe6\x7e\x15\x64\x96\xc1\xa6\x10\xea\x32\xb4\x82\xfa\x0f" + + "\x26\x9e\x5d\xc1\x36\x70\x29\x2c\x50\x66\x4d\x1f\xea\xb5\xdf\xb8\x81\xc5\x52\x96\x87\x10\x09\x41\x66\xd4\x6f\x21" + + "\x00\x6c\xb6\xc3\x8e\x51\x9b\x93\x2e\xf8\x06\x06\xef\xd9\x6a\x70\x59\x47\x80\xa5\x76\x0f\x5d\xb8\x00\x83\xea\x70" + + "\x06\x70\xa9\xb8\x15\xf8\x0d\x95\x7e\xc1\x76\xd7\x9c\x0e\xa6\x82\x2b\x67\x68\x1b\x84\xc6\x93\xfa\x41\x28\x18\x97" + + "\xaf\xd6\xbe\xb5\x50\x01\xf4\x6d\xeb\x89\x22\x50\x88\x80\xd2\xb4\x96\x97\xb0\xc1\x26\x23\x18\x02\xa6\x12\x73\xed" + + "\xd3\x39\x5e\xb0\x25\xd6\x6e\xbe\x21\xef\xc4\xe4\x53\x02\xa7\x38\x96\xbe\xf7\x5f\x6e\x5b\x3e\x28\x40\x37\x69\xea" + + "\x7b\xb5\x85\x4e\xd8\x93\x17\x84\x8a\x9a\xdf\xd0\x18\x6b\x77\x49\x38\x97\x8b\xd4\xef\x4c\x4e\x1a\x68\x9a\x49\x3f" + + "\x08\x51\x96\x61\x9a\xf8\x9e\xe4\x5e\x10\xea\x02\x0a\x27\xbe\xd7\x58\x72\x08\x92\xe7\x38\x38\x98\xf2\x13\x91\x7e" + + "\x10\x9a\xbd\xfb\x41\xd3\x3a\x4c\x96\x66\x9a\x58\x56\xae\xa4\xf6\xd4\x50\x41\x1d\x54\x54\x89\xa5\x4e\x0a\x49\x74" + + "\x77\x7a\xef\x06\x5f\x51\x26\xa6\x0a\x7c\x76\xef\x46\xdf\xa2\x90\x2d\x02\x26\xac\xf0\x80\x63\x30\xc5\xa9\x64\x90" + + "\x10\x91\xa5\x68\x1d\x36\xe7\x28\xc7\x09\x6c\xf2\x49\x05\xda\x15\x49\x53\x98\x60\x10\x92\x71\x9c\xa8\xe0\xe7\xe4" + + "\xe1\x92\x41\x8c\xd2\x38\x4f\x51\x51\x83\x08\x40\x6a\x9d\x34\x75\x18\x2d\x38\x18\x41\xd3\xc6\x3c\xca\x56\xde\x65" + + "\x65\x03\x8e\x2a\x7b\xb1\x8a\x27\x7f\xe4\x0b\x0f\x2e\xe1\x21\x14\xf9\xa2\x08\x1b\x26\xce\x74\x21\xdf\x2c\x67\x06" + + "\x79\x81\x11\xdd\x85\xad\x4b\xc9\x0e\xd2\x26\xe2\x74\x21\x77\x91\x6e\x63\x2b\x57\xe8\xec\x5d\x33\xd6\x97\x3e\x55" + + "\xc0\x9b\xe5\xac\x0b\xae\x97\xee\x9b\x5c\x01\xab\xc9\x7b\x92\xa2\x5a\x74\xff\x40\x52\x64\x74\x20\xa4\x4e\x56\x44" + + "\x77\x67\x45\x25\x85\x96\x77\xb6\x8f\xb5\xa4\x5d\x2a\xcc\x49\x4d\xbe\x1c\xea\x4b\x4b\xc0\x15\x41\x83\xd6\xcd\x72" + + "\xe6\xd2\xba\x59\xce\xfa\x69\xd9\xe2\x2a\x09\x95\xaa\x66\x11\x2a\x87\x0e\x20\xd4\x64\xaa\xd4\x44\x97\xd6\x0e\xa6" + + "\xb6\x3d\x49\x53\x14\x01\x4a\x12\x1d\x80\x0a\xeb\x9c\x32\xae\xef\x16\x28\x7e\x92\xc0\x59\x4e\x13\x27\xc7\xb2\xd4" + + "\xa7\xce\xb1\xcc\xe4\xa0\xa5\x7b\xfa\xb2\xa2\x65\xf1\x0b\x42\x8d\x11\x91\xd2\x86\x5e\x2b\xbc\xc0\xca\x30\x0a\x65" + + "\x30\x9f\xa6\x25\x2a\xd3\x73\x6d\xf0\xa8\xf9\xe8\xc9\x4c\x47\x4f\x47\xcf\xde\xf6\x67\x91\xda\x4b\x77\x27\x53\x05" + + "\x86\xbd\xe1\xb7\x48\xc8\x22\x13\xb7\xf3\x2a\x2f\xf2\x5e\x36\x36\xb6\x1b\xaf\xda\x41\x0f\xda\x2e\x6e\x2a\x8d\x3f" + + "\x7a\xc2\xcd\x72\x76\xe8\x9c\x4a\x83\x8f\x9e\x70\xc4\x22\xf6\xb9\x9a\x43\x78\x8b\x66\xcd\x73\x88\x06\x2f\xb5\x2a" + + "\x2a\xcc\x0a\x70\x1e\xf4\xe4\xe3\x35\xdd\x5a\x10\x7b\x09\x63\x74\x14\xe5\x4a\x74\x7b\x09\xa3\xa7\x6e\xba\x96\xb1" + + "\x6d\x9b\xf9\xd9\x8a\xf1\x47\x15\xdf\x19\x2d\x83\xb8\x67\x32\x8d\x39\xe6\xd8\xbb\xac\x51\x7f\xac\xfe\xb4\x0d\xd4" + + "\xd3\xa9\xb9\x77\xe9\xc6\xe9\xd3\xd6\x90\xca\xa3\xd5\xc7\x19\x34\xab\x75\x61\x83\x97\x0b\xcc\x4f\xe2\x94\xc4\x8f" + + "\x0a\xee\xe9\xb8\xec\x5d\xc2\xd9\xf9\x50\x91\xda\xb6\x26\xb8\x23\x4e\x52\xbb\x1d\x76\x7a\xb6\x1f\xa3\x46\xb6\x85" + + "\x66\x33\x8e\x67\x48\x9a\x5c\xfc\xd5\xfa\xb6\xc8\xa8\xda\xbd\x31\xbd\xf5\x21\x08\x96\xf3\x18\x17\xb5\xe3\xff\x19" + + "\xe7\xb6\x71\x12\x2b\xff\x2c\x80\x19\x36\x89\xb0\xf2\x98\x55\x1c\x44\x90\xaa\xf2\x65\xa5\x4e\x5b\x43\x4c\x02\x47" + + "\xd1\x02\x03\xd1\x18\x6a\xf0\x11\xaf\x61\xa0\x21\x83\xae\x7c\x4f\x33\xd2\x55\x39\x99\xba\xa9\x55\x0f\xa5\x48\xbe" + + "\x43\x59\xbb\x38\xa9\x4b\x1c\xb5\x8b\x8e\xf2\xc6\x2a\x70\x54\x49\xa4\xd1\xee\xcc\xfe\xef\xbb\x2b\x91\x9e\x05\xf4" + + "\xfe\x87\x9a\xf9\xce\x65\xda\x1f\x3d\xe3\xce\x7b\xd0\x73\xbc\x7b\x18\x99\xad\xbb\x61\xb0\xfb\x53\x70\xad\x69\xec" + + "\x9b\xb1\x6d\xd5\x3d\x6d\xc5\xb2\x0a\x21\x07\xa2\x13\x62\x4c\x4d\x35\x0a\x44\xc2\x64\x6d\x42\x24\x5a\x60\x07\x55" + + "\xa3\xbc\x5a\x77\xd6\xa5\xf5\x46\xaf\xb6\x5d\x85\x95\x5e\x48\x25\x46\x45\x83\xcc\x06\xfc\x68\x7d\x71\xac\xb2\x61" + + "\x50\x77\x0e\x3f\x6d\x03\x73\x45\x6a\x61\x5c\x74\x63\xb8\xf6\x7d\xdf\x6d\x72\xda\xe8\x1c\x0b\x39\x0f\x8a\xe6\x5f" + + "\x65\x80\xe6\xdd\x82\x49\x35\x24\x9b\x61\x39\xc7\x5c\xe7\x1c\x18\xc5\x73\x23\x7e\xd1\x24\xb2\xc2\x03\x65\x60\x1c" + + "\x51\xc1\xf8\xc2\xb4\x06\x05\xb3\xf0\x61\x8e\x04\x20\x5a\xd7\x2b\x72\x5e\x9a\x20\x9b\x1a\xfd\xe8\x32\xb0\x32\x25" + + "\xd6\x6a\x6f\x4c\xdc\xaf\x2d\x6f\x58\xeb\x75\x8d\xed\xa8\x75\x65\x37\x9a\xb9\x29\xe3\x0b\x0b\xd7\x3d\x45\x95\x69" + + "\x15\x87\xc0\xb8\x72\x03\xb1\x6e\x54\x9b\xbd\xeb\x07\x38\x05\xff\x7e\x87\xd0\xc0\xb5\xf7\x8a\x39\x14\xc7\x45\xf6" + + "\xd5\x61\x73\x51\x04\x33\x06\x6c\x89\x4d\x4a\x87\xe2\x38\x2f\xcf\x5e\x55\xe5\xa6\xe4\x35\x27\xa9\x87\x61\xca\xd9" + + "\xa2\xc8\xfe\x56\x05\x3f\xae\x31\x00\x3c\x84\x53\xc6\x7f\x41\xf1\xbc\xab\x13\xf2\x10\x3e\xe2\xb5\x50\x6c\xf5\xb4" + + "\x31\x2a\xce\x1f\xf1\xba\xd7\x4d\xa0\x38\xbe\x7b\xc4\xeb\xfb\x30\xcb\xc5\xbc\x70\x49\xea\x7b\x87\x19\x43\x87\x29" + + "\x77\xf5\x39\x3a\xe4\x81\x17\x99\x5c\xeb\x53\x59\xa8\xe2\x95\xf1\xcb\x62\x48\xe9\x8e\xc9\x84\x95\x6f\xaf\x0a\x67" + + "\x58\x21\x2a\x5f\xb8\xfb\xfd\x8b\x64\x1f\xf4\x49\xf9\x56\xb0\x18\x16\xde\xb4\x31\x54\x6d\xbd\xf2\x09\x77\xca\x15" + + "\x34\x65\x6b\x7f\xdb\x76\x76\x2d\x6b\x1d\xeb\xbd\x93\xa9\x51\x8a\x46\xda\x43\x28\xd9\x47\x44\xb8\xf0\x3b\x43\x61" + + "\xab\x2f\x37\x2c\x63\xf8\x10\xee\x0a\x6d\x1c\x82\xbe\x65\x14\xea\x8f\x2a\x67\x51\x5f\xea\xdc\x53\x7d\xd3\xb9\xb4" + + "\x77\x1f\x38\x2d\x23\xcb\xaf\x59\xb6\xca\xa6\x26\x7a\x6a\xb3\x1f\xba\xa6\x6b\x1d\xbe\xcf\x28\xae\xdd\x44\xc5\x6d" + + "\x60\x2f\x61\x5a\x2a\x5a\x7d\xf5\xc5\xba\x36\xfd\xe2\x3a\xaa\x88\xb9\x66\x7c\xc2\x9e\x76\x76\xa1\xdc\x57\x06\xfb" + + "\xbb\x50\xcd\xf3\xb6\x24\x9d\x04\x9b\x52\x6c\xc9\xdd\xe9\xfd\x55\x23\x12\x7d\x7e\xf3\xc9\xea\x1e\xb5\x7b\x5e\x7d" + + "\xc0\x56\x3f\xaa\xa3\x21\x65\xee\x4d\x9f\x3b\x52\xcf\x1d\xa9\xbf\xb3\x23\x65\xdd\xd6\x3f\xb7\xa4\xda\x84\x9e\x5b" + + "\x52\x7b\x5a\x52\xb6\xfe\xec\xee\x49\x1d\xdb\xb2\xf9\x4e\x9b\x24\xfd\xad\x01\x2b\x48\x9a\xd7\x3c\x1d\x51\xb2\x00" + + "\xb8\x61\xb2\x7e\xfd\xf3\x05\x59\x85\x11\x55\x57\x5a\xd1\x4c\x19\xba\xe2\x72\xeb\xf9\xd1\xfe\xc0\xec\xf0\xfc\x77" + + "\x45\x66\xb3\xcb\xbe\xd0\xdc\x01\xdd\x1d\x9b\xcd\x5d\x93\xfd\x08\x69\xd7\xe3\x9a\x28\xd2\x32\xc5\x29\x16\xa2\xfe" + + "\x8d\x95\x80\x84\xd1\x81\x34\x19\x98\x79\x19\xe8\x96\x46\xe5\x23\x9b\xce\x57\x40\xed\x77\x40\x4e\x37\xb4\x7e\xee" + + "\x30\xaa\x2f\x34\xdb\x1d\xe1\xe7\x67\x3f\xdf\xd5\xb3\x1f\x8e\x51\xd2\xbf\x0a\x4a\xbe\xc1\x47\x3f\x95\x9d\x3c\xbf" + + "\xf9\x39\xf4\xcd\x8f\xc3\x50\xb5\xa3\xf1\x69\x07\x17\x95\x42\x14\x1c\xe8\xc4\xbb\xe3\x3d\x90\x35\xde\x9f\x4f\xd8" + + "\x3a\xd4\xa0\xd7\x7e\x0e\x64\x8d\x1f\x9b\x53\x34\xd5\xe2\xe8\xa7\x41\x72\x9d\xe1\x03\x9e\x02\x89\x70\x47\x42\x01" + + "\x3d\xaf\x8d\xfe\xd3\x9e\x18\x95\xfa\x71\x0c\xed\x42\x05\x8e\x7b\x78\x55\x2a\x44\xd5\x8b\x38\x7c\x7a\x43\xd1\x74" + + "\xc3\x22\xaa\xaf\xfb\x82\xbe\xdb\x96\x4a\xb1\x9a\x19\x15\x7e\x92\x1c\xc5\xe6\x1a\xe0\x13\x23\xd4\xf4\xd6\xf4\x13" + + "\x19\xed\xd1\x96\x04\x15\xf5\xb2\x1a\xaf\xe4\x7e\xf2\xe6\x75\x23\xc6\xab\xa9\x38\xa9\x12\xab\x9e\xe7\x52\x55\x9f" + + "\xbf\x7d\x73\xd1\xf3\x82\xa9\x6a\x52\x3a\x14\xc3\xdf\xf4\x82\xad\x66\x77\xf9\x69\x3c\x8f\xca\x86\xb0\xaf\xaf\x5f" + + "\xa4\xed\xf5\x75\x87\xfe\xc9\xce\x1c\xd7\x3b\x2e\x7c\x7b\x51\xc5\xf7\x12\xca\xee\x8a\x5b\x91\x03\xee\x03\xb2\x3b" + + "\xeb\xb1\x92\xc2\x37\x47\xb0\x73\x82\x32\x66\x8d\x3b\x50\x47\x3e\xe8\xc7\x2d\x4e\x2e\xeb\xc3\xd8\xf6\x9e\x5d\xdf" + + "\x45\x42\x03\xee\x5b\x09\xa0\x75\x1f\xa0\x55\xaa\x38\x3a\x2d\xb2\x66\x2a\x98\xea\x5f\x02\xf7\xab\x89\xf9\xa5\x70" + + "\x87\x9e\xe8\x94\xaf\x5b\x8c\x8e\x8a\x68\xcc\xf0\x63\xfd\x5a\xfc\x50\x15\xe9\x7d\x41\x57\x7e\xbe\x43\x1d\x31\xe7" + + "\xfd\xbd\x69\xc9\x02\xf3\x99\x71\x26\xe6\x86\x23\x66\x34\x46\xd2\xaf\x7d\xcc\xd0\x52\xa4\xc0\xee\x0d\x95\x77\x5d" + + "\xa5\xff\x32\xcd\xd8\xc9\xba\xcb\x85\x55\x93\x9a\xcb\x59\xdf\xf4\x93\xc2\x5a\x47\xd0\x70\xe2\x68\x87\x49\x23\x45" + + "\x9e\x2a\x75\x46\x3a\xd0\x9a\x5f\x57\xe0\x9f\xd9\x22\x43\x1c\xfb\x13\x3d\xe8\xf4\xf3\xc9\xd4\x2f\x67\x8d\xa0\x2b" + + "\x47\xa9\x29\x2a\xb6\xdb\x24\xb5\x15\xec\xc9\x82\x0e\x5c\xa1\xd6\xdd\x13\x98\xd4\xdf\x76\x11\x2f\x74\xc6\x10\x69" + + "\x94\x2c\xae\x2c\x3e\xff\x29\x65\xfd\x13\xde\xfd\x55\xb2\x25\xb1\xbe\x27\x94\x55\xf6\xf4\x15\x1f\x4f\xf6\xd0\xec" + + "\x29\x85\xeb\xb0\xbb\x22\x34\x61\x2b\x95\xc6\xbd\x51\xeb\x2f\x51\x5a\xab\x58\x60\xb5\x72\x93\x8b\xf0\x93\x60\xd4" + + "\xf7\x36\x9b\x70\x82\x04\x7e\xc8\x90\x9c\x6f\xb7\x91\xda\x73\x54\x15\xc6\xd1\x66\x13\x2e\x11\x17\x21\x49\x9e\xb6" + + "\x5b\x6f\xd8\xfc\x01\x4d\xc5\xda\x76\x08\xe7\x2a\x3b\xa8\xb8\x50\x65\x89\x8a\xf9\x44\x42\xf1\xfb\x9e\x74\xfd\xe2" + + "\xab\x2e\xac\xa9\x5d\x47\xe6\xa7\x50\xfa\x7f\x6b\x63\x7e\x37\x55\xfc\xb3\xd9\x60\x9a\x6c\xb7\x2f\xfe\x15\x00\x00" + + "\xff\xff\xf4\xaf\xc5\x50\x16\x47\x00\x00") + +func bindataWebTemplatesMonitorDetailsprocessorgohtmlBytes() ([]byte, error) { return bindataRead( - _webTemplatesMonitorDetailsGoHtml, - "web/templates/monitor/details.go.html", + _bindataWebTemplatesMonitorDetailsprocessorgohtml, + "web/templates/monitor/details_processor.go.html", ) } -func webTemplatesMonitorDetailsGoHtml() (*asset, error) { - bytes, err := webTemplatesMonitorDetailsGoHtmlBytes() + + +func bindataWebTemplatesMonitorDetailsprocessorgohtml() (*asset, error) { + bytes, err := bindataWebTemplatesMonitorDetailsprocessorgohtmlBytes() if err != nil { return nil, err } - info := bindataFileInfo{name: "web/templates/monitor/details.go.html", size: 8148, mode: os.FileMode(436), modTime: time.Unix(1548705120, 0)} + info := bindataFileInfo{ + name: "web/templates/monitor/details_processor.go.html", + size: 18198, + md5checksum: "", + mode: os.FileMode(420), + modTime: time.Unix(1584119590, 0), + } + a := &asset{bytes: bytes, info: info} + return a, nil } -var _webTemplatesMonitorIndexGoHtml = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xd4\x95\xc1\x8e\xda\x30\x10\x86\xef\xfb\x14\xa3\x88\xf6\x54\x12\x15\xed\x69\x1b\x90\xda\xaa\x42\x6c\xd1\x06\xb5\xa1\xd7\xca\xe0\x81\x58\xf5\xda\x96\xe3\x40\x2b\xcb\xef\x5e\x39\x90\x6c\x48\xa0\x39\xd0\x5d\xb5\x17\xb0\x3d\xbf\xc7\xdf\x4c\xc6\x1e\x6b\x29\x6e\x98\x40\x08\xd6\x52\x18\x14\x26\x70\xee\x26\xa6\x6c\x07\x6b\x4e\xf2\x7c\x1c\x68\xb9\x0f\x26\x37\x00\xcd\x35\x2f\x25\x4c\xa0\x2e\x2d\x6d\x1b\x1f\x3e\xd2\xe1\xdb\xd1\xd1\x06\x10\x67\xa3\xc9\x42\xcb\x35\xe6\xb9\xd4\x79\x1c\x65\xa3\xca\x62\xed\x60\x45\x72\xfc\xae\x88\xc9\xe0\x6e\x0c\x61\x3d\x73\xae\x96\x68\x22\xb6\x08\x03\x26\x28\xfe\x7c\x03\x03\x55\x79\x2a\x37\xd4\xb3\xbc\xde\xd1\x87\xd3\x56\x28\x22\x90\x43\xf9\x3b\xa4\xb8\x21\x05\x37\x27\xda\x33\xea\x61\x86\x84\x32\xb1\x6d\xe9\x7c\xa8\xb7\xa7\x42\xc3\x0c\xc7\x60\x12\x13\xc8\x34\x6e\xc6\x41\x33\x62\xe7\xa2\x1a\x3f\xb2\xf6\x10\xa1\x73\xc1\xc4\xda\xa7\x28\xc3\xa9\x26\x2a\x0b\xa7\x5a\x16\xca\xb9\x38\x22\x9d\x23\xa3\xec\xb6\x85\x1b\x51\xb6\xeb\x8b\x60\x25\xe9\xaf\x0e\xbe\xb5\xb0\x67\x26\x83\xc1\xd6\x1f\xea\x13\xdc\x06\x81\x46\x9a\x8f\xae\x0b\xde\x76\xd3\xf8\x6a\x48\xb7\x38\xa3\xfe\xbb\xf9\x51\xe9\xb1\xf4\x1d\xce\x84\x2a\xcc\x57\xa3\x91\x3c\xe6\x1d\x9f\x00\x31\x67\x93\xd9\xc3\x62\x99\xde\xf9\x22\xf1\x7b\xc3\x54\x2a\xb6\x76\x0e\x5e\x6b\xa2\xf5\x3b\xb0\x56\x69\x26\xcc\x06\x82\x57\x69\x70\x70\x1f\x7e\x94\x14\xd7\x3e\x4b\x9c\x9d\x63\x42\x41\xcf\x1c\xd5\xcf\x7a\x2f\x99\x30\x29\x59\x71\xbc\x84\x7a\x9f\xcc\x1e\xfe\x09\xd4\xb9\x94\x3f\x0a\xf5\x47\xd6\x79\x92\x7c\x5e\x2e\x5e\x02\xb6\xaa\x25\x79\x24\x6a\x70\x4e\xeb\xc5\x0b\x94\xd3\x2f\xc9\x72\x91\xbe\xff\x30\xff\x54\x92\x3e\x39\xe9\xe1\x6d\x08\xaf\xa2\xe6\x52\xaa\xd3\xbc\xaa\x43\xb5\x5e\xce\xea\x21\xa7\x7e\x63\x0f\x63\x29\x79\xa6\x02\x48\x0a\xd3\x7b\xb1\x92\x65\xfa\x02\x37\x2b\x8e\xba\x2f\x83\xb5\x80\x82\xb6\x1f\x91\xce\x83\xd5\x5a\x38\x99\x36\x0f\xab\x0d\xc7\xc1\xd5\x6d\xea\x1b\xc3\xfd\xd5\x1d\x6a\xc7\x70\x5f\x6a\xfd\xe0\xff\xec\x4b\x9e\xbc\xdd\x92\xfc\x5a\x55\x2a\xcf\xda\x87\xfe\x5a\x39\x1c\xff\x2a\xc9\xef\x00\x00\x00\xff\xff\x33\x4f\x5a\x19\xeb\x08\x00\x00") +var _bindataWebTemplatesMonitorDetailsviewgohtml = []byte( + "\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xac\x58\xdd\x6f\xdb\xb6\x16\x7f\xcf\x5f\x71\x2e\xef\x83\x25\xc4\x96\xf2" + + "\xd1\xa7\xd8\xf2\xc5\x6d\xb3\xa1\x01\xd2\xac\x68\x83\xee\x61\x1b\x0a\x46\x3c\xb6\xd9\xc9\xa4\x40\x52\x72\x32\x41" + + "\xff\xfb\x40\x4a\x96\x28\xc5\x6e\x52\x6c\x79\xb0\xac\xf3\xf1\x3b\x87\xe7\x93\x71\x55\x31\x5c\x71\x81\x40\x52\x29" + + "\x0c\x0a\x43\xea\xfa\x64\xc1\x78\x09\x69\x46\xb5\x4e\x88\x92\x3b\xb2\x3c\x01\xf0\x69\x56\x94\x72\x81\xca\x71\xc6" + + "\xbc\x6c\xb6\x65\xb3\xf3\x8b\x3d\x6f\x73\xbe\xfc\xc2\x71\x07\x55\x15\x19\x6e\x32\xac\xeb\x45\xbc\x39\x5f\x9e\x3c" + + "\xd3\xcc\xa9\xc0\x0c\xdc\xe7\x8c\xe1\x8a\x16\x99\x69\x31\x8e\xcb\x6d\x90\x32\x2e\xd6\x9d\x9c\xb5\x77\xb9\xbc\xa7" + + "\x0f\x19\x82\x36\xd4\x70\x6d\x78\xaa\x17\xf1\xe6\xb2\x43\x8a\x19\x2f\x5b\xeb\x07\x70\x67\x0f\x92\x3d\xf9\x70\xc6" + + "\x61\xb5\x22\xcd\x8b\xfb\x9c\x69\xa3\x78\x8e\xcc\x93\xb5\xd2\xd6\x21\x9f\x62\x69\x6a\x48\x70\x62\xe0\x62\x91\x90" + + "\x8f\x54\x19\x6e\xb8\x14\x04\x94\xdc\xe9\x9c\x8a\x84\x5c\x90\x65\x47\x5e\xc4\x66\xf3\x1d\xf5\xcf\x86\x9a\x42\x83" + + "\x5c\x81\xd9\x20\xe4\x4a\xa6\xa8\xb5\x54\x43\x30\x2b\x84\xaf\x00\xd2\xc0\x0a\xc5\xc5\x1a\x14\xa6\xb2\x44\xf5\x44" + + "\x20\x95\x59\x03\x73\x49\x96\x9f\x5a\xea\x0f\x21\x15\x02\x0c\xdf\xa2\x87\xf4\x86\x2c\x6f\x44\x2a\xb7\x96\x7d\xaf" + + "\xe8\x6a\xc5\xd3\xe7\x88\x8b\x78\x1c\xb5\xef\x87\xf1\x9e\x6f\xd1\x46\x21\xa7\x5a\x23\xeb\x0e\x00\x5a\xc2\x8a\x2a" + + "\xb2\xb4\xfc\x17\xfc\xbe\x2b\xb6\x0f\xa8\x2c\xc8\x16\xb5\xa6\x6b\xd4\x7b\x18\x64\x90\xa3\x02\x8d\xa9\x14\x8c\x2c" + + "\x3f\xbd\x1c\xcd\x03\x58\x19\x5d\xaf\xed\x91\x1f\x70\xc3\x05\x83\xf7\xbf\x7e\x20\xcb\x5b\xba\xfe\x71\xa0\x22\x67" + + "\xd4\xfc\x13\x97\x1e\x9e\xcc\x11\x98\xb7\x96\xf3\x02\x8e\xe8\x70\xe4\x6a\xa5\xd1\xe8\xa9\x2b\xbd\xd2\xb6\x38\xef" + + "\x8e\x49\x96\xbf\x38\x2e\xbc\x7c\x44\x5b\x1e\xc0\x30\xa3\x4f\x47\x90\x5c\x6e\x0f\xe2\x8c\xab\xc4\x4a\x8c\xfa\x6f" + + "\x61\x6c\x43\x03\x67\x09\xb1\xc0\x9f\xbb\x99\x40\x46\x8a\x56\xcc\x6b\xfb\xd8\x35\xf9\x68\x68\xc0\x68\x7e\x2c\x74" + + "\xaa\x78\x6e\xc0\x3c\xe5\xf6\x20\xf8\x68\xe2\x6f\xb4\xa4\x0d\x95\x74\x43\xa6\xa4\x0a\x32\xaa\xcd\x97\xd6\x01\x0d" + + "\x09\xb0\xcb\x28\x93\x29\xcd\x82\x70\xee\x49\x69\xd7\xce\x1f\x68\x0e\x09\x54\x67\x57\x44\x1b\x99\xdb\x19\x33\x85" + + "\xf3\x2b\xc2\x05\x37\x9c\x66\xfc\x2f\x1b\x95\x29\x5c\x5c\x01\x69\x0b\xb4\x21\x5c\x5e\x91\x5c\x61\x4e\xdb\xd7\x37" + + "\x96\x5f\x08\x61\xdf\xea\xb9\xef\x8b\xc6\x54\xdf\xcb\x3b\x2a\x24\x24\x70\x7e\xb6\xff\x1b\xc8\x28\x14\x0c\xd5\x35" + + "\x1a\xca\x33\xeb\xef\xaa\x10\xa9\x9d\x47\x41\xbe\x9f\x4c\x3a\xac\x4e\xba\x78\xf1\x95\xc7\x80\x24\x01\x51\x64\x59" + + "\x58\x79\x21\x56\x68\x0a\x25\xe6\x1d\xa5\xee\x95\xad\xbd\xa6\x1c\xbf\x0c\x52\xe4\x9b\xb5\x91\x19\x58\xf4\x03\x06" + + "\x49\x1f\xb9\xdf\x9c\x64\xd4\x0c\xc6\x3f\xe6\xbe\x02\x5f\x05\x1d\x33\xcb\x90\x0d\xfc\x83\x1e\x8b\xe8\x86\x4f\xe6" + + "\x1e\xbf\x1e\x59\x56\x98\xfe\x24\x98\xab\xcd\xc6\xba\x8e\xf6\x03\xb2\xfb\xe2\xb8\xff\x49\x80\x10\xf8\xdf\x77\x65" + + "\xae\x5a\xee\x9d\xdc\xcd\x9f\xdb\xe9\xe5\x12\x08\x04\xee\xe0\x9a\x1a\x0c\x7a\x07\x42\x98\x41\x47\x1e\x99\xf9\x6c" + + "\xa8\x32\x4e\x28\x84\xd8\x25\x3b\x3a\x3b\x66\xc2\x4e\x11\x48\xe0\x28\xff\x96\xae\x9f\x9f\xf5\xfd\x6e\x0b\xb3\x31" + + "\xb1\x69\xfe\xf9\x38\x5b\xa9\x2c\x84\x39\x66\xc5\x8e\xa5\x9e\xe7\xab\xc6\x31\x64\x68\x26\x1a\xd2\x0d\xa6\x7f\x02" + + "\x5f\xc1\x0e\x61\x43\x4b\x04\x0a\xb9\xc2\x92\xcb\x42\x03\xa3\x86\xda\x81\x43\x05\x03\xfb\x34\x1b\xdc\x82\x91\xf6" + + "\xc9\xd5\x10\x2c\x95\x4a\x61\x6a\xa0\xa4\x59\x81\x7a\xe4\x86\x6d\xd4\x7d\x93\x0e\x9a\x36\x5a\xa3\x09\xcc\x86\xeb" + + "\x70\x5c\x54\x9d\xca\xa8\xa0\xe2\x18\x18\x5f\xad\xac\x17\x9d\x9b\x2e\x50\xc0\x45\x3b\x73\xf5\x40\xc1\xda\xb7\xe3" + + "\xf0\xda\x6a\x25\x10\x74\x45\x01\xb3\xde\x2f\xfb\xee\xe7\x72\x80\xd0\x96\x78\xe1\x7a\xd0\x1f\x0f\x23\xd7\x60\x9c" + + "\xf4\x71\xdd\xb4\xe3\xdb\xb7\x3b\xe2\x85\xf1\xde\xd5\xf9\x00\xba\x1e\x7a\xe4\xe7\xbc\x35\x72\x23\xf2\xc2\x44\xef" + + "\x2c\x63\x60\xc0\xa3\x1f\x03\xf7\x8a\x64\x00\xe6\x36\xd7\x01\x30\x47\x3f\x0c\xe6\x37\xf4\x30\xd1\xba\x4d\xf4\xb4" + + "\xc9\x56\xf8\xac\x8e\xdd\xb6\x1a\xbb\x70\x6d\x89\x61\xdc\x0f\xd7\x81\x5a\x33\xfe\x60\xb2\x30\x6c\x39\x39\x6d\xf4" + + "\xba\x89\x19\x19\xf9\x33\x7f\x44\x16\x9c\x85\xa7\x93\x45\x6c\xd8\xf2\x77\x31\x39\x1d\x1c\xdc\x53\x2c\xf4\x0b\x42" + + "\xfe\xc8\x38\x9d\xe8\x57\xc9\xda\xa0\xbe\xde\x0d\x6f\x22\xbc\x20\xd9\xa5\xff\xf5\xe0\xfb\x1c\xff\x58\x54\xf6\x69" + + "\x68\x8a\xf3\x96\xae\x5f\xaf\xee\xd2\xd9\x89\x5f\x84\x5e\xcc\xbc\x7d\x35\x1f\x2e\xac\xb5\x92\x45\xfe\x6e\x43\xb9" + + "\x80\x04\xbe\x46\xa9\xfd\xe6\xed\xc0\xe8\x63\xbf\x27\xe7\x03\xc5\x4e\xe6\x96\x6b\x03\x89\x07\x34\xf0\x2d\xda\xd2" + + "\x3c\xe8\xd6\x5f\x39\x75\x7a\x37\xe3\x9d\xe5\x95\xd6\xd7\x88\x6a\xcd\xd7\x22\x78\x26\x00\x50\xd5\xd3\x03\xd4\xf2" + + "\x10\xb1\x82\xc9\x9d\xdc\x4d\xae\xfa\x8d\x12\x1e\x12\x03\x98\x74\x07\x99\xc0\x95\xf5\x4e\xe3\x8d\x30\x41\xeb\xe6" + + "\x21\x9d\xfa\x19\x2d\x1c\x8d\x8e\x30\x6a\x66\x72\x10\x46\x5a\x2a\xf3\xf6\x29\xf0\xac\xb4\xcc\xc0\xef\xc7\x38\x06" + + "\x2b\x08\x34\xcb\xfa\xc0\x6a\x9f\xdd\x5c\x2c\xa6\x80\xc2\xa0\x72\xdb\x41\xe1\x56\x96\xe8\x16\xc6\x20\x31\xac\xb9" + + "\x96\x69\xcc\x30\x35\x01\xf9\xef\xe8\xbe\x18\xb6\x9c\xff\x67\x59\x40\xfa\xde\x7d\x90\x8f\x24\x8c\x2c\x58\x30\xc8" + + "\xac\x77\x34\x16\x6d\xcc\x36\x0b\x0e\x5d\x71\x06\x52\xce\xc5\x20\x8c\x68\x9e\xa3\x60\x01\x31\x8a\x84\x91\xfb\xcf" + + "\x13\x59\x40\x06\x16\xa7\x60\x54\x81\xe1\x6b\x81\x1f\xb9\x09\xc2\xa8\x39\x78\x10\xee\xc3\xd7\x97\xf4\x8e\x0b\x26" + + "\x77\x76\xf0\xdd\x58\x1f\x4a\x9a\xf5\xa5\x17\x42\x5f\x72\xec\x32\xfa\xa6\xa5\x08\x48\x55\x45\x0f\x54\xe3\xd7\x9c" + + "\x9a\x4d\x5d\xc7\xf6\xf8\xb1\x8d\x57\x5c\x55\x51\x49\x95\x8e\x38\x7b\xac\x6b\x32\x1d\xde\x23\x3b\xa7\xea\x29\x5c" + + "\x9c\x9d\x9d\xf5\x99\xb4\x5b\xd9\xe6\x90\x1b\x68\xaf\xb9\xd9\xd3\xc9\xbf\x65\xb3\xbd\xb7\x37\xd7\x72\xf7\x5b\x46" + + "\x73\x87\x6f\x1f\x55\x85\x82\xd5\xf5\xc9\xdf\x01\x00\x00\xff\xff\x6f\xf7\x6b\x27\x0b\x11\x00\x00") + +func bindataWebTemplatesMonitorDetailsviewgohtmlBytes() ([]byte, error) { + return bindataRead( + _bindataWebTemplatesMonitorDetailsviewgohtml, + "web/templates/monitor/details_view.go.html", + ) +} -func webTemplatesMonitorIndexGoHtmlBytes() ([]byte, error) { + + +func bindataWebTemplatesMonitorDetailsviewgohtml() (*asset, error) { + bytes, err := bindataWebTemplatesMonitorDetailsviewgohtmlBytes() + if err != nil { + return nil, err + } + + info := bindataFileInfo{ + name: "web/templates/monitor/details_view.go.html", + size: 4363, + md5checksum: "", + mode: os.FileMode(420), + modTime: time.Unix(1584119422, 0), + } + + a := &asset{bytes: bytes, info: info} + + return a, nil +} + +var _bindataWebTemplatesMonitorIndexgohtml = []byte( + "\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xd4\x95\xc1\x8e\xda\x30\x10\x86\xef\xfb\x14\xa3\x88\xf6\x54\x12\x15\xed" + + "\x69\x1b\x90\xda\xaa\x42\x6c\xd1\x06\xb5\xa1\xd7\xca\xe0\x81\x58\xf5\xda\x96\xe3\x40\x2b\xcb\xef\x5e\x39\x90\x6c" + + "\x48\xa0\x39\xd0\x5d\xb5\x17\xb0\x3d\xbf\xc7\xdf\x4c\xc6\x1e\x6b\x29\x6e\x98\x40\x08\xd6\x52\x18\x14\x26\x70\xee" + + "\x26\xa6\x6c\x07\x6b\x4e\xf2\x7c\x1c\x68\xb9\x0f\x26\x37\x00\xcd\x35\x2f\x25\x4c\xa0\x2e\x2d\x6d\x1b\x1f\x3e\xd2" + + "\xe1\xdb\xd1\xd1\x06\x10\x67\xa3\xc9\x42\xcb\x35\xe6\xb9\xd4\x79\x1c\x65\xa3\xca\x62\xed\x60\x45\x72\xfc\xae\x88" + + "\xc9\xe0\x6e\x0c\x61\x3d\x73\xae\x96\x68\x22\xb6\x08\x03\x26\x28\xfe\x7c\x03\x03\x55\x79\x2a\x37\xd4\xb3\xbc\xde" + + "\xd1\x87\xd3\x56\x28\x22\x90\x43\xf9\x3b\xa4\xb8\x21\x05\x37\x27\xda\x33\xea\x61\x86\x84\x32\xb1\x6d\xe9\x7c\xa8" + + "\xb7\xa7\x42\xc3\x0c\xc7\x60\x12\x13\xc8\x34\x6e\xc6\x41\x33\x62\xe7\xa2\x1a\x3f\xb2\xf6\x10\xa1\x73\xc1\xc4\xda" + + "\xa7\x28\xc3\xa9\x26\x2a\x0b\xa7\x5a\x16\xca\xb9\x38\x22\x9d\x23\xa3\xec\xb6\x85\x1b\x51\xb6\xeb\x8b\x60\x25\xe9" + + "\xaf\x0e\xbe\xb5\xb0\x67\x26\x83\xc1\xd6\x1f\xea\x13\xdc\x06\x81\x46\x9a\x8f\xae\x0b\xde\x76\xd3\xf8\x6a\x48\xb7" + + "\x38\xa3\xfe\xbb\xf9\x51\xe9\xb1\xf4\x1d\xce\x84\x2a\xcc\x57\xa3\x91\x3c\xe6\x1d\x9f\x00\x31\x67\x93\xd9\xc3\x62" + + "\x99\xde\xf9\x22\xf1\x7b\xc3\x54\x2a\xb6\x76\x0e\x5e\x6b\xa2\xf5\x3b\xb0\x56\x69\x26\xcc\x06\x82\x57\x69\x70\x70" + + "\x1f\x7e\x94\x14\xd7\x3e\x4b\x9c\x9d\x63\x42\x41\xcf\x1c\xd5\xcf\x7a\x2f\x99\x30\x29\x59\x71\xbc\x84\x7a\x9f\xcc" + + "\x1e\xfe\x09\xd4\xb9\x94\x3f\x0a\xf5\x47\xd6\x79\x92\x7c\x5e\x2e\x5e\x02\xb6\xaa\x25\x79\x24\x6a\x70\x4e\xeb\xc5" + + "\x0b\x94\xd3\x2f\xc9\x72\x91\xbe\xff\x30\xff\x54\x92\x3e\x39\xe9\xe1\x6d\x08\xaf\xa2\xe6\x52\xaa\xd3\xbc\xaa\x43" + + "\xb5\x5e\xce\xea\x21\xa7\x7e\x63\x0f\x63\x29\x79\xa6\x02\x48\x0a\xd3\x7b\xb1\x92\x65\xfa\x02\x37\x2b\x8e\xba\x2f" + + "\x83\xb5\x80\x82\xb6\x1f\x91\xce\x83\xd5\x5a\x38\x99\x36\x0f\xab\x0d\xc7\xc1\xd5\x6d\xea\x1b\xc3\xfd\xd5\x1d\x6a" + + "\xc7\x70\x5f\x6a\xfd\xe0\xff\xec\x4b\x9e\xbc\xdd\x92\xfc\x5a\x55\x2a\xcf\xda\x87\xfe\x5a\x39\x1c\xff\x2a\xc9\xef" + + "\x00\x00\x00\xff\xff\x33\x4f\x5a\x19\xeb\x08\x00\x00") + +func bindataWebTemplatesMonitorIndexgohtmlBytes() ([]byte, error) { return bindataRead( - _webTemplatesMonitorIndexGoHtml, + _bindataWebTemplatesMonitorIndexgohtml, "web/templates/monitor/index.go.html", ) } -func webTemplatesMonitorIndexGoHtml() (*asset, error) { - bytes, err := webTemplatesMonitorIndexGoHtmlBytes() + + +func bindataWebTemplatesMonitorIndexgohtml() (*asset, error) { + bytes, err := bindataWebTemplatesMonitorIndexgohtmlBytes() if err != nil { return nil, err } - info := bindataFileInfo{name: "web/templates/monitor/index.go.html", size: 2283, mode: os.FileMode(436), modTime: time.Unix(1548705120, 0)} + info := bindataFileInfo{ + name: "web/templates/monitor/index.go.html", + size: 2283, + md5checksum: "", + mode: os.FileMode(420), + modTime: time.Unix(1559056733, 0), + } + a := &asset{bytes: bytes, info: info} + return a, nil } -var _webTemplatesMonitorMenuGoHtml = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\x8c\x8e\x41\xca\x83\x30\x10\x46\xf7\x9e\x62\x08\x2e\xfe\x1f\x4a\xb2\x2f\x31\x5b\x2f\xd0\xbd\xa4\x3a\xd6\x80\x4d\x42\xa2\x6d\x61\x98\xbb\x97\x54\x70\x25\xa5\xab\x99\xc5\xe3\x7b\x8f\x68\xc0\xd1\x79\x04\x71\x47\xbf\x76\x7d\xf0\x0b\xfa\x45\x30\x57\x44\xc9\xfa\x1b\x42\xed\xfc\x80\xaf\x13\xd4\x31\x85\x1e\xce\x0d\xc8\xf2\x60\xce\x21\x65\xe6\x0a\x40\xcf\xce\x68\x0b\x53\xc2\xb1\x11\x44\xb5\xbc\xda\x8c\x5d\xb4\xcb\xc4\xac\x76\x56\x11\x6d\x4b\xcc\xc2\x10\x7d\xd6\x64\x9b\x6c\x9c\x64\x9b\xc2\x1a\x99\xe1\x6f\x87\xff\xb5\xb2\x46\xab\xd9\x99\x8a\x08\xfd\xf0\x35\xe7\xe1\xf0\xf9\x43\x49\xc1\x0e\x23\x2e\x21\xba\xbe\xe8\x0b\x71\x68\xde\xee\x3b\x00\x00\xff\xff\xfc\x37\x76\xe3\x2b\x01\x00\x00") +var _bindataWebTemplatesMonitorMenugohtml = []byte( + "\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\x8c\x8e\x41\xca\x83\x30\x10\x46\xf7\x9e\x62\x08\x2e\xfe\x1f\x4a\xb2\x2f" + + "\x31\x5b\x2f\xd0\xbd\xa4\x3a\xd6\x80\x4d\x42\xa2\x6d\x61\x98\xbb\x97\x54\x70\x25\xa5\xab\x99\xc5\xe3\x7b\x8f\x68" + + "\xc0\xd1\x79\x04\x71\x47\xbf\x76\x7d\xf0\x0b\xfa\x45\x30\x57\x44\xc9\xfa\x1b\x42\xed\xfc\x80\xaf\x13\xd4\x31\x85" + + "\x1e\xce\x0d\xc8\xf2\x60\xce\x21\x65\xe6\x0a\x40\xcf\xce\x68\x0b\x53\xc2\xb1\x11\x44\xb5\xbc\xda\x8c\x5d\xb4\xcb" + + "\xc4\xac\x76\x56\x11\x6d\x4b\xcc\xc2\x10\x7d\xd6\x64\x9b\x6c\x9c\x64\x9b\xc2\x1a\x99\xe1\x6f\x87\xff\xb5\xb2\x46" + + "\xab\xd9\x99\x8a\x08\xfd\xf0\x35\xe7\xe1\xf0\xf9\x43\x49\xc1\x0e\x23\x2e\x21\xba\xbe\xe8\x0b\x71\x68\xde\xee\x3b" + + "\x00\x00\xff\xff\xfc\x37\x76\xe3\x2b\x01\x00\x00") -func webTemplatesMonitorMenuGoHtmlBytes() ([]byte, error) { +func bindataWebTemplatesMonitorMenugohtmlBytes() ([]byte, error) { return bindataRead( - _webTemplatesMonitorMenuGoHtml, + _bindataWebTemplatesMonitorMenugohtml, "web/templates/monitor/menu.go.html", ) } -func webTemplatesMonitorMenuGoHtml() (*asset, error) { - bytes, err := webTemplatesMonitorMenuGoHtmlBytes() + + +func bindataWebTemplatesMonitorMenugohtml() (*asset, error) { + bytes, err := bindataWebTemplatesMonitorMenugohtmlBytes() if err != nil { return nil, err } - info := bindataFileInfo{name: "web/templates/monitor/menu.go.html", size: 299, mode: os.FileMode(436), modTime: time.Unix(1548705120, 0)} + info := bindataFileInfo{ + name: "web/templates/monitor/menu.go.html", + size: 299, + md5checksum: "", + mode: os.FileMode(420), + modTime: time.Unix(1559056733, 0), + } + a := &asset{bytes: bytes, info: info} + return a, nil } -var _webTemplatesQueryIndexGoHtml = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xcc\x56\x4d\x6f\xe3\x36\x10\xbd\xe7\x57\x4c\x08\xa3\x39\xb4\xb2\xd0\xe4\xd6\x50\xea\xa5\x39\x15\x28\x50\xf4\xd0\x63\x40\x89\x23\x89\x28\x4d\xaa\xe4\x30\x8e\x21\xe8\xbf\x2f\xf4\x19\x5b\x96\xed\x2c\x16\x0b\xec\x45\xc9\x68\x3e\x38\xef\xcd\xe3\x58\x4d\x23\xb1\x50\x06\x81\xe5\xd6\x10\x1a\x62\x6d\x7b\xc7\xa5\x7a\x83\x5c\x0b\xef\x13\xe6\xec\x9e\xa5\x77\x00\x00\xc7\x6f\xbb\x60\xa1\x0c\xba\xd1\xb7\xf4\x7f\x64\xad\x79\x73\xab\xa3\x9d\x8c\x7e\x7d\x5c\xc4\x00\xf0\xea\x31\xfd\x3b\xa0\x3b\xc0\x9f\x78\xf0\x3c\xae\x1e\x97\x11\x4d\xa3\x0a\xd8\xa2\x73\xd6\xb5\xed\x32\xfb\xe8\x0c\xa1\xd1\x11\xf4\xcf\x48\x0a\x53\xa2\x9b\x0c\xe5\x77\xca\x7b\x95\x69\x64\xe0\xac\xc6\x31\xf6\xac\x17\x00\x9e\x05\x22\x6b\x80\x0e\x35\x26\x6c\x30\xd8\x0c\x42\x5b\x8f\x0c\xa4\x20\x31\xd5\x9c\x2b\x71\x5f\x0b\x93\xfe\x44\x6a\x87\xfe\x99\xc7\xbd\xc5\xe3\xa1\xc0\xca\x31\x9e\x9c\x35\x65\xfa\xd2\x81\xba\xe7\xf1\x68\x42\xd3\x0c\x38\xb7\x2f\xeb\x68\x63\xa9\xde\xce\xe9\x41\x23\xcf\x42\x07\xd2\xf6\xc2\x19\x65\xca\x4f\xd3\x36\xc6\xff\xf0\xbc\xfd\x3b\xf4\x79\xca\xdc\xd8\xfc\xb7\x73\x37\xb1\xe7\x6d\x70\x39\xfa\x15\x3f\x2f\xac\xdb\x81\x35\x3e\x64\x3b\x45\x09\xdb\x2b\x23\xed\x7e\xab\x6d\x2e\x48\x59\x03\x09\x3c\x34\xcd\x36\x13\x1e\x5f\x6b\x41\x55\xdb\xc6\x4d\xb3\xf5\xa8\x31\x27\x94\xaf\x43\xdd\xb6\x8d\x1f\xe0\x67\xf0\x28\x5c\x5e\x6d\xdf\x84\x0e\xf8\x0c\x0e\x29\x38\x03\x85\xd0\x1e\x9f\x57\x78\x5e\x0e\x4f\x99\x3a\x50\x54\x3a\x1b\x6a\x38\xfa\x3f\xd2\xe5\x85\xe4\x2b\x05\xa2\x8c\xcc\x95\xac\x9b\x63\xce\xc8\x40\x46\x26\x92\x58\x88\xa0\x09\xa4\xb3\xb5\xb4\x7b\x13\x91\x2d\x4b\x3d\x09\x60\x30\x12\x36\x79\x59\xba\x46\x4d\x2f\x8b\x59\x3f\xc2\x61\x27\x95\x5b\xea\x38\xe9\x34\xe8\x29\x7d\xee\x63\x87\x26\xdc\x00\x08\xfd\xf0\x37\xf3\xe4\xe0\xb7\x04\x8e\xe7\xf8\x89\x64\xd7\xad\x1e\xd8\x28\x23\xf1\xfd\x17\xd8\x0c\x88\xfa\x3a\x97\xf5\x74\xd6\xbd\x56\x29\x17\x50\x39\x2c\x12\x76\xdc\x4f\xaf\xa4\xcd\xc4\x52\xc7\xdd\x6c\xf0\x58\xa4\x3c\xd6\xea\x33\x00\xd7\x55\x7f\xd2\x41\x1c\xf4\x15\x05\xad\x5d\xa6\x23\x77\xaf\x2a\x50\x32\x61\x83\xbc\xd9\xa8\x18\xc2\x77\x9a\xf5\xd2\xdd\xa1\xa8\xfb\x59\x71\x56\x33\x70\xf8\x7f\x50\x0e\xe5\xf7\x97\xed\x70\x69\x2f\xc9\x96\xa5\xff\xf4\x2d\xdf\x96\xd9\x15\x12\x2e\xb8\x78\xdc\x61\x3e\x7f\xdf\x34\xa8\x3d\xae\xed\x99\xeb\x7b\x7a\xb1\x95\xff\xb2\x30\x4a\x0c\x0a\x1b\x8c\xbc\x87\x3f\x94\x84\x83\x0d\x50\x58\x57\x22\x01\x59\x10\x44\x22\xaf\x80\x2a\xdc\xfd\x7e\xa1\xcb\x35\x79\x2c\x42\x17\xe6\xb0\x2e\xfb\x15\x76\x94\xc7\x2b\x17\xaf\x7f\x2b\xd4\xc2\xa0\x86\xfe\x39\x6f\x8b\xc1\xf2\x21\xcf\xd1\xfb\x2b\x5f\x12\x43\x5c\x85\x42\x76\x04\xac\x70\x5c\x3d\x9d\x86\x92\x22\x8d\xfd\x96\xf9\x0f\x0f\xdd\x35\xa9\x9e\xd2\x6b\xd8\xd6\x0f\xcc\xac\x3c\xac\x9d\x56\x3b\xec\x4a\x8f\xd8\x79\xdc\xd9\x5f\xc5\xdc\x07\xd7\xa3\x6b\xfc\x33\x79\xbe\x04\x00\x00\xff\xff\x05\xda\xbe\x9a\xaf\x09\x00\x00") - -func webTemplatesQueryIndexGoHtmlBytes() ([]byte, error) { +var _bindataWebTemplatesQueryIndexgohtml = []byte( + "\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\xcc\x56\x4d\x6f\xe3\x36\x10\xbd\xe7\x57\x4c\x08\xa3\x39\xb4\xb2\xd0\xe4" + + "\xd6\x50\xea\xa5\x39\x15\x28\x50\xf4\xd0\x63\x40\x89\x23\x89\x28\x4d\xaa\xe4\x30\x8e\x21\xe8\xbf\x2f\xf4\x19\x5b" + + "\x96\xed\x2c\x16\x0b\xec\x45\xc9\x68\x3e\x38\xef\xcd\xe3\x58\x4d\x23\xb1\x50\x06\x81\xe5\xd6\x10\x1a\x62\x6d\x7b" + + "\xc7\xa5\x7a\x83\x5c\x0b\xef\x13\xe6\xec\x9e\xa5\x77\x00\x00\xc7\x6f\xbb\x60\xa1\x0c\xba\xd1\xb7\xf4\x7f\x64\xad" + + "\x79\x73\xab\xa3\x9d\x8c\x7e\x7d\x5c\xc4\x00\xf0\xea\x31\xfd\x3b\xa0\x3b\xc0\x9f\x78\xf0\x3c\xae\x1e\x97\x11\x4d" + + "\xa3\x0a\xd8\xa2\x73\xd6\xb5\xed\x32\xfb\xe8\x0c\xa1\xd1\x11\xf4\xcf\x48\x0a\x53\xa2\x9b\x0c\xe5\x77\xca\x7b\x95" + + "\x69\x64\xe0\xac\xc6\x31\xf6\xac\x17\x00\x9e\x05\x22\x6b\x80\x0e\x35\x26\x6c\x30\xd8\x0c\x42\x5b\x8f\x0c\xa4\x20" + + "\x31\xd5\x9c\x2b\x71\x5f\x0b\x93\xfe\x44\x6a\x87\xfe\x99\xc7\xbd\xc5\xe3\xa1\xc0\xca\x31\x9e\x9c\x35\x65\xfa\xd2" + + "\x81\xba\xe7\xf1\x68\x42\xd3\x0c\x38\xb7\x2f\xeb\x68\x63\xa9\xde\xce\xe9\x41\x23\xcf\x42\x07\xd2\xf6\xc2\x19\x65" + + "\xca\x4f\xd3\x36\xc6\xff\xf0\xbc\xfd\x3b\xf4\x79\xca\xdc\xd8\xfc\xb7\x73\x37\xb1\xe7\x6d\x70\x39\xfa\x15\x3f\x2f" + + "\xac\xdb\x81\x35\x3e\x64\x3b\x45\x09\xdb\x2b\x23\xed\x7e\xab\x6d\x2e\x48\x59\x03\x09\x3c\x34\xcd\x36\x13\x1e\x5f" + + "\x6b\x41\x55\xdb\xc6\x4d\xb3\xf5\xa8\x31\x27\x94\xaf\x43\xdd\xb6\x8d\x1f\xe0\x67\xf0\x28\x5c\x5e\x6d\xdf\x84\x0e" + + "\xf8\x0c\x0e\x29\x38\x03\x85\xd0\x1e\x9f\x57\x78\x5e\x0e\x4f\x99\x3a\x50\x54\x3a\x1b\x6a\x38\xfa\x3f\xd2\xe5\x85" + + "\xe4\x2b\x05\xa2\x8c\xcc\x95\xac\x9b\x63\xce\xc8\x40\x46\x26\x92\x58\x88\xa0\x09\xa4\xb3\xb5\xb4\x7b\x13\x91\x2d" + + "\x4b\x3d\x09\x60\x30\x12\x36\x79\x59\xba\x46\x4d\x2f\x8b\x59\x3f\xc2\x61\x27\x95\x5b\xea\x38\xe9\x34\xe8\x29\x7d" + + "\xee\x63\x87\x26\xdc\x00\x08\xfd\xf0\x37\xf3\xe4\xe0\xb7\x04\x8e\xe7\xf8\x89\x64\xd7\xad\x1e\xd8\x28\x23\xf1\xfd" + + "\x17\xd8\x0c\x88\xfa\x3a\x97\xf5\x74\xd6\xbd\x56\x29\x17\x50\x39\x2c\x12\x76\xdc\x4f\xaf\xa4\xcd\xc4\x52\xc7\xdd" + + "\x6c\xf0\x58\xa4\x3c\xd6\xea\x33\x00\xd7\x55\x7f\xd2\x41\x1c\xf4\x15\x05\xad\x5d\xa6\x23\x77\xaf\x2a\x50\x32\x61" + + "\x83\xbc\xd9\xa8\x18\xc2\x77\x9a\xf5\xd2\xdd\xa1\xa8\xfb\x59\x71\x56\x33\x70\xf8\x7f\x50\x0e\xe5\xf7\x97\xed\x70" + + "\x69\x2f\xc9\x96\xa5\xff\xf4\x2d\xdf\x96\xd9\x15\x12\x2e\xb8\x78\xdc\x61\x3e\x7f\xdf\x34\xa8\x3d\xae\xed\x99\xeb" + + "\x7b\x7a\xb1\x95\xff\xb2\x30\x4a\x0c\x0a\x1b\x8c\xbc\x87\x3f\x94\x84\x83\x0d\x50\x58\x57\x22\x01\x59\x10\x44\x22" + + "\xaf\x80\x2a\xdc\xfd\x7e\xa1\xcb\x35\x79\x2c\x42\x17\xe6\xb0\x2e\xfb\x15\x76\x94\xc7\x2b\x17\xaf\x7f\x2b\xd4\xc2" + + "\xa0\x86\xfe\x39\x6f\x8b\xc1\xf2\x21\xcf\xd1\xfb\x2b\x5f\x12\x43\x5c\x85\x42\x76\x04\xac\x70\x5c\x3d\x9d\x86\x92" + + "\x22\x8d\xfd\x96\xf9\x0f\x0f\xdd\x35\xa9\x9e\xd2\x6b\xd8\xd6\x0f\xcc\xac\x3c\xac\x9d\x56\x3b\xec\x4a\x8f\xd8\x79" + + "\xdc\xd9\x5f\xc5\xdc\x07\xd7\xa3\x6b\xfc\x33\x79\xbe\x04\x00\x00\xff\xff\x05\xda\xbe\x9a\xaf\x09\x00\x00") + +func bindataWebTemplatesQueryIndexgohtmlBytes() ([]byte, error) { return bindataRead( - _webTemplatesQueryIndexGoHtml, + _bindataWebTemplatesQueryIndexgohtml, "web/templates/query/index.go.html", ) } -func webTemplatesQueryIndexGoHtml() (*asset, error) { - bytes, err := webTemplatesQueryIndexGoHtmlBytes() + + +func bindataWebTemplatesQueryIndexgohtml() (*asset, error) { + bytes, err := bindataWebTemplatesQueryIndexgohtmlBytes() if err != nil { return nil, err } - info := bindataFileInfo{name: "web/templates/query/index.go.html", size: 2479, mode: os.FileMode(436), modTime: time.Unix(1548705120, 0)} + info := bindataFileInfo{ + name: "web/templates/query/index.go.html", + size: 2479, + md5checksum: "", + mode: os.FileMode(420), + modTime: time.Unix(1559056733, 0), + } + a := &asset{bytes: bytes, info: info} + return a, nil } -var _webTemplatesIndexIndexGoHtml = []byte("\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\x84\x90\xcf\x4a\xc4\x30\x10\x87\xef\x7d\x8a\x21\xec\xd1\x6e\x51\xf6\x24\x6d\x0f\x3e\x80\xf8\x0a\x43\x67\xba\x09\xa4\xd3\xa5\x8d\xab\x30\xcc\xbb\x8b\x4b\xac\xa6\x0a\x5e\x92\xcc\x7c\x5f\xfe\xfc\xa2\x4a\x3c\x06\x61\x70\xc3\x2c\x89\x25\x39\xb3\xaa\xa5\x70\x85\x21\xe2\xba\x76\x6e\x99\xdf\x5c\x5f\x01\xfc\xec\x7d\xaa\x18\x84\x97\x1b\xd9\xb3\x58\x4f\x54\xdf\x3f\x64\x06\xa0\xba\xa0\x9c\x19\x0e\x41\x88\xdf\xef\xe0\x30\xcc\xd3\x65\x16\x96\x04\x8f\x1d\x1c\xb7\x6a\x35\xcb\x3b\xfe\x3b\x71\x6f\x5c\x50\x38\xc2\x6d\xac\x89\x47\x7c\x8d\xa9\x70\xff\xb0\x6b\xcf\x48\x41\xce\x3b\x0f\xa0\xf5\xa7\x52\x4c\x21\x45\x76\x7d\x8b\xe0\x17\x1e\x3b\xa7\xfa\x1d\xe0\xf8\x84\x2b\xbf\x60\xf2\x66\x8d\xeb\x0b\xf2\x8c\x13\x9b\xb5\x0d\xfe\xba\xa0\xf1\xa7\xbe\x2a\x5f\xd7\x50\xb8\x16\xe1\xca\x46\x51\xaa\xb2\x50\xfe\xaa\x0d\xe4\x45\x9e\xbe\x94\x8f\x00\x00\x00\xff\xff\xcd\x13\xfe\x16\xdd\x01\x00\x00") +var _bindataWebTemplatesIndexIndexgohtml = []byte( + "\x1f\x8b\x08\x00\x00\x00\x00\x00\x00\xff\x84\x90\xcf\x4a\xc4\x30\x10\x87\xef\x7d\x8a\x21\xec\xd1\x6e\x51\xf6\x24" + + "\x6d\x0f\x3e\x80\xf8\x0a\x43\x67\xba\x09\xa4\xd3\xa5\x8d\xab\x30\xcc\xbb\x8b\x4b\xac\xa6\x0a\x5e\x92\xcc\x7c\x5f" + + "\xfe\xfc\xa2\x4a\x3c\x06\x61\x70\xc3\x2c\x89\x25\x39\xb3\xaa\xa5\x70\x85\x21\xe2\xba\x76\x6e\x99\xdf\x5c\x5f\x01" + + "\xfc\xec\x7d\xaa\x18\x84\x97\x1b\xd9\xb3\x58\x4f\x54\xdf\x3f\x64\x06\xa0\xba\xa0\x9c\x19\x0e\x41\x88\xdf\xef\xe0" + + "\x30\xcc\xd3\x65\x16\x96\x04\x8f\x1d\x1c\xb7\x6a\x35\xcb\x3b\xfe\x3b\x71\x6f\x5c\x50\x38\xc2\x6d\xac\x89\x47\x7c" + + "\x8d\xa9\x70\xff\xb0\x6b\xcf\x48\x41\xce\x3b\x0f\xa0\xf5\xa7\x52\x4c\x21\x45\x76\x7d\x8b\xe0\x17\x1e\x3b\xa7\xfa" + + "\x1d\xe0\xf8\x84\x2b\xbf\x60\xf2\x66\x8d\xeb\x0b\xf2\x8c\x13\x9b\xb5\x0d\xfe\xba\xa0\xf1\xa7\xbe\x2a\x5f\xd7\x50" + + "\xb8\x16\xe1\xca\x46\x51\xaa\xb2\x50\xfe\xaa\x0d\xe4\x45\x9e\xbe\x94\x8f\x00\x00\x00\xff\xff\xcd\x13\xfe\x16\xdd" + + "\x01\x00\x00") -func webTemplatesIndexIndexGoHtmlBytes() ([]byte, error) { +func bindataWebTemplatesIndexIndexgohtmlBytes() ([]byte, error) { return bindataRead( - _webTemplatesIndexIndexGoHtml, + _bindataWebTemplatesIndexIndexgohtml, "web/templates/index/index.go.html", ) } -func webTemplatesIndexIndexGoHtml() (*asset, error) { - bytes, err := webTemplatesIndexIndexGoHtmlBytes() + + +func bindataWebTemplatesIndexIndexgohtml() (*asset, error) { + bytes, err := bindataWebTemplatesIndexIndexgohtmlBytes() if err != nil { return nil, err } - info := bindataFileInfo{name: "web/templates/index/index.go.html", size: 477, mode: os.FileMode(436), modTime: time.Unix(1548705120, 0)} + info := bindataFileInfo{ + name: "web/templates/index/index.go.html", + size: 477, + md5checksum: "", + mode: os.FileMode(420), + modTime: time.Unix(1559056733, 0), + } + a := &asset{bytes: bytes, info: info} + return a, nil } + +// // Asset loads and returns the asset for the given name. // It returns an error if the asset could not be found or // could not be loaded. +// func Asset(name string) ([]byte, error) { cannonicalName := strings.Replace(name, "\\", "/", -1) if f, ok := _bindata[cannonicalName]; ok { @@ -247,11 +641,14 @@ func Asset(name string) ([]byte, error) { } return a.bytes, nil } - return nil, fmt.Errorf("Asset %s not found", name) + return nil, &os.PathError{Op: "open", Path: name, Err: os.ErrNotExist} } +// // MustAsset is like Asset but panics when Asset would return an error. // It simplifies safe initialization of global variables. +// nolint: deadcode +// func MustAsset(name string) []byte { a, err := Asset(name) if err != nil { @@ -261,9 +658,10 @@ func MustAsset(name string) []byte { return a } +// // AssetInfo loads and returns the asset info for the given name. -// It returns an error if the asset could not be found or -// could not be loaded. +// It returns an error if the asset could not be found or could not be loaded. +// func AssetInfo(name string) (os.FileInfo, error) { cannonicalName := strings.Replace(name, "\\", "/", -1) if f, ok := _bindata[cannonicalName]; ok { @@ -273,10 +671,13 @@ func AssetInfo(name string) (os.FileInfo, error) { } return a.info, nil } - return nil, fmt.Errorf("AssetInfo %s not found", name) + return nil, &os.PathError{Op: "open", Path: name, Err: os.ErrNotExist} } +// // AssetNames returns the names of the assets. +// nolint: deadcode +// func AssetNames() []string { names := make([]string, 0, len(_bindata)) for name := range _bindata { @@ -285,18 +686,22 @@ func AssetNames() []string { return names } +// // _bindata is a table, holding each asset generator, mapped to its name. +// var _bindata = map[string]func() (*asset, error){ - "web/templates/common/base.go.html": webTemplatesCommonBaseGoHtml, - "web/templates/common/head.go.html": webTemplatesCommonHeadGoHtml, - "web/templates/common/menu.go.html": webTemplatesCommonMenuGoHtml, - "web/templates/monitor/details.go.html": webTemplatesMonitorDetailsGoHtml, - "web/templates/monitor/index.go.html": webTemplatesMonitorIndexGoHtml, - "web/templates/monitor/menu.go.html": webTemplatesMonitorMenuGoHtml, - "web/templates/query/index.go.html": webTemplatesQueryIndexGoHtml, - "web/templates/index/index.go.html": webTemplatesIndexIndexGoHtml, -} - + "web/templates/common/base.go.html": bindataWebTemplatesCommonBasegohtml, + "web/templates/common/head.go.html": bindataWebTemplatesCommonHeadgohtml, + "web/templates/common/menu.go.html": bindataWebTemplatesCommonMenugohtml, + "web/templates/monitor/details_processor.go.html": bindataWebTemplatesMonitorDetailsprocessorgohtml, + "web/templates/monitor/details_view.go.html": bindataWebTemplatesMonitorDetailsviewgohtml, + "web/templates/monitor/index.go.html": bindataWebTemplatesMonitorIndexgohtml, + "web/templates/monitor/menu.go.html": bindataWebTemplatesMonitorMenugohtml, + "web/templates/query/index.go.html": bindataWebTemplatesQueryIndexgohtml, + "web/templates/index/index.go.html": bindataWebTemplatesIndexIndexgohtml, +} + +// // AssetDir returns the file names below a certain // directory embedded in the file by go-bindata. // For example if you run go-bindata on data/... and data contains the @@ -310,6 +715,7 @@ var _bindata = map[string]func() (*asset, error){ // AssetDir("data/img") would return []string{"a.png", "b.png"} // AssetDir("foo.txt") and AssetDir("notexist") would return an error // AssetDir("") will return []string{"data"}. +// func AssetDir(name string) ([]string, error) { node := _bintree if len(name) != 0 { @@ -318,12 +724,20 @@ func AssetDir(name string) ([]string, error) { for _, p := range pathList { node = node.Children[p] if node == nil { - return nil, fmt.Errorf("Asset %s not found", name) + return nil, &os.PathError{ + Op: "open", + Path: name, + Err: os.ErrNotExist, + } } } } if node.Func != nil { - return nil, fmt.Errorf("Asset %s not found", name) + return nil, &os.PathError{ + Op: "open", + Path: name, + Err: os.ErrNotExist, + } } rv := make([]string, 0, len(node.Children)) for childName := range node.Children { @@ -332,28 +746,31 @@ func AssetDir(name string) ([]string, error) { return rv, nil } + type bintree struct { Func func() (*asset, error) Children map[string]*bintree } -var _bintree = &bintree{nil, map[string]*bintree{ - "web": &bintree{nil, map[string]*bintree{ - "templates": &bintree{nil, map[string]*bintree{ - "common": &bintree{nil, map[string]*bintree{ - "base.go.html": &bintree{webTemplatesCommonBaseGoHtml, map[string]*bintree{}}, - "head.go.html": &bintree{webTemplatesCommonHeadGoHtml, map[string]*bintree{}}, - "menu.go.html": &bintree{webTemplatesCommonMenuGoHtml, map[string]*bintree{}}, + +var _bintree = &bintree{Func: nil, Children: map[string]*bintree{ + "web": {Func: nil, Children: map[string]*bintree{ + "templates": {Func: nil, Children: map[string]*bintree{ + "common": {Func: nil, Children: map[string]*bintree{ + "base.go.html": {Func: bindataWebTemplatesCommonBasegohtml, Children: map[string]*bintree{}}, + "head.go.html": {Func: bindataWebTemplatesCommonHeadgohtml, Children: map[string]*bintree{}}, + "menu.go.html": {Func: bindataWebTemplatesCommonMenugohtml, Children: map[string]*bintree{}}, }}, - "index": &bintree{nil, map[string]*bintree{ - "index.go.html": &bintree{webTemplatesIndexIndexGoHtml, map[string]*bintree{}}, + "index": {Func: nil, Children: map[string]*bintree{ + "index.go.html": {Func: bindataWebTemplatesIndexIndexgohtml, Children: map[string]*bintree{}}, }}, - "monitor": &bintree{nil, map[string]*bintree{ - "details.go.html": &bintree{webTemplatesMonitorDetailsGoHtml, map[string]*bintree{}}, - "index.go.html": &bintree{webTemplatesMonitorIndexGoHtml, map[string]*bintree{}}, - "menu.go.html": &bintree{webTemplatesMonitorMenuGoHtml, map[string]*bintree{}}, + "monitor": {Func: nil, Children: map[string]*bintree{ + "details_processor.go.html": {Func: bindataWebTemplatesMonitorDetailsprocessorgohtml, Children: map[string]*bintree{}}, + "details_view.go.html": {Func: bindataWebTemplatesMonitorDetailsviewgohtml, Children: map[string]*bintree{}}, + "index.go.html": {Func: bindataWebTemplatesMonitorIndexgohtml, Children: map[string]*bintree{}}, + "menu.go.html": {Func: bindataWebTemplatesMonitorMenugohtml, Children: map[string]*bintree{}}, }}, - "query": &bintree{nil, map[string]*bintree{ - "index.go.html": &bintree{webTemplatesQueryIndexGoHtml, map[string]*bintree{}}, + "query": {Func: nil, Children: map[string]*bintree{ + "index.go.html": {Func: bindataWebTemplatesQueryIndexgohtml, Children: map[string]*bintree{}}, }}, }}, }}, @@ -377,11 +794,7 @@ func RestoreAsset(dir, name string) error { if err != nil { return err } - err = os.Chtimes(_filePath(dir, name), info.ModTime(), info.ModTime()) - if err != nil { - return err - } - return nil + return os.Chtimes(_filePath(dir, name), info.ModTime(), info.ModTime()) } // RestoreAssets restores an asset under the given directory recursively @@ -405,4 +818,3 @@ func _filePath(dir, name string) string { cannonicalName := strings.Replace(name, "\\", "/", -1) return filepath.Join(append([]string{dir}, strings.Split(cannonicalName, "/")...)...) } - diff --git a/web/templates/monitor/details.go.html b/web/templates/monitor/details.go.html deleted file mode 100644 index 4e421244..00000000 --- a/web/templates/monitor/details.go.html +++ /dev/null @@ -1,216 +0,0 @@ -{{define "content"}} -
-
-
-

{{.title}}

-
-
-

Table statistics

-
-
- - - - - - - - - - - - - - - - - - - - - -
PartitionStateOffset-LagHWMWrite-RateInputOutput
ratebytesdelayratebytes
-
-
- -{{if eq .renderType "processor"}} -
-
-

Co-Joined Tables

-
-
- - - - - - - - - - - - - -
PartitionTableStateOffset-LagHWMWrite-Rate
-
-
-{{end}} - - -
-
-{{end}} diff --git a/web/templates/monitor/details_processor.go.html b/web/templates/monitor/details_processor.go.html new file mode 100644 index 00000000..330d406a --- /dev/null +++ b/web/templates/monitor/details_processor.go.html @@ -0,0 +1,498 @@ +{{define "content"}} +
+
+
+

{{.title}}

+ +
+
+

Partition Table (State)

+
+ +
+ + + + + + + + + + + + + + + + + + +
PartitionStateRecoveryWrites
TimeRateLagRateBytes
+
+
+ +
+
+

Joins/Lookups

+
+
+ + + + + + + + + + + + + + + + + + + + + + +
TypeTopicPartitionStateRecoveryIncoming
TimeRateLagRateBytesOffset LagTime Lag
+
+
+ +
+
+

Input Traffic

+
    +
+ +
+
+ + + + + + + + + + + + + + + + + + + + + +
TopicOffset (min/avg/max)Event RateByte RateLag (offset/time)
TotalPer PartitionTotalPer PartitionMinAvgMax
+
+
+ +
+
+

Outgoing Traffic

+
    +
+ +
+
+ + + + + + + + + + + + + + + + +
TopicEvent RateByte Rate
TotalPer PartitionTotalPer Partition
+
+
+ + +
+
+{{end}} diff --git a/web/templates/monitor/details_view.go.html b/web/templates/monitor/details_view.go.html new file mode 100644 index 00000000..149f9a2e --- /dev/null +++ b/web/templates/monitor/details_view.go.html @@ -0,0 +1,127 @@ +{{define "content"}} +
+
+
+

View {{.title}}

+ +
+
+

Table statistics

+
+ +
+ + + + + + + + + + + + + + + + + + + + +
PartitionStateRecoveryIncoming Traffic
TimeRateLagRateBytesOffset LagTime Lag
+
+
+ + +
+
+{{end}}