├── .github ├── actions │ └── run-application │ │ └── action.yml └── workflows │ ├── install.yml │ ├── main.yml │ └── runexamples.yml ├── .gitignore ├── CHANGELOG.md ├── LICENSE ├── MIGRATION.md ├── Makefile ├── README.md ├── assignment.go ├── broker.go ├── builders.go ├── codec.go ├── codec └── codec.go ├── config.go ├── config_test.go ├── context.go ├── context_test.go ├── copartition_strategy.go ├── copartition_strategy_test.go ├── doc.go ├── emitter.go ├── emitter_test.go ├── errors.go ├── examples ├── .env ├── 1-simplest │ ├── README.md │ └── main.go ├── 10-visit │ └── main.go ├── 2-clicks │ ├── README.md │ └── main.go ├── 3-messaging │ ├── README.md │ ├── blocker │ │ └── blocker.go │ ├── cmd │ │ ├── block-user │ │ │ └── main.go │ │ ├── loadgen │ │ │ └── main.go │ │ ├── processor │ │ │ └── main.go │ │ ├── service │ │ │ └── main.go │ │ └── translate-word │ │ │ └── main.go │ ├── collector │ │ └── collector.go │ ├── detector │ │ └── detector.go │ ├── figs │ │ ├── goka-arch-blocker.png │ │ └── goka-arch-simple.png │ ├── filter │ │ └── filter.go │ ├── message.go │ ├── service │ │ └── service.go │ ├── topicinit │ │ └── topicinit.go │ └── translator │ │ └── translator.go ├── 4-tests │ ├── README.md │ └── example_test.go ├── 5-multiple │ └── main.go ├── 6-reconnecting-view │ ├── README.md │ └── main.go ├── 7-redis │ ├── README.md │ ├── codec.go │ ├── config.yaml │ ├── consumer.go │ ├── event.go │ ├── kafka.yml │ ├── main.go │ └── producer.go ├── 8-monitoring │ ├── README.md │ ├── images │ │ ├── index.png │ │ ├── processor-view.png │ │ └── query.png │ └── main.go ├── 9-defer-commit │ ├── README.md │ └── main.go ├── Makefile ├── README.md ├── create-kafka-commands.sh ├── docker-compose.yml ├── wait-kafka.sh ├── wait.sh └── wait_for_output.py ├── go.mod ├── go.sum ├── graph.go ├── graph_test.go ├── header_test.go ├── headers.go ├── integrationtest ├── processor_test.go └── view_test.go ├── iterator.go ├── iterator_test.go ├── logger.go ├── mockautoconsumers.go ├── mockbuilder.go ├── mockcontroller.go ├── mocks.go ├── mockssarama.go ├── mockstorage.go ├── multierr ├── errgroup.go ├── errgroup_test.go ├── errors.go └── multiwait.go ├── once.go ├── once_test.go ├── options.go ├── options_test.go ├── partition_processor.go ├── partition_table.go ├── partition_table_test.go ├── processor.go ├── processor_test.go ├── producer.go ├── producer_test.go ├── promise.go ├── promise_test.go ├── proxy.go ├── proxy_test.go ├── signal.go ├── signal_test.go ├── simple_backoff.go ├── simple_backoff_test.go ├── stats.go ├── storage ├── append.go ├── builders.go ├── iterator.go ├── iterator_test.go ├── leveldb_test.go ├── memory.go ├── memory_test.go ├── merge_iterator.go ├── merge_iterator_test.go ├── null.go ├── redis │ ├── builders.go │ ├── redis.go │ └── redis_test.go ├── storage.go ├── storage_test.go └── test.go ├── systemtest ├── commit_test.go ├── emitter_disconnect_test.go ├── emitter_offset_test.go ├── env.go ├── multitopic_test.go ├── proc_disconnect_test.go ├── processor_test.go ├── processor_visit_test.go ├── proxy.go ├── topicmanager_test.go ├── utils_test.go └── view_reconnect_test.go ├── tester ├── client.go ├── consumer.go ├── consumergroup.go ├── doc.go ├── producer.go ├── queue.go ├── tester.go ├── tester_test.go └── topic_manager.go ├── tools └── tools.go ├── topic_manager.go ├── topic_manager_test.go ├── view.go ├── view_test.go └── web ├── README.md ├── actions ├── action.go ├── action_test.go ├── actions.go ├── option.go └── server.go ├── index └── index.go ├── monitor ├── monitoring.go └── option.go ├── query ├── option.go └── query.go └── templates ├── actions └── index.go.html ├── common ├── base.go.html ├── head.go.html └── menu.go.html ├── index └── index.go.html ├── loader.go ├── monitor ├── details_processor.go.html ├── details_view.go.html ├── index.go.html └── menu.go.html ├── query └── index.go.html └── utils.go /.github/actions/run-application/action.yml: -------------------------------------------------------------------------------- 1 | name: 'run-application' 2 | description: 'Run an application, wait until an expected output appears, then optionally terminate the application' 3 | inputs: 4 | file: 5 | description: 'Binary to run (file name only)' 6 | required: true 7 | args: 8 | description: 'Parameters for the started binary' 9 | required: false 10 | default: '' 11 | wait-for: 12 | description: 'JSON string or array of strings to search for in the stdout/stderr output of the started application' 13 | required: true 14 | working-directory: 15 | description: 'The directory where to run the application' 16 | required: true 17 | terminate: 18 | description: 'When set to true, the started process will be terminated' 19 | required: false 20 | default: 'true' 21 | outputs: 22 | pid: 23 | description: 'PID of started process, only relevant when terminate is set to true' 24 | value: ${{ steps.runapp.outputs.pid }} 25 | runs: 26 | using: 'composite' 27 | steps: 28 | - name: 'Start application in background and save the PID' 29 | id: runapp 30 | shell: bash 31 | working-directory: '${{ inputs.working-directory }}' 32 | run: | 33 | ./${{ inputs.file }} ${{ inputs.args }} &> ${{ inputs.file }}.log & PID=$! 34 | echo "pid=${PID}" >> "$GITHUB_OUTPUT" 35 | 36 | - name: 'Wait for output to appear' 37 | shell: bash 38 | working-directory: '${{ inputs.working-directory }}' 39 | run: | 40 | python3 ${GITHUB_WORKSPACE}/examples/wait_for_output.py ${{ inputs.file }}.log '${{ inputs.wait-for }}' 41 | 42 | - name: 'Display log file' 43 | shell: bash 44 | working-directory: '${{ inputs.working-directory }}' 45 | run: | 46 | cat ${{ inputs.file }}.log 47 | if: always() 48 | 49 | - name: 'Send SIGTERM to process with pid ${{ steps.runapp.outputs.pid }}' 50 | shell: bash 51 | working-directory: '${{ inputs.working-directory }}' 52 | run: | 53 | kill ${{ steps.runapp.outputs.pid }} 54 | if: ${{ inputs.terminate == 'true' }} 55 | -------------------------------------------------------------------------------- /.github/workflows/install.yml: -------------------------------------------------------------------------------- 1 | name: install 2 | 3 | on: 4 | push: 5 | branches: [ '**' ] 6 | pull_request: 7 | branches: [master] 8 | env: 9 | GO_VERSION: '1.20' 10 | 11 | jobs: 12 | install_go_get: 13 | runs-on: ubuntu-22.04 14 | steps: 15 | - name: Set up Go 16 | uses: actions/setup-go@v3 17 | with: 18 | go-version: "^${{ env.GO_VERSION }}" 19 | 20 | - name: Install 21 | shell: bash 22 | run: | 23 | set +x 24 | go mod init test 25 | go get github.com/lovoo/goka@master 26 | cat > main.go << EndOfMessage 27 | package main 28 | 29 | import ( 30 | "context" 31 | 32 | "github.com/lovoo/goka" 33 | ) 34 | func main() { 35 | proc, _ := goka.NewProcessor(nil, goka.DefineGroup("test")) 36 | proc.Run(context.Background()) 37 | } 38 | EndOfMessage 39 | cat main.go 40 | go mod tidy 41 | cat go.mod 42 | echo "Compiling module" 43 | go build -v . 44 | echo "... done" 45 | 46 | -------------------------------------------------------------------------------- /.github/workflows/main.yml: -------------------------------------------------------------------------------- 1 | name: run tests 2 | 3 | on: 4 | push: 5 | branches: [master] 6 | pull_request: 7 | branches: [master] 8 | env: 9 | GO_VERSION: '1.20' 10 | 11 | jobs: 12 | unit: 13 | runs-on: ubuntu-22.04 14 | steps: 15 | - uses: actions/checkout@v3 16 | 17 | - name: Set up Go 18 | uses: actions/setup-go@v3 19 | with: 20 | go-version: "^${{ env.GO_VERSION }}" 21 | 22 | - name: Test 23 | shell: bash 24 | run: | 25 | go test -v -race ./... 26 | 27 | system-test: 28 | runs-on: ubuntu-22.04 29 | steps: 30 | - uses: actions/checkout@v3 31 | 32 | - name: Set up Go 33 | uses: actions/setup-go@v3 34 | with: 35 | go-version: "^${{ env.GO_VERSION }}" 36 | 37 | - name: Services 38 | shell: bash 39 | run: | 40 | sudo apt update -y && sudo apt install -y netcat docker-compose 41 | cd examples && make start && cd .. 42 | 43 | - name: Test 44 | shell: bash 45 | run: | 46 | GOKA_SYSTEMTEST=y go test -v github.com/lovoo/goka/systemtest 47 | 48 | 49 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | tmp* 2 | *.*~ 3 | .tags* 4 | vendor 5 | .vscode 6 | .idea 7 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2017 LOVOO GmbH 2 | All rights reserved. 3 | 4 | Redistribution and use in source and binary forms, with or without 5 | modification, are permitted provided that the following conditions are met: 6 | 7 | * Redistributions of source code must retain the above copyright notice, this 8 | list of conditions and the following disclaimer. 9 | 10 | * Redistributions in binary form must reproduce the above copyright notice, 11 | this list of conditions and the following disclaimer in the documentation 12 | and/or other materials provided with the distribution. 13 | 14 | * Neither the name of the copyright holder nor the names of its 15 | contributors may be used to endorse or promote products derived from 16 | this software without specific prior written permission. 17 | 18 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 19 | AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 20 | IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE 21 | DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE 22 | FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL 23 | DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR 24 | SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 25 | CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, 26 | OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 27 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 28 | -------------------------------------------------------------------------------- /MIGRATION.md: -------------------------------------------------------------------------------- 1 | This document sums up issues for migrating between 0.1.4 to 0.9.x 2 | 3 | ## restartable/auto reconnect view.Run() 4 | 5 | In 0.1.4, if a view was created to be restartable, it returned from `Run` in case of errors, but allowed 6 | to be restarted calling `Run` again. The view was still usable, even if it was not running and receiving updates. 7 | 8 | The behavior of that option has changed in 0.9.x in a way that the `view.Run` does the reconnect internally using configurable backoff. 9 | The Option was also renamed (the old version has been kept for compatibility reasons). 10 | 11 | ```go 12 | 13 | // create a view 14 | view014, _ := NewView(..., WithViewRestartable()) 15 | 16 | // reconnect logic in 0.1.4 17 | go func(){ 18 | for running{ 19 | err:= view014.Run(context.Background()) 20 | // handle error 21 | 22 | // sleep or simple backoff logic 23 | time.Sleep(time.Second) 24 | } 25 | }() 26 | 27 | // After migration: 28 | // create a view 29 | view09x, _ := NewView(..., WithViewAutoReconnect()) 30 | ctx, cancel := context.WithCancel(context.Background()) 31 | 32 | // no need for reconnect logic, it's handled by the view internally 33 | go func(){ 34 | err:= view09x.Run(ctx) 35 | // handle shutdown error 36 | }() 37 | 38 | // stop view 39 | cancel() 40 | 41 | ``` 42 | 43 | 44 | ## Offset bug in local storage 45 | In 0.1.4 there was a bug that caused the table offset being stored in the local cache always be +1 compared the actual offset stored in kafka. 46 | A second bug kind of evened it out so it never was an issue. 47 | 48 | From 0.9.x, both bugs are fixed. However, if you upgrade goka and restart a processor using the same cache files that were maintained by the old version you'll see a warning like this 49 | ``` 50 | Error: local offset is higher than partition offset. topic some-topic, partition 0, hwm 1312, local offset 1314. This can have several reasons: 51 | (1) The kafka topic storing the table is gone --> delete the local cache and restart! 52 | (2) the processor crashed last time while writing to disk. 53 | (3) You found a bug! 54 | ``` 55 | This is because goka sees an offset that it is not expecting. 56 | You should see this error only once per partition and processor. The offset will be fixed automatically. If it appears on every start or regularily, it might actually a bug or some error and should be further investigated 57 | (or reported to goka :)). -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | 2 | 3 | test: 4 | go test -race ./... 5 | 6 | test-systemtest: 7 | GOKA_SYSTEMTEST=y go test -v github.com/lovoo/goka/systemtest 8 | 9 | test-all: test test-systemtest 10 | -------------------------------------------------------------------------------- /assignment.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "fmt" 5 | ) 6 | 7 | // Assignment represents a partition:offset assignment for the current connection 8 | type Assignment map[int32]int64 9 | 10 | func (a *Assignment) string() string { 11 | var am map[int32]int64 = *a 12 | return fmt.Sprintf("Assignment %v", am) 13 | } 14 | -------------------------------------------------------------------------------- /broker.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import "github.com/IBM/sarama" 4 | 5 | // Broker is an interface for the sarama broker 6 | type Broker interface { 7 | Addr() string 8 | Connected() (bool, error) 9 | CreateTopics(request *sarama.CreateTopicsRequest) (*sarama.CreateTopicsResponse, error) 10 | Open(conf *sarama.Config) error 11 | } 12 | -------------------------------------------------------------------------------- /builders.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "hash" 5 | 6 | "github.com/IBM/sarama" 7 | ) 8 | 9 | // ProducerBuilder create a Kafka producer. 10 | type ProducerBuilder func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) 11 | 12 | // DefaultProducerBuilder creates a Kafka producer using the Sarama library. 13 | func DefaultProducerBuilder(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { 14 | config := globalConfig 15 | config.ClientID = clientID 16 | config.Producer.Partitioner = sarama.NewCustomHashPartitioner(hasher) 17 | return NewProducer(brokers, &config) 18 | } 19 | 20 | // ProducerBuilderWithConfig creates a Kafka consumer using the Sarama library. 21 | func ProducerBuilderWithConfig(config *sarama.Config) ProducerBuilder { 22 | return func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { 23 | config.ClientID = clientID 24 | config.Producer.Partitioner = sarama.NewCustomHashPartitioner(hasher) 25 | return NewProducer(brokers, config) 26 | } 27 | } 28 | 29 | // TopicManagerBuilder creates a TopicManager to check partition counts and 30 | // create tables. 31 | type TopicManagerBuilder func(brokers []string) (TopicManager, error) 32 | 33 | // DefaultTopicManagerBuilder creates TopicManager using the Sarama library. 34 | func DefaultTopicManagerBuilder(brokers []string) (TopicManager, error) { 35 | config := globalConfig 36 | config.ClientID = "goka-topic-manager" 37 | return NewTopicManager(brokers, &config, NewTopicManagerConfig()) 38 | } 39 | 40 | // TopicManagerBuilderWithConfig creates TopicManager using the Sarama library. 41 | func TopicManagerBuilderWithConfig(config *sarama.Config, tmConfig *TopicManagerConfig) TopicManagerBuilder { 42 | return func(brokers []string) (TopicManager, error) { 43 | return NewTopicManager(brokers, config, tmConfig) 44 | } 45 | } 46 | 47 | // TopicManagerBuilderWithTopicManagerConfig creates TopicManager using the Sarama library. 48 | func TopicManagerBuilderWithTopicManagerConfig(tmConfig *TopicManagerConfig) TopicManagerBuilder { 49 | return func(brokers []string) (TopicManager, error) { 50 | config := globalConfig 51 | config.ClientID = "goka-topic-manager" 52 | return NewTopicManager(brokers, &config, tmConfig) 53 | } 54 | } 55 | 56 | // ConsumerGroupBuilder creates a `sarama.ConsumerGroup` 57 | type ConsumerGroupBuilder func(brokers []string, group, clientID string) (sarama.ConsumerGroup, error) 58 | 59 | // DefaultConsumerGroupBuilder creates a Kafka consumer using the Sarama library. 60 | func DefaultConsumerGroupBuilder(brokers []string, group, clientID string) (sarama.ConsumerGroup, error) { 61 | config := globalConfig 62 | config.ClientID = clientID 63 | return sarama.NewConsumerGroup(brokers, group, &config) 64 | } 65 | 66 | // ConsumerGroupBuilderWithConfig creates a sarama consumergroup using passed config 67 | func ConsumerGroupBuilderWithConfig(config *sarama.Config) ConsumerGroupBuilder { 68 | return func(brokers []string, group, clientID string) (sarama.ConsumerGroup, error) { 69 | config.ClientID = clientID 70 | return sarama.NewConsumerGroup(brokers, group, config) 71 | } 72 | } 73 | 74 | // SaramaConsumerBuilder creates a `sarama.Consumer` 75 | type SaramaConsumerBuilder func(brokers []string, clientID string) (sarama.Consumer, error) 76 | 77 | // DefaultSaramaConsumerBuilder creates a Kafka consumer using the Sarama library. 78 | func DefaultSaramaConsumerBuilder(brokers []string, clientID string) (sarama.Consumer, error) { 79 | config := globalConfig 80 | config.ClientID = clientID 81 | return sarama.NewConsumer(brokers, &config) 82 | } 83 | 84 | // SaramaConsumerBuilderWithConfig creates a sarama consumer using passed config 85 | func SaramaConsumerBuilderWithConfig(config *sarama.Config) SaramaConsumerBuilder { 86 | return func(brokers []string, clientID string) (sarama.Consumer, error) { 87 | config.ClientID = clientID 88 | return sarama.NewConsumer(brokers, config) 89 | } 90 | } 91 | 92 | // BackoffBuilder creates a backoff 93 | type BackoffBuilder func() (Backoff, error) 94 | 95 | // DefaultBackoffBuilder returnes a simpleBackoff with 10 seconds step increase and 2 minutes max wait 96 | func DefaultBackoffBuilder() (Backoff, error) { 97 | return NewSimpleBackoff(defaultBackoffStep, defaultBackoffMax), nil 98 | } 99 | -------------------------------------------------------------------------------- /codec.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | // Codec decodes and encodes from and to []byte 4 | type Codec interface { 5 | Encode(value interface{}) (data []byte, err error) 6 | Decode(data []byte) (value interface{}, err error) 7 | } 8 | -------------------------------------------------------------------------------- /codec/codec.go: -------------------------------------------------------------------------------- 1 | /* 2 | Package codec provides a set of codecs to encode and decode various data types to and from 3 | byte slices([]byte). 4 | */ 5 | package codec 6 | 7 | import ( 8 | "fmt" 9 | "strconv" 10 | ) 11 | 12 | // Bytes is a commonly used codec to encode and decode string <-> []byte 13 | type Bytes struct{} 14 | 15 | // Encode does a type conversion into []byte 16 | func (d *Bytes) Encode(value interface{}) ([]byte, error) { 17 | var err error 18 | data, isByte := value.([]byte) 19 | if !isByte { 20 | err = fmt.Errorf("DefaultCodec: value to encode is not of type []byte") 21 | } 22 | return data, err 23 | } 24 | 25 | // Decode of defaultCodec simply returns the data 26 | func (d *Bytes) Decode(data []byte) (interface{}, error) { 27 | return data, nil 28 | } 29 | 30 | // String is a commonly used codec to encode and decode string <-> []byte 31 | type String struct{} 32 | 33 | // Encode encodes from string to []byte 34 | func (c *String) Encode(value interface{}) ([]byte, error) { 35 | stringVal, isString := value.(string) 36 | if !isString { 37 | return nil, fmt.Errorf("String: value to encode is not of type string but %T", value) 38 | } 39 | return []byte(stringVal), nil 40 | } 41 | 42 | // Decode decodes from []byte to string 43 | func (c *String) Decode(data []byte) (interface{}, error) { 44 | return string(data), nil 45 | } 46 | 47 | // Int64 is a commonly used codec to encode and decode string <-> []byte 48 | type Int64 struct{} 49 | 50 | // Encode encodes from string to []byte 51 | func (c *Int64) Encode(value interface{}) ([]byte, error) { 52 | intVal, isInt := value.(int64) 53 | if !isInt { 54 | return nil, fmt.Errorf("Int64: value to encode is not of type int64") 55 | } 56 | return []byte(strconv.FormatInt(intVal, 10)), nil 57 | } 58 | 59 | // Decode decodes from []byte to string 60 | func (c *Int64) Decode(data []byte) (interface{}, error) { 61 | intVal, err := strconv.ParseInt(string(data), 10, 64) 62 | if err != nil { 63 | return 0, fmt.Errorf("Error parsing data from string %d: %v", intVal, err) 64 | } 65 | return intVal, nil 66 | } 67 | -------------------------------------------------------------------------------- /config.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "time" 5 | 6 | "github.com/IBM/sarama" 7 | ) 8 | 9 | var ( 10 | globalConfig = *DefaultConfig() 11 | ) 12 | 13 | const ( 14 | // size of sarama buffer for consumer and producer 15 | defaultChannelBufferSize = 256 16 | 17 | // time sarama-cluster assumes the processing of an event may take 18 | defaultMaxProcessingTime = 1 * time.Second 19 | 20 | // producer flush configuration 21 | defaultFlushFrequency = 100 * time.Millisecond 22 | defaultFlushBytes = 64 * 1024 23 | defaultProducerMaxRetries = 10 24 | ) 25 | 26 | // DefaultConfig creates a new config used by goka per default 27 | // Use it to modify and pass to `goka.ReplaceGlobalConifg(...)` to modify 28 | // goka's global config 29 | func DefaultConfig() *sarama.Config { 30 | config := sarama.NewConfig() 31 | config.Version = sarama.V2_0_0_0 32 | 33 | // consumer configuration 34 | config.Consumer.Return.Errors = true 35 | config.Consumer.MaxProcessingTime = defaultMaxProcessingTime 36 | // this configures the initial offset for streams. Tables are always 37 | // consumed from OffsetOldest. 38 | config.Consumer.Offsets.Initial = sarama.OffsetNewest 39 | config.Consumer.Group.Rebalance.Strategy = CopartitioningStrategy 40 | // producer configuration 41 | config.Producer.RequiredAcks = sarama.WaitForLocal 42 | config.Producer.Compression = sarama.CompressionSnappy 43 | config.Producer.Flush.Frequency = defaultFlushFrequency 44 | config.Producer.Flush.Bytes = defaultFlushBytes 45 | config.Producer.Return.Successes = true 46 | config.Producer.Return.Errors = true 47 | config.Producer.Retry.Max = defaultProducerMaxRetries 48 | return config 49 | } 50 | 51 | // ReplaceGlobalConfig registeres a standard config used during building if no 52 | // other config is specified 53 | func ReplaceGlobalConfig(config *sarama.Config) { 54 | if config == nil { 55 | panic("nil config registered as global config") 56 | } 57 | globalConfig = *config 58 | } 59 | -------------------------------------------------------------------------------- /config_test.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/IBM/sarama" 7 | "github.com/stretchr/testify/require" 8 | ) 9 | 10 | func TestConfig_DefaultConfig(t *testing.T) { 11 | t.Run("equal", func(t *testing.T) { 12 | cfg := DefaultConfig() 13 | require.True(t, cfg.Version == sarama.V2_0_0_0) 14 | require.True(t, cfg.Consumer.Return.Errors == true) 15 | require.True(t, cfg.Consumer.MaxProcessingTime == defaultMaxProcessingTime) 16 | require.True(t, cfg.Consumer.Offsets.Initial == sarama.OffsetNewest) 17 | require.True(t, cfg.Producer.RequiredAcks == sarama.WaitForLocal) 18 | require.True(t, cfg.Producer.Compression == sarama.CompressionSnappy) 19 | require.True(t, cfg.Producer.Flush.Frequency == defaultFlushFrequency) 20 | require.True(t, cfg.Producer.Flush.Bytes == defaultFlushBytes) 21 | require.True(t, cfg.Producer.Return.Successes == true) 22 | require.True(t, cfg.Producer.Return.Errors == true) 23 | require.True(t, cfg.Producer.Retry.Max == defaultProducerMaxRetries) 24 | }) 25 | } 26 | 27 | func TestConfig_ReplaceGlobalConfig(t *testing.T) { 28 | t.Run("succeed", func(t *testing.T) { 29 | custom := DefaultConfig() 30 | custom.Version = sarama.V0_8_2_0 31 | ReplaceGlobalConfig(custom) 32 | require.Equal(t, custom.Version, globalConfig.Version) 33 | }) 34 | t.Run("panic", func(t *testing.T) { 35 | defer func() { 36 | if r := recover(); r == nil { 37 | t.Fatal("there was no panic") 38 | } 39 | }() 40 | ReplaceGlobalConfig(nil) 41 | }) 42 | } 43 | -------------------------------------------------------------------------------- /doc.go: -------------------------------------------------------------------------------- 1 | //go:generate go run go.uber.org/mock/mockgen -self_package github.com/lovoo/goka -package goka -destination mockstorage.go github.com/lovoo/goka/storage Storage 2 | //go:generate go run go.uber.org/mock/mockgen -self_package github.com/lovoo/goka -package goka -destination mocks.go github.com/lovoo/goka TopicManager,Producer,Broker 3 | //go:generate go run go.uber.org/mock/mockgen -self_package github.com/lovoo/goka -package goka -destination mockssarama.go github.com/IBM/sarama Client,ClusterAdmin 4 | 5 | /* 6 | Package goka is a stateful stream processing library for Apache Kafka (version 0.9+) that eases 7 | the development of microservices. 8 | Goka extends the concept of consumer group with a group table, which represents the state of the 9 | group. 10 | A microservice modifies and serves the content of a table employing two complementary object types: 11 | processors and views. 12 | 13 | # Processors 14 | 15 | A processor is a set of callback functions that modify the group table when messages arrive and may 16 | also emit messages into other topics. 17 | Messages as well as rows in the group table are key-value pairs. 18 | Callbacks receive the arriving message and the row addressed by the message's key. 19 | 20 | In Kafka, keys are used to partition topics. 21 | A goka processor consumes from a set of co-partitioned topics (topics with the same number of 22 | partitions and the same key range). 23 | A group topic keeps track of the group table updates, allowing for recovery and rebalancing of 24 | processors: 25 | When multiple processor instances start in the same consumer group, the instances split the 26 | co-partitioned input topics and load the respective group table partitions from the group topic. 27 | A local disk storage minimizes recovery time by caching partitions of group table. 28 | 29 | # Views 30 | 31 | A view is a materialized (ie, persistent) cache of a group table. 32 | A view subscribes for the updates of all partitions of a group table and keeps local disk storage 33 | in sync with the group topic. 34 | With a view, one can easily serve up-to-date content of the group table via, for example, gRPC. 35 | */ 36 | package goka 37 | -------------------------------------------------------------------------------- /emitter.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "errors" 5 | "fmt" 6 | "sync" 7 | ) 8 | 9 | var ( 10 | // ErrEmitterAlreadyClosed is returned when Emit is called after the emitter has been finished. 11 | ErrEmitterAlreadyClosed error = errors.New("emitter already closed") 12 | ) 13 | 14 | // Emitter emits messages into a specific Kafka topic, first encoding the message with the given codec. 15 | type Emitter struct { 16 | codec Codec 17 | producer Producer 18 | 19 | topic string 20 | defaultHeaders Headers 21 | 22 | wg sync.WaitGroup 23 | mu sync.RWMutex 24 | done chan struct{} 25 | } 26 | 27 | // NewEmitter creates a new emitter using passed brokers, topic, codec and possibly options. 28 | func NewEmitter(brokers []string, topic Stream, codec Codec, options ...EmitterOption) (*Emitter, error) { 29 | options = append( 30 | // default options comes first 31 | []EmitterOption{ 32 | WithEmitterClientID(fmt.Sprintf("goka-emitter-%s", topic)), 33 | }, 34 | 35 | // user-defined options (may overwrite default ones) 36 | options..., 37 | ) 38 | 39 | opts := new(eoptions) 40 | 41 | opts.applyOptions(topic, codec, options...) 42 | 43 | prod, err := opts.builders.producer(brokers, opts.clientID, opts.hasher) 44 | if err != nil { 45 | return nil, fmt.Errorf(errBuildProducer, err) 46 | } 47 | 48 | return &Emitter{ 49 | codec: codec, 50 | producer: prod, 51 | topic: string(topic), 52 | defaultHeaders: opts.defaultHeaders, 53 | done: make(chan struct{}), 54 | }, nil 55 | } 56 | 57 | func (e *Emitter) emitDone(err error) { e.wg.Done() } 58 | 59 | // EmitWithHeaders sends a message with the given headers for the passed key using the emitter's codec. 60 | func (e *Emitter) EmitWithHeaders(key string, msg interface{}, headers Headers) (*Promise, error) { 61 | var ( 62 | err error 63 | data []byte 64 | ) 65 | 66 | if msg != nil { 67 | data, err = e.codec.Encode(msg) 68 | if err != nil { 69 | return nil, fmt.Errorf("Error encoding value for key %s in topic %s: %v", key, e.topic, err) 70 | } 71 | } 72 | 73 | // protect e.done channel and e.wg WaitGroup together to reject all new emits after calling e.Finish 74 | // wg.Add must not be called after wg.Wait finished 75 | e.mu.RLock() 76 | select { 77 | case <-e.done: 78 | e.mu.RUnlock() 79 | return NewPromise().finish(nil, ErrEmitterAlreadyClosed), nil 80 | default: 81 | e.wg.Add(1) 82 | e.mu.RUnlock() 83 | } 84 | 85 | if headers == nil && e.defaultHeaders == nil { 86 | return e.producer.Emit(e.topic, key, data).Then(e.emitDone), nil 87 | } 88 | 89 | return e.producer.EmitWithHeaders(e.topic, key, data, e.defaultHeaders.Merged(headers)).Then(e.emitDone), nil 90 | } 91 | 92 | // Emit sends a message for passed key using the emitter's codec. 93 | func (e *Emitter) Emit(key string, msg interface{}) (*Promise, error) { 94 | return e.EmitWithHeaders(key, msg, nil) 95 | } 96 | 97 | // EmitSyncWithHeaders sends a message with the given headers to passed topic and key. 98 | func (e *Emitter) EmitSyncWithHeaders(key string, msg interface{}, headers Headers) error { 99 | var ( 100 | err error 101 | promise *Promise 102 | ) 103 | promise, err = e.EmitWithHeaders(key, msg, headers) 104 | 105 | if err != nil { 106 | return err 107 | } 108 | 109 | done := make(chan struct{}) 110 | promise.Then(func(asyncErr error) { 111 | err = asyncErr 112 | close(done) 113 | }) 114 | <-done 115 | return err 116 | } 117 | 118 | // EmitSync sends a message to passed topic and key. 119 | func (e *Emitter) EmitSync(key string, msg interface{}) error { 120 | return e.EmitSyncWithHeaders(key, msg, nil) 121 | } 122 | 123 | // Finish waits until the emitter is finished producing all pending messages. 124 | func (e *Emitter) Finish() error { 125 | e.mu.Lock() 126 | close(e.done) 127 | e.mu.Unlock() 128 | e.wg.Wait() 129 | return e.producer.Close() 130 | } 131 | -------------------------------------------------------------------------------- /errors.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "errors" 5 | "fmt" 6 | reflect "reflect" 7 | "regexp" 8 | "strings" 9 | 10 | "github.com/go-stack/stack" 11 | ) 12 | 13 | var ( 14 | errBuildConsumer = "error creating Kafka consumer: %v" 15 | errBuildProducer = "error creating Kafka producer: %v" 16 | errApplyOptions = "error applying options: %v" 17 | errTopicNotFound = errors.New("requested topic was not found") 18 | ) 19 | 20 | type topicNoCreateError string 21 | 22 | func (e topicNoCreateError) Error() string { 23 | return fmt.Sprintf("topic '%s' does not exist but the manager is configured with NoCreate, so it will not attempt to create it", string(e)) 24 | } 25 | 26 | // this regex matches the package name + some hash info, if we're in gomod but not subpackages 27 | // examples which match 28 | // * github.com/lovoo/goka/processor.go 29 | // * github.com/lovoo/goka@v1.0.0/view.go 30 | // * github.com/some-fork/goka/view.go 31 | // examples which do not match 32 | // * github.com/something/else 33 | // * github.com/lovoo/goka/subpackage/file.go 34 | // this regex is used to filter out entries from the stack trace that origin 35 | // from the root-package of go (but not the subpackages, otherwise we would not see the stack in the example-tests) 36 | // reflect.TypeOf(Processor{}).PkgPath() returns (in the main repo) "github.com/lovoo/goka" 37 | var gokaPackageRegex = regexp.MustCompile(fmt.Sprintf(`%s(?:@[^/]+)?/[^/]+$`, reflect.TypeOf(Processor{}).PkgPath())) 38 | 39 | // ErrVisitAborted indicates a call to VisitAll could not finish due to rebalance or processor shutdown 40 | var ErrVisitAborted = errors.New("VisitAll aborted due to context cancel or rebalance") 41 | 42 | // type to indicate that some non-transient error occurred while processing 43 | // the message, e.g. panic, decoding/encoding errors or invalid usage of context. 44 | type errProcessing struct { 45 | partition int32 46 | err error 47 | } 48 | 49 | func (ec *errProcessing) Error() string { 50 | return fmt.Sprintf("error processing message (partition=%d): %v", ec.partition, ec.err) 51 | } 52 | 53 | func newErrProcessing(partition int32, err error) error { 54 | return &errProcessing{ 55 | partition: partition, 56 | err: err, 57 | } 58 | } 59 | 60 | func (ec *errProcessing) Unwrap() error { 61 | return ec.err 62 | } 63 | 64 | // type to indicate that some non-transient error occurred while setting up the partitions on 65 | // rebalance. 66 | type errSetup struct { 67 | partition int32 68 | err error 69 | } 70 | 71 | func (ec *errSetup) Error() string { 72 | return fmt.Sprintf("error setting up (partition=%d): %v", ec.partition, ec.err) 73 | } 74 | 75 | func (ec *errSetup) Unwrap() error { 76 | return ec.err 77 | } 78 | 79 | func newErrSetup(partition int32, err error) error { 80 | return &errSetup{ 81 | partition: partition, 82 | err: err, 83 | } 84 | } 85 | 86 | // userStacktrace returns a formatted stack trace only containing the stack trace of the user-code 87 | // This is mainly used to properly format the error message built after a panic happened in a 88 | // processor-callback. 89 | func userStacktrace() []string { 90 | trace := stack.Trace() 91 | 92 | // pop calls from the top that are either from runtime or goka's internal functions 93 | for len(trace) > 0 { 94 | if strings.HasPrefix(fmt.Sprintf("%+s", trace[0]), "runtime/") { 95 | trace = trace[1:] 96 | continue 97 | } 98 | if gokaPackageRegex.MatchString(fmt.Sprintf("%+s", trace[0])) { 99 | trace = trace[1:] 100 | continue 101 | } 102 | break 103 | } 104 | 105 | var lines []string 106 | for _, frame := range trace { 107 | 108 | // as soon as we hit goka's internal package again we'll stop because from this point on we would 109 | // only print library or runtime frames 110 | if gokaPackageRegex.MatchString(fmt.Sprintf("%+s", frame)) { 111 | break 112 | } 113 | lines = append(lines, fmt.Sprintf("%n\n\t%+s:%d", frame, frame, frame)) 114 | } 115 | 116 | // if we don't have anything unfiltered, it means there was an error within goka itself, so we should just 117 | // return the whole stack trace. 118 | if len(lines) == 0 { 119 | for _, frame := range stack.Trace() { 120 | lines = append(lines, fmt.Sprintf("%n\n\t%+s:%d", frame, frame, frame)) 121 | } 122 | } 123 | 124 | return lines 125 | } 126 | -------------------------------------------------------------------------------- /examples/.env: -------------------------------------------------------------------------------- 1 | REPLICATION_FACTOR=1 2 | NUM_PARTITIONS=10 -------------------------------------------------------------------------------- /examples/1-simplest/README.md: -------------------------------------------------------------------------------- 1 | # Simplest example 2 | 3 | This example shows how to implement a simple application using an Emitter, Processor and a View. 4 | The application counts the clicks of a user in the processor's group table. 5 | 6 | Details are explained in the project's [readme]. 7 | 8 | [readme]: https://github.com/lovoo/goka/blob/master/README.md 9 | -------------------------------------------------------------------------------- /examples/1-simplest/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "context" 5 | "log" 6 | "os" 7 | "os/signal" 8 | "syscall" 9 | 10 | "github.com/IBM/sarama" 11 | "github.com/lovoo/goka" 12 | "github.com/lovoo/goka/codec" 13 | ) 14 | 15 | var ( 16 | brokers = []string{"localhost:9092"} 17 | topic goka.Stream = "example-stream" 18 | group goka.Group = "example-group" 19 | 20 | tmc *goka.TopicManagerConfig 21 | ) 22 | 23 | func init() { 24 | // This sets the default replication to 1. If you have more then one broker 25 | // the default configuration can be used. 26 | tmc = goka.NewTopicManagerConfig() 27 | tmc.Table.Replication = 1 28 | tmc.Stream.Replication = 1 29 | } 30 | 31 | // emits a single message and leave 32 | func runEmitter() { 33 | emitter, err := goka.NewEmitter(brokers, topic, new(codec.String)) 34 | if err != nil { 35 | log.Fatalf("error creating emitter: %v", err) 36 | } 37 | defer emitter.Finish() 38 | err = emitter.EmitSync("some-key", "some-value") 39 | if err != nil { 40 | log.Fatalf("error emitting message: %v", err) 41 | } 42 | log.Println("message emitted") 43 | } 44 | 45 | // process messages until ctrl-c is pressed 46 | func runProcessor() { 47 | // process callback is invoked for each message delivered from 48 | // "example-stream" topic. 49 | cb := func(ctx goka.Context, msg interface{}) { 50 | var counter int64 51 | // ctx.Value() gets from the group table the value that is stored for 52 | // the message's key. 53 | if val := ctx.Value(); val != nil { 54 | counter = val.(int64) 55 | } 56 | counter++ 57 | // SetValue stores the incremented counter in the group table for in 58 | // the message's key. 59 | ctx.SetValue(counter) 60 | log.Printf("key = %s, counter = %v, msg = %v", ctx.Key(), counter, msg) 61 | } 62 | 63 | // Define a new processor group. The group defines all inputs, outputs, and 64 | // serialization formats. The group-table topic is "example-group-table". 65 | g := goka.DefineGroup(group, 66 | goka.Input(topic, new(codec.String), cb), 67 | goka.Persist(new(codec.Int64)), 68 | ) 69 | 70 | p, err := goka.NewProcessor(brokers, 71 | g, 72 | goka.WithTopicManagerBuilder(goka.TopicManagerBuilderWithTopicManagerConfig(tmc)), 73 | goka.WithConsumerGroupBuilder(goka.DefaultConsumerGroupBuilder), 74 | ) 75 | if err != nil { 76 | log.Fatalf("error creating processor: %v", err) 77 | } 78 | ctx, cancel := context.WithCancel(context.Background()) 79 | done := make(chan struct{}) 80 | go func() { 81 | defer close(done) 82 | if err = p.Run(ctx); err != nil { 83 | log.Printf("error running processor: %v", err) 84 | } 85 | }() 86 | 87 | sigs := make(chan os.Signal) 88 | go func() { 89 | signal.Notify(sigs, syscall.SIGINT, syscall.SIGTERM, syscall.SIGKILL) 90 | }() 91 | 92 | select { 93 | case <-sigs: 94 | case <-done: 95 | } 96 | cancel() 97 | <-done 98 | } 99 | 100 | func main() { 101 | config := goka.DefaultConfig() 102 | // since the emitter only emits one message, we need to tell the processor 103 | // to read from the beginning 104 | // As the processor is slower to start than the emitter, it would not consume the first 105 | // message otherwise. 106 | // In production systems however, check whether you really want to read the whole topic on first start, which 107 | // can be a lot of messages. 108 | config.Consumer.Offsets.Initial = sarama.OffsetOldest 109 | goka.ReplaceGlobalConfig(config) 110 | 111 | tm, err := goka.NewTopicManager(brokers, goka.DefaultConfig(), tmc) 112 | if err != nil { 113 | log.Fatalf("Error creating topic manager: %v", err) 114 | } 115 | defer tm.Close() 116 | err = tm.EnsureStreamExists(string(topic), 8) 117 | if err != nil { 118 | log.Printf("Error creating kafka topic %s: %v", topic, err) 119 | } 120 | 121 | runEmitter() // emits one message and stops 122 | runProcessor() // press ctrl-c to stop 123 | } 124 | -------------------------------------------------------------------------------- /examples/10-visit/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "log" 7 | "os" 8 | "os/signal" 9 | "syscall" 10 | "time" 11 | 12 | "github.com/lovoo/goka" 13 | "github.com/lovoo/goka/codec" 14 | "github.com/lovoo/goka/multierr" 15 | ) 16 | 17 | var ( 18 | brokers = []string{"localhost:9092"} 19 | topic goka.Stream = "example-visit-clicks-input" 20 | group goka.Group = "example-visit-group" 21 | 22 | tmc *goka.TopicManagerConfig 23 | ) 24 | 25 | func init() { 26 | // This sets the default replication to 1. If you have more then one broker 27 | // the default configuration can be used. 28 | tmc = goka.NewTopicManagerConfig() 29 | tmc.Table.Replication = 1 30 | tmc.Stream.Replication = 1 31 | } 32 | 33 | // emit messages until stopped 34 | func runEmitter(ctx context.Context) { 35 | emitter, err := goka.NewEmitter(brokers, topic, new(codec.Int64)) 36 | if err != nil { 37 | log.Fatalf("error creating emitter: %v", err) 38 | } 39 | defer emitter.Finish() 40 | ticker := time.NewTicker(100 * time.Millisecond) 41 | defer ticker.Stop() 42 | for i := 0; ; i++ { 43 | select { 44 | case <-ticker.C: 45 | 46 | err = emitter.EmitSync(fmt.Sprintf("key-%d", i%10), int64(1)) 47 | if err != nil { 48 | log.Fatalf("error emitting message: %v", err) 49 | } 50 | case <-ctx.Done(): 51 | return 52 | } 53 | } 54 | } 55 | 56 | func main() { 57 | 58 | tm, err := goka.NewTopicManager(brokers, goka.DefaultConfig(), tmc) 59 | if err != nil { 60 | log.Fatalf("Error creating topic manager: %v", err) 61 | } 62 | defer tm.Close() 63 | err = tm.EnsureStreamExists(string(topic), 8) 64 | if err != nil { 65 | log.Fatalf("Error creating kafka topic %s: %v", topic, err) 66 | } 67 | 68 | sigs := make(chan os.Signal) 69 | go func() { 70 | signal.Notify(sigs, syscall.SIGINT, syscall.SIGTERM, syscall.SIGKILL) 71 | }() 72 | 73 | ctx, cancel := context.WithCancel(context.Background()) 74 | defer cancel() 75 | errg, ctx := multierr.NewErrGroup(ctx) 76 | 77 | g := goka.DefineGroup(group, 78 | goka.Input(topic, new(codec.Int64), func(ctx goka.Context, msg interface{}) { 79 | var counter int64 80 | if val := ctx.Value(); val != nil { 81 | counter = val.(int64) 82 | } 83 | counter += msg.(int64) 84 | log.Printf("%s: %d", ctx.Key(), counter) 85 | ctx.SetValue(counter) 86 | 87 | }), 88 | goka.Visitor("reset", func(ctx goka.Context, meta interface{}) { 89 | log.Printf("resetting %s: %d", ctx.Key(), meta.(int64)) 90 | ctx.SetValue(meta) 91 | }), 92 | goka.Persist(new(codec.Int64)), 93 | ) 94 | 95 | proc, err := goka.NewProcessor(brokers, 96 | g, 97 | goka.WithTopicManagerBuilder(goka.TopicManagerBuilderWithTopicManagerConfig(tmc)), 98 | ) 99 | if err != nil { 100 | log.Fatalf("error creating processor: %v", err) 101 | } 102 | 103 | // start the emitter 104 | errg.Go(func() error { 105 | runEmitter(ctx) 106 | return nil 107 | }) 108 | 109 | // start the processor 110 | errg.Go(func() error { 111 | return proc.Run(ctx) 112 | }) 113 | 114 | errg.Go(func() error { 115 | select { 116 | case <-sigs: 117 | case <-ctx.Done(): 118 | } 119 | cancel() 120 | return nil 121 | }) 122 | 123 | time.Sleep(5 * time.Second) 124 | 125 | visited, err := proc.VisitAllWithStats(ctx, "reset", int64(0)) 126 | if err != nil { 127 | log.Printf("error visiting: %v", err) 128 | } 129 | 130 | log.Printf("visited %d values", visited) 131 | 132 | time.Sleep(5 * time.Second) 133 | log.Printf("stopping...") 134 | cancel() 135 | if err := errg.Wait().ErrorOrNil(); err != nil { 136 | log.Fatalf("error running: %v", err) 137 | } 138 | log.Printf("done.") 139 | 140 | } 141 | -------------------------------------------------------------------------------- /examples/2-clicks/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "context" 5 | "encoding/json" 6 | "fmt" 7 | "log" 8 | "net/http" 9 | "time" 10 | 11 | "github.com/gorilla/mux" 12 | "github.com/lovoo/goka" 13 | "github.com/lovoo/goka/codec" 14 | ) 15 | 16 | var ( 17 | brokers = []string{"127.0.0.1:9092"} 18 | topic goka.Stream = "user-click" 19 | group goka.Group = "mini-group" 20 | 21 | tmc *goka.TopicManagerConfig 22 | ) 23 | 24 | // A user is the object that is stored in the processor's group table 25 | type user struct { 26 | // number of clicks the user has performed. 27 | Clicks int 28 | } 29 | 30 | // This codec allows marshalling (encode) and unmarshalling (decode) the user to and from the 31 | // group table. 32 | type userCodec struct{} 33 | 34 | func init() { 35 | tmc = goka.NewTopicManagerConfig() 36 | tmc.Table.Replication = 1 37 | tmc.Stream.Replication = 1 38 | } 39 | 40 | // Encodes a user into []byte 41 | func (jc *userCodec) Encode(value interface{}) ([]byte, error) { 42 | if _, isUser := value.(*user); !isUser { 43 | return nil, fmt.Errorf("Codec requires value *user, got %T", value) 44 | } 45 | return json.Marshal(value) 46 | } 47 | 48 | // Decodes a user from []byte to it's go representation. 49 | func (jc *userCodec) Decode(data []byte) (interface{}, error) { 50 | var ( 51 | c user 52 | err error 53 | ) 54 | err = json.Unmarshal(data, &c) 55 | if err != nil { 56 | return nil, fmt.Errorf("Error unmarshaling user: %v", err) 57 | } 58 | return &c, nil 59 | } 60 | 61 | func runEmitter() { 62 | emitter, err := goka.NewEmitter(brokers, topic, 63 | new(codec.String)) 64 | if err != nil { 65 | panic(err) 66 | } 67 | defer emitter.Finish() 68 | 69 | t := time.NewTicker(100 * time.Millisecond) 70 | defer t.Stop() 71 | 72 | var i int 73 | for range t.C { 74 | key := fmt.Sprintf("user-%d", i%10) 75 | value := fmt.Sprintf("%s", time.Now()) 76 | emitter.EmitSync(key, value) 77 | i++ 78 | } 79 | } 80 | 81 | func process(ctx goka.Context, msg interface{}) { 82 | var u *user 83 | if val := ctx.Value(); val != nil { 84 | u = val.(*user) 85 | } else { 86 | u = new(user) 87 | } 88 | 89 | u.Clicks++ 90 | ctx.SetValue(u) 91 | fmt.Printf("[proc] key: %s clicks: %d, msg: %v\n", ctx.Key(), u.Clicks, msg) 92 | } 93 | 94 | func runProcessor(initialized chan struct{}) { 95 | g := goka.DefineGroup(group, 96 | goka.Input(topic, new(codec.String), process), 97 | goka.Persist(new(userCodec)), 98 | ) 99 | p, err := goka.NewProcessor(brokers, 100 | g, 101 | goka.WithTopicManagerBuilder(goka.TopicManagerBuilderWithTopicManagerConfig(tmc)), 102 | goka.WithConsumerGroupBuilder(goka.DefaultConsumerGroupBuilder), 103 | ) 104 | if err != nil { 105 | panic(err) 106 | } 107 | 108 | close(initialized) 109 | 110 | p.Run(context.Background()) 111 | } 112 | 113 | func runView(initialized chan struct{}) { 114 | <-initialized 115 | 116 | view, err := goka.NewView(brokers, 117 | goka.GroupTable(group), 118 | new(userCodec), 119 | ) 120 | if err != nil { 121 | panic(err) 122 | } 123 | 124 | root := mux.NewRouter() 125 | root.HandleFunc("/{key}", func(w http.ResponseWriter, r *http.Request) { 126 | value, _ := view.Get(mux.Vars(r)["key"]) 127 | data, _ := json.Marshal(value) 128 | w.Write(data) 129 | }) 130 | fmt.Println("View opened at http://localhost:9095/") 131 | go http.ListenAndServe(":9095", root) 132 | 133 | view.Run(context.Background()) 134 | } 135 | 136 | func main() { 137 | tm, err := goka.NewTopicManager(brokers, goka.DefaultConfig(), tmc) 138 | if err != nil { 139 | log.Fatalf("Error creating topic manager: %v", err) 140 | } 141 | defer tm.Close() 142 | err = tm.EnsureStreamExists(string(topic), 8) 143 | if err != nil { 144 | log.Printf("Error creating kafka topic %s: %v", topic, err) 145 | } 146 | 147 | // When this example is run the first time, wait for creation of all internal topics (this is done 148 | // by goka.NewProcessor) 149 | initialized := make(chan struct{}) 150 | 151 | go runEmitter() 152 | go runProcessor(initialized) 153 | runView(initialized) 154 | } 155 | -------------------------------------------------------------------------------- /examples/3-messaging/blocker/blocker.go: -------------------------------------------------------------------------------- 1 | package blocker 2 | 3 | import ( 4 | "context" 5 | "encoding/json" 6 | "github.com/lovoo/goka" 7 | "github.com/lovoo/goka/examples/3-messaging/topicinit" 8 | ) 9 | 10 | var ( 11 | group goka.Group = "blocker" 12 | Table goka.Table = goka.GroupTable(group) 13 | Stream goka.Stream = "block_user" 14 | ) 15 | 16 | type BlockEvent struct { 17 | Unblock bool 18 | } 19 | 20 | type BlockEventCodec struct{} 21 | 22 | func (c *BlockEventCodec) Encode(value interface{}) ([]byte, error) { 23 | return json.Marshal(value) 24 | } 25 | 26 | func (c *BlockEventCodec) Decode(data []byte) (interface{}, error) { 27 | var m BlockEvent 28 | return &m, json.Unmarshal(data, &m) 29 | } 30 | 31 | type BlockValue struct { 32 | Blocked bool 33 | } 34 | type BlockValueCodec struct{} 35 | 36 | func (c *BlockValueCodec) Encode(value interface{}) ([]byte, error) { 37 | return json.Marshal(value) 38 | } 39 | 40 | func (c *BlockValueCodec) Decode(data []byte) (interface{}, error) { 41 | var m BlockValue 42 | return &m, json.Unmarshal(data, &m) 43 | } 44 | 45 | func block(ctx goka.Context, msg interface{}) { 46 | var s *BlockValue 47 | if v := ctx.Value(); v == nil { 48 | s = new(BlockValue) 49 | } else { 50 | s = v.(*BlockValue) 51 | } 52 | 53 | if msg.(*BlockEvent).Unblock { 54 | s.Blocked = false 55 | } else { 56 | s.Blocked = true 57 | } 58 | ctx.SetValue(s) 59 | } 60 | 61 | func PrepareTopics(brokers []string) { 62 | topicinit.EnsureStreamExists(string(Stream), brokers) 63 | } 64 | 65 | func Run(ctx context.Context, brokers []string) func() error { 66 | return func() error { 67 | g := goka.DefineGroup(group, 68 | goka.Input(Stream, new(BlockEventCodec), block), 69 | goka.Persist(new(BlockValueCodec)), 70 | ) 71 | p, err := goka.NewProcessor(brokers, g) 72 | if err != nil { 73 | return err 74 | } 75 | 76 | return p.Run(ctx) 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /examples/3-messaging/cmd/block-user/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "fmt" 6 | "os" 7 | 8 | "github.com/lovoo/goka" 9 | "github.com/lovoo/goka/examples/3-messaging/blocker" 10 | ) 11 | 12 | var ( 13 | user = flag.String("user", "", "user to block") 14 | unblock = flag.Bool("unblock", false, "unblock user instead of blocking") 15 | broker = flag.String("broker", "localhost:9092", "boostrap Kafka broker") 16 | ) 17 | 18 | func main() { 19 | flag.Parse() 20 | if *user == "" { 21 | fmt.Println("cannot block user ''") 22 | os.Exit(1) 23 | } 24 | emitter, err := goka.NewEmitter([]string{*broker}, blocker.Stream, new(blocker.BlockEventCodec)) 25 | if err != nil { 26 | panic(err) 27 | } 28 | defer emitter.Finish() 29 | 30 | err = emitter.EmitSync(*user, &blocker.BlockEvent{Unblock: *unblock}) 31 | if err != nil { 32 | panic(err) 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /examples/3-messaging/cmd/loadgen/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "bytes" 5 | "encoding/json" 6 | "fmt" 7 | "log" 8 | "math/rand" 9 | "net/http" 10 | "time" 11 | 12 | "github.com/lovoo/goka/examples/3-messaging" 13 | ) 14 | 15 | const ( 16 | urlTmpl = "http://localhost:8080/%s/send" 17 | spamProb = 0.3 18 | ) 19 | 20 | var ( 21 | users = []string{ 22 | "Alice", 23 | "Bob", 24 | "Charlie", 25 | "Dave", 26 | "Eve", 27 | } 28 | 29 | contents = []string{ 30 | "Hi how are you doing", 31 | "Hello let's have lunch together", 32 | "Bye", 33 | } 34 | ) 35 | 36 | func send(from, to, content string) { 37 | m := messaging.Message{ 38 | To: to, 39 | Content: content, 40 | } 41 | 42 | b, err := json.Marshal(&m) 43 | if err != nil { 44 | log.Printf("error encoding message: %v", err) 45 | return 46 | } 47 | 48 | url := fmt.Sprintf(urlTmpl, from) 49 | 50 | req, err := http.NewRequest("POST", url, bytes.NewBuffer(b)) 51 | if err != nil { 52 | log.Printf("error creating request: %v", err) 53 | return 54 | } 55 | req.Header.Set("Content-Type", "application/json") 56 | 57 | client := &http.Client{} 58 | resp, err := client.Do(req) 59 | if err != nil { 60 | log.Printf("error sending request: %v", err) 61 | return 62 | } 63 | log.Printf("Posted message '%s' (%s -> %s)", content, from, to) 64 | defer resp.Body.Close() 65 | //TODO(diogo) check response status code 66 | } 67 | 68 | func main() { 69 | t := time.NewTicker(200 * time.Millisecond) 70 | defer t.Stop() 71 | 72 | for range t.C { 73 | var ( 74 | cnt = "spam!" 75 | from = "Bob" 76 | ) 77 | if rand.Float64() < 1-spamProb { 78 | from = users[rand.Intn(len(users))] 79 | cnt = contents[rand.Intn(len(contents))] 80 | } 81 | to := users[rand.Intn(len(users))] 82 | for to == from { 83 | to = users[rand.Intn(len(users))] 84 | } 85 | send(from, to, cnt) 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /examples/3-messaging/cmd/processor/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "context" 5 | "flag" 6 | "log" 7 | "os" 8 | "os/signal" 9 | "syscall" 10 | 11 | "github.com/lovoo/goka/examples/3-messaging/blocker" 12 | "github.com/lovoo/goka/examples/3-messaging/collector" 13 | "github.com/lovoo/goka/examples/3-messaging/detector" 14 | "github.com/lovoo/goka/examples/3-messaging/filter" 15 | "github.com/lovoo/goka/examples/3-messaging/translator" 16 | "golang.org/x/sync/errgroup" 17 | ) 18 | 19 | var ( 20 | brokers = []string{"localhost:9092"} 21 | runFilter = flag.Bool("filter", false, "run filter processor") 22 | runCollector = flag.Bool("collector", false, "run collector processor") 23 | runTranslator = flag.Bool("translator", false, "run translator processor") 24 | runBlocker = flag.Bool("blocker", false, "run blocker processor") 25 | runDetector = flag.Bool("detector", false, "run detector processor") 26 | broker = flag.String("broker", "localhost:9092", "boostrap Kafka broker") 27 | ) 28 | 29 | func main() { 30 | flag.Parse() 31 | ctx, cancel := context.WithCancel(context.Background()) 32 | grp, ctx := errgroup.WithContext(ctx) 33 | 34 | // Create topics if they do not already exist 35 | if *runCollector { 36 | collector.PrepareTopics(brokers) 37 | } 38 | if *runFilter { 39 | filter.PrepareTopics(brokers) 40 | } 41 | if *runBlocker { 42 | blocker.PrepareTopics(brokers) 43 | } 44 | if *runDetector { 45 | detector.PrepareTopics(brokers) 46 | } 47 | if *runTranslator { 48 | translator.PrepareTopics(brokers) 49 | } 50 | 51 | if *runCollector { 52 | log.Println("starting collector") 53 | grp.Go(collector.Run(ctx, brokers)) 54 | } 55 | if *runFilter { 56 | log.Println("starting filter") 57 | grp.Go(filter.Run(ctx, brokers)) 58 | } 59 | if *runBlocker { 60 | log.Println("starting blocker") 61 | grp.Go(blocker.Run(ctx, brokers)) 62 | } 63 | if *runDetector { 64 | log.Println("starting detector") 65 | grp.Go(detector.Run(ctx, brokers)) 66 | } 67 | if *runTranslator { 68 | log.Println("starting translator") 69 | grp.Go(translator.Run(ctx, brokers)) 70 | } 71 | 72 | // Wait for SIGINT/SIGTERM 73 | waiter := make(chan os.Signal, 1) 74 | signal.Notify(waiter, syscall.SIGINT, syscall.SIGTERM) 75 | select { 76 | case <-waiter: 77 | case <-ctx.Done(): 78 | } 79 | cancel() 80 | if err := grp.Wait(); err != nil { 81 | log.Println(err) 82 | } 83 | log.Println("done") 84 | } 85 | -------------------------------------------------------------------------------- /examples/3-messaging/cmd/service/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | 6 | "github.com/lovoo/goka/examples/3-messaging" 7 | "github.com/lovoo/goka/examples/3-messaging/service" 8 | ) 9 | 10 | var ( 11 | sent = flag.Bool("sent", false, "emit to SentStream") 12 | broker = flag.String("broker", "localhost:9092", "boostrap Kafka broker") 13 | ) 14 | 15 | func main() { 16 | flag.Parse() 17 | if *sent { 18 | service.Run([]string{*broker}, messaging.SentStream) 19 | } else { 20 | service.Run([]string{*broker}, messaging.ReceivedStream) 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /examples/3-messaging/cmd/translate-word/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "fmt" 6 | "os" 7 | 8 | "github.com/lovoo/goka" 9 | "github.com/lovoo/goka/codec" 10 | "github.com/lovoo/goka/examples/3-messaging/translator" 11 | ) 12 | 13 | var ( 14 | word = flag.String("word", "", "word to translate") 15 | with = flag.String("with", "", "word translation") 16 | broker = flag.String("broker", "localhost:9092", "boostrap Kafka broker") 17 | ) 18 | 19 | func main() { 20 | flag.Parse() 21 | if *word == "" { 22 | fmt.Println("cannot translate word ''") 23 | os.Exit(1) 24 | } 25 | emitter, err := goka.NewEmitter([]string{*broker}, translator.Stream, new(codec.String)) 26 | if err != nil { 27 | panic(err) 28 | } 29 | defer emitter.Finish() 30 | 31 | err = emitter.EmitSync(*word, *with) 32 | if err != nil { 33 | panic(err) 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /examples/3-messaging/collector/collector.go: -------------------------------------------------------------------------------- 1 | package collector 2 | 3 | import ( 4 | "context" 5 | "encoding/json" 6 | "github.com/lovoo/goka" 7 | "github.com/lovoo/goka/examples/3-messaging" 8 | "github.com/lovoo/goka/examples/3-messaging/topicinit" 9 | ) 10 | 11 | const maxMessages = 5 12 | 13 | var ( 14 | group goka.Group = "collector" 15 | Table goka.Table = goka.GroupTable(group) 16 | ) 17 | 18 | type MessageListCodec struct{} 19 | 20 | func (c *MessageListCodec) Encode(value interface{}) ([]byte, error) { 21 | return json.Marshal(value) 22 | } 23 | 24 | func (c *MessageListCodec) Decode(data []byte) (interface{}, error) { 25 | var m []messaging.Message 26 | err := json.Unmarshal(data, &m) 27 | return m, err 28 | } 29 | 30 | func collect(ctx goka.Context, msg interface{}) { 31 | var ml []messaging.Message 32 | if v := ctx.Value(); v != nil { 33 | ml = v.([]messaging.Message) 34 | } 35 | 36 | m := msg.(*messaging.Message) 37 | ml = append(ml, *m) 38 | 39 | if len(ml) > maxMessages { 40 | ml = ml[len(ml)-maxMessages:] 41 | } 42 | ctx.SetValue(ml) 43 | } 44 | 45 | func PrepareTopics(brokers []string) { 46 | topicinit.EnsureStreamExists(string(messaging.ReceivedStream), brokers) 47 | } 48 | 49 | func Run(ctx context.Context, brokers []string) func() error { 50 | return func() error { 51 | g := goka.DefineGroup(group, 52 | goka.Input(messaging.ReceivedStream, new(messaging.MessageCodec), collect), 53 | goka.Persist(new(MessageListCodec)), 54 | ) 55 | p, err := goka.NewProcessor(brokers, g) 56 | if err != nil { 57 | return err 58 | } 59 | return p.Run(ctx) 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /examples/3-messaging/detector/detector.go: -------------------------------------------------------------------------------- 1 | package detector 2 | 3 | import ( 4 | "context" 5 | "encoding/json" 6 | "github.com/lovoo/goka" 7 | "github.com/lovoo/goka/examples/3-messaging" 8 | "github.com/lovoo/goka/examples/3-messaging/blocker" 9 | "github.com/lovoo/goka/examples/3-messaging/topicinit" 10 | ) 11 | 12 | const ( 13 | minMessages = 200 14 | maxRate = 0.5 15 | ) 16 | 17 | var ( 18 | group goka.Group = "detector" 19 | ) 20 | 21 | type Counters struct { 22 | Sent int 23 | Received int 24 | } 25 | 26 | type CountersCodec struct{} 27 | 28 | func (c *CountersCodec) Encode(value interface{}) ([]byte, error) { 29 | return json.Marshal(value) 30 | } 31 | 32 | func (c *CountersCodec) Decode(data []byte) (interface{}, error) { 33 | var m Counters 34 | return &m, json.Unmarshal(data, &m) 35 | } 36 | 37 | func getValue(ctx goka.Context) *Counters { 38 | if v := ctx.Value(); v != nil { 39 | return v.(*Counters) 40 | } 41 | return &Counters{} 42 | } 43 | 44 | func detectSpammer(ctx goka.Context, c *Counters) bool { 45 | var ( 46 | total = float64(c.Sent + c.Received) 47 | rate = float64(c.Sent) / total 48 | ) 49 | return total >= minMessages && rate >= maxRate 50 | } 51 | 52 | func PrepareTopics(brokers []string) { 53 | topicinit.EnsureStreamExists(string(messaging.SentStream), brokers) 54 | } 55 | 56 | func Run(ctx context.Context, brokers []string) func() error { 57 | return func() error { 58 | g := goka.DefineGroup(group, 59 | goka.Input(messaging.SentStream, new(messaging.MessageCodec), func(ctx goka.Context, msg interface{}) { 60 | c := getValue(ctx) 61 | c.Sent++ 62 | ctx.SetValue(c) 63 | 64 | // check if sender is a spammer 65 | if detectSpammer(ctx, c) { 66 | ctx.Emit(blocker.Stream, ctx.Key(), new(blocker.BlockEvent)) 67 | } 68 | 69 | // Loop to receiver 70 | m := msg.(*messaging.Message) 71 | ctx.Loopback(m.To, m) 72 | }), 73 | goka.Loop(new(messaging.MessageCodec), func(ctx goka.Context, msg interface{}) { 74 | c := getValue(ctx) 75 | c.Received++ 76 | ctx.SetValue(c) 77 | 78 | // check if receiver is a spammer 79 | if detectSpammer(ctx, c) { 80 | ctx.Emit(blocker.Stream, ctx.Key(), new(blocker.BlockEvent)) 81 | } 82 | }), 83 | goka.Output(blocker.Stream, new(blocker.BlockEventCodec)), 84 | goka.Persist(new(CountersCodec)), 85 | ) 86 | p, err := goka.NewProcessor(brokers, g) 87 | if err != nil { 88 | return err 89 | } 90 | 91 | return p.Run(ctx) 92 | } 93 | } 94 | -------------------------------------------------------------------------------- /examples/3-messaging/figs/goka-arch-blocker.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/lovoo/goka/e3997d4ef01a22bb53b6d3728350b2e70ea44aba/examples/3-messaging/figs/goka-arch-blocker.png -------------------------------------------------------------------------------- /examples/3-messaging/figs/goka-arch-simple.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/lovoo/goka/e3997d4ef01a22bb53b6d3728350b2e70ea44aba/examples/3-messaging/figs/goka-arch-simple.png -------------------------------------------------------------------------------- /examples/3-messaging/filter/filter.go: -------------------------------------------------------------------------------- 1 | package filter 2 | 3 | import ( 4 | "context" 5 | "github.com/lovoo/goka" 6 | messaging "github.com/lovoo/goka/examples/3-messaging" 7 | "github.com/lovoo/goka/examples/3-messaging/blocker" 8 | "github.com/lovoo/goka/examples/3-messaging/topicinit" 9 | "github.com/lovoo/goka/examples/3-messaging/translator" 10 | "strings" 11 | ) 12 | 13 | var ( 14 | group goka.Group = "message_filter" 15 | ) 16 | 17 | func shouldDrop(ctx goka.Context) bool { 18 | v := ctx.Join(blocker.Table) 19 | return v != nil && v.(*blocker.BlockValue).Blocked 20 | } 21 | 22 | func filter(ctx goka.Context, msg interface{}) { 23 | if shouldDrop(ctx) { 24 | return 25 | } 26 | m := translate(ctx, msg.(*messaging.Message)) 27 | ctx.Emit(messaging.ReceivedStream, m.To, m) 28 | } 29 | 30 | func translate(ctx goka.Context, m *messaging.Message) *messaging.Message { 31 | words := strings.Split(m.Content, " ") 32 | for i, w := range words { 33 | if tw := ctx.Lookup(translator.Table, w); tw != nil { 34 | words[i] = tw.(string) 35 | } 36 | } 37 | return &messaging.Message{ 38 | From: m.From, 39 | To: m.To, 40 | Content: strings.Join(words, " "), 41 | } 42 | } 43 | 44 | func PrepareTopics(brokers []string) { 45 | topicinit.EnsureStreamExists(string(messaging.SentStream), brokers) 46 | 47 | // We refer to these tables, ensure that they exist initially also in the 48 | // case that the translator or blocker processors are not started 49 | for _, topicName := range []string{string(translator.Table), string(blocker.Table)} { 50 | topicinit.EnsureTableExists(topicName, brokers) 51 | } 52 | } 53 | 54 | func Run(ctx context.Context, brokers []string) func() error { 55 | return func() error { 56 | g := goka.DefineGroup(group, 57 | goka.Input(messaging.SentStream, new(messaging.MessageCodec), filter), 58 | goka.Output(messaging.ReceivedStream, new(messaging.MessageCodec)), 59 | goka.Join(blocker.Table, new(blocker.BlockValueCodec)), 60 | goka.Lookup(translator.Table, new(translator.ValueCodec)), 61 | ) 62 | p, err := goka.NewProcessor(brokers, g) 63 | if err != nil { 64 | return err 65 | } 66 | 67 | return p.Run(ctx) 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /examples/3-messaging/message.go: -------------------------------------------------------------------------------- 1 | package messaging 2 | 3 | import ( 4 | "encoding/json" 5 | 6 | "github.com/lovoo/goka" 7 | ) 8 | 9 | var ( 10 | SentStream goka.Stream = "message_sent" 11 | ReceivedStream goka.Stream = "message_received" 12 | ) 13 | 14 | type Message struct { 15 | From string 16 | To string 17 | Content string 18 | } 19 | 20 | type MessageCodec struct{} 21 | 22 | func (c *MessageCodec) Encode(value interface{}) ([]byte, error) { 23 | return json.Marshal(value) 24 | } 25 | 26 | func (c *MessageCodec) Decode(data []byte) (interface{}, error) { 27 | var m Message 28 | return &m, json.Unmarshal(data, &m) 29 | } 30 | 31 | type MessageListCodec struct{} 32 | 33 | func (c *MessageListCodec) Encode(value interface{}) ([]byte, error) { 34 | return json.Marshal(value) 35 | } 36 | 37 | func (c *MessageListCodec) Decode(data []byte) (interface{}, error) { 38 | var m []Message 39 | err := json.Unmarshal(data, &m) 40 | return m, err 41 | } 42 | -------------------------------------------------------------------------------- /examples/3-messaging/service/service.go: -------------------------------------------------------------------------------- 1 | package service 2 | 3 | import ( 4 | "context" 5 | "encoding/json" 6 | "fmt" 7 | "io" 8 | "log" 9 | "net/http" 10 | 11 | "github.com/gorilla/mux" 12 | "github.com/lovoo/goka" 13 | "github.com/lovoo/goka/examples/3-messaging" 14 | "github.com/lovoo/goka/examples/3-messaging/collector" 15 | ) 16 | 17 | func Run(brokers []string, stream goka.Stream) { 18 | view, err := goka.NewView(brokers, collector.Table, new(collector.MessageListCodec)) 19 | if err != nil { 20 | panic(err) 21 | } 22 | go view.Run(context.Background()) 23 | 24 | emitter, err := goka.NewEmitter(brokers, stream, new(messaging.MessageCodec)) 25 | if err != nil { 26 | panic(err) 27 | } 28 | defer emitter.Finish() 29 | 30 | router := mux.NewRouter() 31 | router.HandleFunc("/{user}/send", send(emitter, stream)).Methods("POST") 32 | router.HandleFunc("/{user}/feed", feed(view)).Methods("GET") 33 | 34 | log.Printf("Listen port 8080") 35 | log.Fatal(http.ListenAndServe(":8080", router)) 36 | } 37 | 38 | func send(emitter *goka.Emitter, stream goka.Stream) func(w http.ResponseWriter, r *http.Request) { 39 | return func(w http.ResponseWriter, r *http.Request) { 40 | var m messaging.Message 41 | 42 | b, err := io.ReadAll(r.Body) 43 | if err != nil { 44 | fmt.Fprintf(w, "error: %v", err) 45 | return 46 | } 47 | 48 | err = json.Unmarshal(b, &m) 49 | if err != nil { 50 | fmt.Fprintf(w, "error: %v", err) 51 | return 52 | } 53 | 54 | m.From = mux.Vars(r)["user"] 55 | 56 | if stream == messaging.ReceivedStream { 57 | err = emitter.EmitSync(m.To, &m) 58 | } else { 59 | err = emitter.EmitSync(m.From, &m) 60 | } 61 | if err != nil { 62 | fmt.Fprintf(w, "error: %v", err) 63 | return 64 | } 65 | log.Printf("Sent message:\n %v\n", m) 66 | fmt.Fprintf(w, "Sent message:\n %v\n", m) 67 | } 68 | } 69 | 70 | func feed(view *goka.View) func(w http.ResponseWriter, r *http.Request) { 71 | return func(w http.ResponseWriter, r *http.Request) { 72 | user := mux.Vars(r)["user"] 73 | val, _ := view.Get(user) 74 | if val == nil { 75 | fmt.Fprintf(w, "%s not found!", user) 76 | return 77 | } 78 | messages := val.([]messaging.Message) 79 | fmt.Fprintf(w, "Latest messages for %s\n", user) 80 | for i, m := range messages { 81 | fmt.Fprintf(w, "%d %10s: %v\n", i, m.From, m.Content) 82 | } 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /examples/3-messaging/topicinit/topicinit.go: -------------------------------------------------------------------------------- 1 | package topicinit 2 | 3 | import ( 4 | "github.com/lovoo/goka" 5 | "log" 6 | ) 7 | 8 | // EnsureStreamExists is a convenience wrapper for TopicManager.EnsureStreamExists 9 | func EnsureStreamExists(topic string, brokers []string) { 10 | tm := createTopicManager(brokers) 11 | defer tm.Close() 12 | err := tm.EnsureStreamExists(topic, 8) 13 | if err != nil { 14 | log.Printf("Error creating kafka topic %s: %v", topic, err) 15 | } 16 | } 17 | 18 | // EnsureTableExists is a convenience wrapper for TopicManager.EnsureTableExists 19 | func EnsureTableExists(topic string, brokers []string) { 20 | tm := createTopicManager(brokers) 21 | defer tm.Close() 22 | err := tm.EnsureTableExists(topic, 8) 23 | if err != nil { 24 | log.Printf("Error creating kafka topic %s: %v", topic, err) 25 | } 26 | } 27 | 28 | func createTopicManager(brokers []string) goka.TopicManager { 29 | tmc := goka.NewTopicManagerConfig() 30 | tm, err := goka.NewTopicManager(brokers, goka.DefaultConfig(), tmc) 31 | if err != nil { 32 | log.Fatalf("Error creating topic manager: %v", err) 33 | } 34 | return tm 35 | } 36 | -------------------------------------------------------------------------------- /examples/3-messaging/translator/translator.go: -------------------------------------------------------------------------------- 1 | package translator 2 | 3 | import ( 4 | "context" 5 | "github.com/lovoo/goka" 6 | "github.com/lovoo/goka/codec" 7 | "github.com/lovoo/goka/examples/3-messaging/topicinit" 8 | ) 9 | 10 | var ( 11 | group goka.Group = "translator" 12 | Table goka.Table = goka.GroupTable(group) 13 | Stream goka.Stream = "translate-word" 14 | ) 15 | 16 | type ValueCodec struct { 17 | codec.String 18 | } 19 | 20 | func translate(ctx goka.Context, msg interface{}) { 21 | ctx.SetValue(msg.(string)) 22 | } 23 | 24 | func PrepareTopics(brokers []string) { 25 | topicinit.EnsureStreamExists(string(Stream), brokers) 26 | } 27 | 28 | func Run(ctx context.Context, brokers []string) func() error { 29 | return func() error { 30 | g := goka.DefineGroup(group, 31 | goka.Input(Stream, new(ValueCodec), translate), 32 | goka.Persist(new(ValueCodec)), 33 | ) 34 | p, err := goka.NewProcessor(brokers, g) 35 | if err != nil { 36 | return err 37 | } 38 | 39 | return p.Run(ctx) 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /examples/4-tests/README.md: -------------------------------------------------------------------------------- 1 | # Test Example 2 | 3 | This Example demonstrates the testing capabilities of the tester package. 4 | 5 | Check the [source file](https://github.com/lovoo/goka/blob/master/examples/4-tests/example_test.go) to get the examples. 6 | -------------------------------------------------------------------------------- /examples/6-reconnecting-view/README.md: -------------------------------------------------------------------------------- 1 | # Reconnecting View 2 | 3 | This example shows a reconnecting view by observing the state changes. 4 | Run a local Kafka cluster by calling `make start` in folder `examples/`. 5 | 6 | Then run this example (`go run 6-reconnecting-views/main.go`). 7 | You should see the view state changes upon starting. 8 | 9 | Now kill the kafka cluster `make stop`, you should see some error messages and the view 10 | trying to reconnect using a default backoff 11 | -------------------------------------------------------------------------------- /examples/6-reconnecting-view/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "context" 5 | "log" 6 | "os" 7 | "os/signal" 8 | "syscall" 9 | 10 | "github.com/lovoo/goka" 11 | "github.com/lovoo/goka/codec" 12 | ) 13 | 14 | func main() { 15 | var brokers = []string{"127.0.0.1:9092"} 16 | var topic goka.Table = "restartable-view-test-table" 17 | 18 | tmc := goka.NewTopicManagerConfig() 19 | tm, err := goka.NewTopicManager(brokers, goka.DefaultConfig(), tmc) 20 | if err != nil { 21 | log.Fatalf("Error creating topic manager: %v", err) 22 | } 23 | defer tm.Close() 24 | err = tm.EnsureStreamExists(string(topic), 8) 25 | if err != nil { 26 | log.Printf("Error creating kafka topic %s: %v", topic, err) 27 | } 28 | 29 | view, err := goka.NewView( 30 | // connect to example kafka cluster 31 | []string{"localhost:9092"}, 32 | // name does not matter, table will be empty 33 | topic, 34 | // codec doesn't matter, the table will be empty 35 | new(codec.String), 36 | // start the view autoconnecting 37 | goka.WithViewAutoReconnect(), 38 | ) 39 | if err != nil { 40 | log.Fatalf("Cannot create view: %v", err) 41 | } 42 | // context we'll use to run the view and the state change observer 43 | ctx, cancel := context.WithCancel(context.Background()) 44 | 45 | // channel used to wait for the view to finish 46 | done := make(chan struct{}) 47 | go func() { 48 | defer close(done) 49 | err := view.Run(ctx) 50 | if err != nil { 51 | log.Printf("View finished with error: %v", err) 52 | } 53 | }() 54 | 55 | // Get a state change observer and 56 | go func() { 57 | obs := view.ObserveStateChanges() 58 | defer obs.Stop() 59 | for { 60 | select { 61 | case state, ok := <-obs.C(): 62 | if !ok { 63 | return 64 | } 65 | log.Printf("View is in state: %v", goka.ViewState(state)) 66 | case <-ctx.Done(): 67 | return 68 | } 69 | } 70 | }() 71 | 72 | go func() { 73 | waiter := make(chan os.Signal, 1) 74 | signal.Notify(waiter, syscall.SIGINT, syscall.SIGTERM) 75 | <-waiter 76 | cancel() 77 | }() 78 | 79 | <-done 80 | } 81 | -------------------------------------------------------------------------------- /examples/7-redis/README.md: -------------------------------------------------------------------------------- 1 | # Redis storage example 2 | 3 | Using [Redis](https://redis.io/) as option of storage. 4 | 5 | This example has an **Emitter** and one **Processor**. The emitter generates 6 | events with random keys (_user_id_) consumed by Kafka. The processor consumes 7 | this events and uses redis as storage/cache. 8 | 9 | 10 | ## Usage 11 | 12 | Run a local Kafka cluster and redis server by calling `make start` in folder `examples/`. 13 | This will start a kafka cluster accessible under `127.0.0.1:9092` and a redis service under 14 | `127.0.0.1:6379` if you want to use a different configuration you need to adjust the config 15 | file `config.yaml`. 16 | 17 | The config file also has some more options to set: 18 | 19 | ```yaml 20 | kafka: 21 | brokers: 22 | - "127.0.0.1:9092" 23 | group: "examples" 24 | stream: "events" 25 | redis: "127.0.0.1:6379" 26 | namespace: "producer" 27 | ``` 28 | 29 | Where: 30 | * **brokers** : slice of kafka brokers hosts. 31 | * **group** : group name of this example belongs. 32 | * **stream**: stream name of this example belongs. 33 | * **redis**: address of redis (`localhost:6379`). 34 | * **namespace**: namespace distinguish applications that write to the same keys on Redis. 35 | 36 | After starting the machines and adjusting the config run the example with 37 | `go run 7-redis/*.go -config 7-redis/config.yaml`. 38 | 39 | The events are produced and consumed by Kafka with random keys. It's possible 40 | run several of the same binary and check the behaviour of kafka 41 | rebalancing and removing partitions. 42 | 43 | After you are finished with the example you may stop the Kafka cluster and redis server by 44 | entering `make stop` in the `examples/` folder. 45 | -------------------------------------------------------------------------------- /examples/7-redis/codec.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import "encoding/json" 4 | 5 | type Codec struct{} 6 | 7 | // Encode encodes a event struct into an array. 8 | func (c *Codec) Encode(value interface{}) ([]byte, error) { 9 | return json.Marshal(value) 10 | } 11 | 12 | // Decode decodes a event from byte encoded array. 13 | func (c *Codec) Decode(data []byte) (interface{}, error) { 14 | event := new(Event) 15 | 16 | err := json.Unmarshal(data, event) 17 | return event, err 18 | } 19 | -------------------------------------------------------------------------------- /examples/7-redis/config.yaml: -------------------------------------------------------------------------------- 1 | 2 | kafka: 3 | brokers: 4 | - "127.0.0.1:9092" 5 | group: "examples" 6 | stream: "events" 7 | redis: "127.0.0.1:6379" 8 | namespace: "producer" 9 | -------------------------------------------------------------------------------- /examples/7-redis/consumer.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "context" 5 | "log" 6 | 7 | "github.com/lovoo/goka" 8 | storage "github.com/lovoo/goka/storage/redis" 9 | 10 | redis "gopkg.in/redis.v5" 11 | ) 12 | 13 | // Publisher defines an interface to Publish the event somewhere. 14 | type Publisher interface { 15 | Publish(ctx context.Context, key string, event *Event) error 16 | Close() error 17 | } 18 | 19 | // Consume starts goka events consumer. 20 | func Consume(pub Publisher, brokers []string, group string, stream string, store string, namespace string) error { 21 | codec := new(Codec) 22 | 23 | tmc := goka.NewTopicManagerConfig() 24 | tm, err := goka.NewTopicManager(brokers, goka.DefaultConfig(), tmc) 25 | if err != nil { 26 | log.Fatalf("Error creating topic manager: %v", err) 27 | } 28 | defer tm.Close() 29 | err = tm.EnsureStreamExists(stream, 8) 30 | if err != nil { 31 | log.Printf("Error creating kafka topic %s: %v", stream, err) 32 | } 33 | 34 | input := goka.Input(goka.Stream(stream), codec, func(ctx goka.Context, msg interface{}) { 35 | event, ok := msg.(*Event) 36 | if ok { 37 | pub.Publish(context.Background(), ctx.Key(), event) 38 | } 39 | }) 40 | graph := goka.DefineGroup(goka.Group(group), input, goka.Persist(codec)) 41 | 42 | opts := []goka.ProcessorOption{} 43 | switch { 44 | case store != "": 45 | client := redis.NewClient(&redis.Options{ 46 | Addr: store, 47 | }) 48 | opts = append(opts, goka.WithStorageBuilder(storage.RedisBuilder(client, namespace))) 49 | defer client.Close() 50 | } 51 | processor, err := goka.NewProcessor(brokers, graph, opts...) 52 | if err != nil { 53 | return err 54 | } 55 | 56 | return processor.Run(context.Background()) 57 | } 58 | -------------------------------------------------------------------------------- /examples/7-redis/event.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | type Event struct { 4 | UserID string `json:"user_id"` 5 | Timestamp int64 `json:"timestamp"` 6 | } 7 | -------------------------------------------------------------------------------- /examples/7-redis/kafka.yml: -------------------------------------------------------------------------------- 1 | version: '2' 2 | services: 3 | 4 | # zookeeper ========================== 5 | zookeeper: 6 | container_name: zookeeper 7 | image: confluent/zookeeper 8 | ports: 9 | - "2181:2181" 10 | 11 | # kafka ============================== 12 | kafka: 13 | container_name: kafka 14 | image: wurstmeister/kafka 15 | volumes: 16 | - /var/run/docker.sock:/var/run/docker.sock 17 | depends_on: 18 | - zookeeper 19 | - redis 20 | ports: 21 | - "9092:9092" 22 | - "9997:9997" 23 | environment: 24 | KAFKA_PORT: 9092 25 | KAFKA_ADVERTISED_PORT: 9092 26 | KAFKA_ADVERTISED_HOST_NAME: kafka 27 | KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 28 | KAFKA_NUM_PARTITIONS: 10 29 | KAFKA_DEFAULT_REPLICATION_FACTOR: 1 30 | KAFKA_REPLICATION_FACTOR: 1 31 | 32 | # redis ======================================= 33 | redis: 34 | container_name: redis 35 | image: redis:latest 36 | ports: 37 | - "6379:6379" 38 | expose: 39 | - "6379" 40 | -------------------------------------------------------------------------------- /examples/7-redis/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "context" 5 | "flag" 6 | "fmt" 7 | "log" 8 | "math/rand" 9 | "os" 10 | "strconv" 11 | "time" 12 | 13 | yaml "gopkg.in/yaml.v2" 14 | ) 15 | 16 | type Config struct { 17 | Kafka struct { 18 | Brokers []string `yaml:"brokers"` 19 | Group string `yaml:"group"` 20 | Stream string `yaml:"stream"` 21 | Redis string `yaml:"redis"` 22 | Namespace string `yaml:"namespace"` 23 | } `yaml:"kafka"` 24 | } 25 | 26 | var ( 27 | filename = flag.String("config", "config.yaml", "path to config file") 28 | ) 29 | 30 | func main() { 31 | flag.Parse() 32 | 33 | conf, err := readConfig(*filename) 34 | if err != nil { 35 | log.Fatal(err) 36 | } 37 | 38 | // consuming 39 | go func() { 40 | err := Consume(new(nopPublisher), conf.Kafka.Brokers, conf.Kafka.Group, 41 | conf.Kafka.Stream, conf.Kafka.Redis, conf.Kafka.Namespace) 42 | if err != nil { 43 | log.Fatal(err) 44 | } 45 | }() 46 | 47 | // producing 48 | producer, err := NewProducer(conf.Kafka.Brokers, conf.Kafka.Stream) 49 | for { 50 | event := &Event{ 51 | UserID: strconv.FormatInt(rand.Int63n(255), 10), 52 | Timestamp: time.Now().UnixNano(), 53 | } 54 | fmt.Printf("emit ->key: `%v` ->event: `%v`\n", event.UserID, event) 55 | err = producer.Emit(event.UserID, event) 56 | if err != nil { 57 | log.Fatal(err) 58 | } 59 | time.Sleep(5 * time.Second) 60 | } 61 | } 62 | 63 | func readConfig(filename string) (*Config, error) { 64 | b, err := os.ReadFile(filename) 65 | if err != nil { 66 | return nil, err 67 | } 68 | conf := new(Config) 69 | err = yaml.Unmarshal(b, conf) 70 | return conf, err 71 | } 72 | 73 | type nopPublisher struct{} 74 | 75 | func (p *nopPublisher) Publish(ctx context.Context, key string, event *Event) error { 76 | fmt.Printf("published ->key: `%v` ->event: `%v`\n", key, event) 77 | return nil 78 | } 79 | 80 | func (p *nopPublisher) Close() error { 81 | return nil 82 | } 83 | -------------------------------------------------------------------------------- /examples/7-redis/producer.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import "github.com/lovoo/goka" 4 | 5 | // Producer defines an interface whose events are produced on kafka. 6 | type Producer interface { 7 | Emit(key string, event *Event) error 8 | Close() error 9 | } 10 | 11 | type kafkaProducer struct { 12 | emitter *goka.Emitter 13 | } 14 | 15 | // NewProducer returns a new kafka producer. 16 | func NewProducer(brokers []string, stream string) (Producer, error) { 17 | codec := new(Codec) 18 | emitter, err := goka.NewEmitter(brokers, goka.Stream(stream), codec) 19 | if err != nil { 20 | return nil, err 21 | } 22 | return &kafkaProducer{emitter}, nil 23 | } 24 | 25 | func (p *kafkaProducer) Emit(key string, event *Event) error { 26 | return p.emitter.EmitSync(key, event) 27 | } 28 | 29 | func (p *kafkaProducer) Close() error { 30 | p.emitter.Finish() 31 | return nil 32 | } 33 | -------------------------------------------------------------------------------- /examples/8-monitoring/README.md: -------------------------------------------------------------------------------- 1 | # Goka web interface 2 | 3 | Observing the progress of recovery and execution of a goka processor is often vital for basic 4 | debugging purposes or performance analyses. 5 | 6 | Goka provides a very simple web interface that allows to show statistics about individual partitions including 7 | state, offsets and lag as well as input- and output rate. 8 | 9 | For low level inspection of table data, it provides a simple interface to query values from 10 | views or processor tables. 11 | 12 | ## Monitoring 13 | 14 | First, to get a web interface we need a router. If the program using Goka already uses a router, Goka's web 15 | interface can easily be attached to it instead. 16 | 17 | So first, initialize the monitoring server using a router with 18 | 19 | ```go 20 | root := mux.NewRouter() 21 | monitorServer := monitor.NewServer("/monitor", root) 22 | ``` 23 | 24 | Suppose we have a processor *proc*, then we can simply attach it to the monitor 25 | ```go 26 | proc := goka.NewProcessor(...) 27 | monitorServer.AttachProcessor(proc) 28 | ``` 29 | 30 | Ok, then let's start the processor and the server with 31 | ```go 32 | go proc.Run(context.Background()) 33 | http.ListenAndServe(":9095", root) 34 | ``` 35 | 36 | Opening the browser on [localhost:9095/monitor/](http://localhost:9095/monitor/) will then show us a list of all attached processors 37 | and views and by selecting one of them we'll get statistics of the processor, similar like this 38 | 39 | ![Processor View](images/processor-view.png?raw=true "processor view") 40 | 41 | ## Index 42 | 43 | Ok nice, but if you don't have another router with an index and links to the monitor, 44 | it's really annoying to add *monitor/* to your browser url every time you want to open the monitor. 45 | That's why Goka provides a small convenience index server, that the Monitoring and the Query servers can be attached to. 46 | 47 | ```go 48 | idxServer := index.NewServer("/", root) 49 | idxServer.AddComponent(monitorServer, "Monitor") 50 | ``` 51 | 52 | So next time we open the page at [localhost:9095](http://localhost:9095), we'll get some nice links to the monitor. 53 | 54 | ![Index](images/index.png?raw=true "index view") 55 | 56 | 57 | ## Query 58 | As mentioned earlier, we can also add a query server that allows us to request a value by key from 59 | a View or a Processor table, simply by attaching it to a Query server. Also let's add the new server to the 60 | index as well to avoid repeated typing of *query/* in the url. 61 | 62 | ```go 63 | queryServer := query.NewServer("/query", root) 64 | idxServer.AddComponent(queryServer, "Query") 65 | queryServer.AttachSource("user-clicks", p.Get) 66 | ``` 67 | 68 | Opening a query page gives us a page like this: 69 | 70 | ![Query](images/query.png?raw=true "query view") 71 | 72 | Voila! 73 | 74 | ## Example 75 | 76 | The example in [main.go](https://github.com/lovoo/goka/blob/master/examples/8-monitoring/main.go) demonstrates the concepts and typical applications of the web interface by 77 | creating an Emitter, multiple Processors and a web interface. 78 | 79 | Be sure to have Apache Kafka and Zookeeper running by starting it in the examples-folder. 80 | 81 | ```console 82 | examples$ make start 83 | ``` 84 | 85 | Then run the monitoring example with: 86 | 87 | ```console 88 | examples$ go run 8-monitoring/main.go 89 | ``` 90 | -------------------------------------------------------------------------------- /examples/8-monitoring/images/index.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/lovoo/goka/e3997d4ef01a22bb53b6d3728350b2e70ea44aba/examples/8-monitoring/images/index.png -------------------------------------------------------------------------------- /examples/8-monitoring/images/processor-view.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/lovoo/goka/e3997d4ef01a22bb53b6d3728350b2e70ea44aba/examples/8-monitoring/images/processor-view.png -------------------------------------------------------------------------------- /examples/8-monitoring/images/query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/lovoo/goka/e3997d4ef01a22bb53b6d3728350b2e70ea44aba/examples/8-monitoring/images/query.png -------------------------------------------------------------------------------- /examples/9-defer-commit/README.md: -------------------------------------------------------------------------------- 1 | # Defer Commit 2 | 3 | This example demonstrates the context function `DeferCommit`. 4 | 5 | Usually the goka-context takes care of committing the incoming message offset once the callback has returned *and* all the started side-operations like `SetValue`, `Emit` or `Loopback` have returned to make sure that the message is not reconsumed, except in case of failure. 6 | 7 | However, there are rare use cases which require more control over the comitting behavior of the context, especially when using external services that implement asynchronous/callback interfaces. 8 | 9 | The following example simulates to forward messages into a different kafka cluster (so `ctx.Emit` won't work, but an extra `goka.Emitter` will do). 10 | 11 | Without `DeferCommit` we would have two options: (a) use `emitter.EmitSync` and wait for the result. This however disables batching - so it comes with performance penalty. (b) use `emitter.Emit` which fails when the returned `Promise` fails asynchronously. This however means that failing to emit a message results in data loss as the message will not be reconsumed. 12 | 13 | `DeferCommit` returns a function `func(error)` that passes the responsibility to commit to the caller. Failing to call that function will cause the messages to be reconsumed eventually and most likely to block the processor, as the involved input messages never seem to be processed and the Processor waits indefinitely. 14 | **If `DeferCommit` is called multiple times, all returned functions need to be called individually.** 15 | 16 | It should be used like this: 17 | ```go 18 | func callback(ctx goka.Context, msg interface{}){ 19 | // ... 20 | 21 | commit := ctx.DeferCommit() 22 | 23 | // in some async-code 24 | go func(){ 25 | 26 | if /* no error */{ 27 | // success 28 | commit(nil) 29 | }else{ 30 | // call commit with error. 31 | // This will actually not commit the message but shutdown the processor. 32 | commit(errors.New("some error")) 33 | } 34 | }() 35 | } 36 | ``` 37 | 38 | The example uses one emitter that emits the current unix-timestamp into topic `input-topic`. 39 | The forwarding processor instantiates a new emitter that represents the asynchronous component and emits the message 40 | to a `forward-topic` using `DeferCommit`. The third processor just prints the received messages. 41 | 42 | Running the example prints the message every second: 43 | ```bash 44 | go run main.go 45 | 46 | # output: 47 | # 2021/02/17 11:18:50 received message time: 1613557129 48 | # 2021/02/17 11:18:51 received message time: 1613557130 49 | # 2021/02/17 11:18:52 received message time: 1613557131 50 | # 2021/02/17 11:18:53 received message time: 1613557132 51 | ``` 52 | Stopping and restarting the example continues to print the incrementing timestamp. 53 | 54 | Now run the example with disabled commit. Note that stopping with `ctrl-c` does not work as the processor hangs at the uncommitted messages. Doing `ctrl-\` does the trick and prints the stack trace. 55 | ```bash 56 | go run main.go --no-commit 57 | 58 | # output: 59 | # 2021/02/17 11:20:28 received message time: 1613557228 60 | # 2021/02/17 11:20:29 received message time: 1613557229 61 | # 2021/02/17 11:20:30 received message time: 1613557230 62 | # ^C 63 | # 64 | ``` 65 | 66 | Running it again, shows that the messages are reconsumed as the offset was not commited 67 | ```bash 68 | go run main.go --no-commit 69 | 70 | # output: 71 | # --> old messages are reconsumed 72 | # 2021/02/17 11:21:48 received message time: 1613557228 73 | # 2021/02/17 11:21:48 received message time: 1613557229 74 | # 2021/02/17 11:21:48 received message time: 1613557230 75 | ## --> new messages start here 76 | # 2021/02/17 11:21:48 received message time: 1613557306 77 | # 2021/02/17 11:21:48 received message time: 1613557307 78 | # 2021/02/17 11:21:48 received message time: 1613557308 79 | ``` 80 | 81 | -------------------------------------------------------------------------------- /examples/Makefile: -------------------------------------------------------------------------------- 1 | all: 2 | @echo targets: start restart stop 3 | 4 | # stop and start docker compose (zookeeper and kafka) 5 | restart: stop start 6 | @echo done 7 | 8 | # start docker compose (zookeeper and kafka) and wait until all are up 9 | start: compose-up 10 | @echo done 11 | @sh wait.sh "localhost:2181,localhost:9092,localhost:9093,localhost:9094,localhost:6379" 12 | @sh wait-kafka.sh "kafka1:9092,kafka2:9093,kafka3:9094" 13 | 14 | # shutdown docker compose (zookeeper and kafka) 15 | stop: compose-down 16 | @echo done 17 | 18 | compose-up: 19 | @echo start docker compose 20 | @docker-compose up -d 21 | 22 | compose-down: 23 | @echo stop docker compose 24 | @docker-compose down 25 | -------------------------------------------------------------------------------- /examples/README.md: -------------------------------------------------------------------------------- 1 | # Goka Examples 2 | 3 | The examples in this directory will demonstrate different patterns and features 4 | of goka. 5 | 6 | ## Setup 7 | All examples are runnable out of the box. All you need to do is start 8 | Kafka and Zookeeper locally. Just run 9 | 10 | ```shell 11 | make start 12 | # or 13 | make restart # if it was started already and you need fresh instances. 14 | ``` 15 | 16 | This will start the docker containers and configure kafka to auto-create the topics. 17 | 18 | ### Simple Example 19 | This is a very simple toy application that demonstrates how to 20 | * use an Emitter 21 | * use a Processor 22 | * run a View to print the group table's values 23 | 24 | [Example](https://github.com/lovoo/goka/tree/master/examples/1-simplest/) 25 | 26 | ### Clicks 27 | Similar to the first example, Emitter, Processor and View are demonstrated here. 28 | In addition, it shows how to 29 | 30 | * access the View using a web endpoint 31 | * use a Codecs 32 | 33 | [Example](https://github.com/lovoo/goka/tree/master/examples/2-clicks/) 34 | 35 | ### More complex examples 36 | The following examples show the combination of multiple processors, views, etc. 37 | 38 | [Messaging Example](https://github.com/lovoo/goka/tree/master/examples/3-messaging/) 39 | 40 | By generating a random folder for storage, this example can be executed in parallel multiple times to get a feeling for the rebalancing that's happening under the hood. 41 | 42 | [Example](https://github.com/lovoo/goka/tree/master/examples/5-multiple/) 43 | 44 | 45 | ### Monitoring 46 | Shows how to use the monitoring- and query-interface of goka. 47 | 48 | [Example](https://github.com/lovoo/goka/tree/master/examples/8-monitoring) 49 | 50 | 51 | ### Testing 52 | Shows how to (unit-)test programs using goka. 53 | 54 | [Example](https://github.com/lovoo/goka/tree/master/examples/4-tests) 55 | 56 | ### DeferCommit 57 | 58 | Demonstrates the rather new context function to defer (postpone) the commit of a callback. 59 | 60 | [Example](https://github.com/lovoo/goka/tree/master/examples/9-defer-commit) 61 | -------------------------------------------------------------------------------- /examples/create-kafka-commands.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | ## lists the scripts inside the Kafka container and creates local scripts to call them with docker. 4 | 5 | set -e 6 | 7 | # directory to save the scripts 8 | TARGET=$1 9 | mkdir -p $TARGET 10 | 11 | # create Kafka scripts 12 | SCRIPTS=$(docker run --rm -it --entrypoint /bin/bash confluentinc/cp-kafka:7.6.0 -c "compgen -c | grep -E '(zook*|kafka*)'") 13 | for SCRIPT in $SCRIPTS; do 14 | SCRIPT=$(echo $SCRIPT | tr -d '\r') 15 | FN=$TARGET/$(basename $SCRIPT) 16 | echo creating $FN 17 | cat <<-EOF > $FN 18 | #!/bin/bash 19 | CMD="$SCRIPT \$@" 20 | docker run --net=host --rm -it --entrypoint /bin/bash confluentinc/cp-kafka:7.6.0 -c "\$CMD" 21 | EOF 22 | chmod +x $FN 23 | done 24 | 25 | # create ZooKeeper client scriptt 26 | echo creating $TARGET/zkCli.sh 27 | cat <<-EOF > $TARGET/zkCli.sh 28 | #!/bin/bash 29 | CMD="bin/zkCli.sh \$@" 30 | docker run --net=host --rm -it confluentinc/cp-zookeeper:7.6.0 bash -c "\$CMD" 31 | EOF 32 | chmod +x $TARGET/zkCli.sh 33 | -------------------------------------------------------------------------------- /examples/docker-compose.yml: -------------------------------------------------------------------------------- 1 | version: '3' 2 | services: 3 | zoo1: 4 | image: confluentinc/cp-zookeeper:7.6.0 5 | hostname: zoo1 6 | container_name: zoo1 7 | ports: 8 | - "2181:2181" 9 | environment: 10 | ZOOKEEPER_CLIENT_PORT: 2181 11 | ZOOKEEPER_TICK_TIME: 2000 12 | ulimits: 13 | nofile: 14 | soft: 65536 15 | hard: 65536 16 | kafka1: 17 | image: confluentinc/cp-kafka:7.6.0 18 | hostname: kafka1 19 | container_name: kafka1 20 | ulimits: 21 | nofile: 22 | soft: 65536 23 | hard: 65536 24 | ports: 25 | - "9092:9092" 26 | environment: 27 | KAFKA_ZOOKEEPER_CONNECT: zoo1:2181 28 | KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092,PLAINTEXT_HOST://localhost:29092,EXTERNAL_SAME_HOST://${DOCKER_HOST_IP:-127.0.0.1}:9092 29 | KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT,EXTERNAL_SAME_HOST:PLAINTEXT 30 | KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT 31 | KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 32 | KAFKA_AUTO_CREATE_TOPICS_ENABLE: "false" 33 | KAFKA_OFFSETS_COMMIT_TIMEOUT_MS: "10000" 34 | KAFKA_BROKER_ID: 1 35 | KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" 36 | KAFKA_DEFAULT_REPLICATION_FACTOR: 2 37 | KAFKA_NUM_PARTITIONS: 6 38 | depends_on: 39 | - zoo1 40 | kafka2: 41 | image: confluentinc/cp-kafka:7.6.0 42 | hostname: kafka2 43 | container_name: kafka2 44 | ulimits: 45 | nofile: 46 | soft: 65536 47 | hard: 65536 48 | ports: 49 | - "9093:9093" 50 | environment: 51 | KAFKA_ZOOKEEPER_CONNECT: zoo1:2181 52 | KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka2:19093,PLAINTEXT_HOST://localhost:29093,EXTERNAL_SAME_HOST://${DOCKER_HOST_IP:-127.0.0.1}:9093 53 | KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT,EXTERNAL_SAME_HOST:PLAINTEXT 54 | KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT 55 | KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 56 | KAFKA_AUTO_CREATE_TOPICS_ENABLE: "false" 57 | KAFKA_OFFSETS_COMMIT_TIMEOUT_MS: "10000" 58 | KAFKA_BROKER_ID: 2 59 | KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" 60 | KAFKA_DEFAULT_REPLICATION_FACTOR: 2 61 | KAFKA_NUM_PARTITIONS: 6 62 | depends_on: 63 | - zoo1 64 | kafka3: 65 | image: confluentinc/cp-kafka:7.6.0 66 | hostname: kafka3 67 | container_name: kafka3 68 | ulimits: 69 | nofile: 70 | soft: 65536 71 | hard: 65536 72 | ports: 73 | - "9094:9094" 74 | environment: 75 | KAFKA_ZOOKEEPER_CONNECT: zoo1:2181 76 | KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka3:19094,PLAINTEXT_HOST://localhost:29094,EXTERNAL_SAME_HOST://${DOCKER_HOST_IP:-127.0.0.1}:9094 77 | KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT,PLAINTEXT_HOST:PLAINTEXT,EXTERNAL_SAME_HOST:PLAINTEXT 78 | KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT 79 | KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 80 | KAFKA_AUTO_CREATE_TOPICS_ENABLE: "false" 81 | KAFKA_OFFSETS_COMMIT_TIMEOUT_MS: "10000" 82 | KAFKA_BROKER_ID: 3 83 | KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" 84 | KAFKA_DEFAULT_REPLICATION_FACTOR: 2 85 | KAFKA_NUM_PARTITIONS: 6 86 | depends_on: 87 | - zoo1 88 | redis: 89 | container_name: redis 90 | image: redis:latest 91 | ports: 92 | - "6379:6379" 93 | -------------------------------------------------------------------------------- /examples/wait-kafka.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # 4 | # waits for kafka to be ready, when it can list topics 5 | # $1: expects the hostname from docker compose and port in array form: localhost:9092,localhost:9093 6 | # 7 | 8 | wait_for_kafka() { 9 | IFS=','; for i in $1; do 10 | host="$(echo "$i" | cut -d':' -f1)" 11 | port="$(echo "$i" | cut -d':' -f2)" 12 | while ! docker exec -i ${host} sh -c "kafka-topics --bootstrap-server localhost:${port} --list"; do 13 | echo "Waiting for kafka '$i' to list topics..." 14 | sleep 5 15 | done 16 | echo "$i is ready. Continuing." 17 | done 18 | } 19 | 20 | if [ -z $1 ]; then 21 | echo "no arguments given. Please add then in the comma separated array form 'host:port,host2:port2" 22 | exit 1 23 | fi 24 | 25 | wait_for_kafka "$1"; 26 | 27 | echo "All dependencies are up. Ready!" -------------------------------------------------------------------------------- /examples/wait.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # 4 | # waits for tcp ports to be open. 5 | # $1: expects the hosts and port in array form: localhost:9092,localhost:9093 6 | # 7 | wait_for() { 8 | IFS=','; for i in $1; do 9 | host="$(echo "$i" | cut -d':' -f1)" 10 | port="$(echo "$i" | cut -d':' -f2)" 11 | while ! $(nc -z -v -w5 $host $port); do 12 | echo "Waiting for '$i' to come up..." 13 | sleep 5 14 | done 15 | echo "$i is up. Continuing." 16 | done 17 | } 18 | 19 | if [ -z $1 ]; then 20 | echo "no arguments given. Please add then in the comma separated array form 'host:port,host2:port2" 21 | exit 1 22 | fi 23 | 24 | wait_for "$1"; 25 | 26 | echo "All dependencies are up. Ready!" -------------------------------------------------------------------------------- /examples/wait_for_output.py: -------------------------------------------------------------------------------- 1 | """ Reads a file line by line repeatedly and searches the first occurrence of one or 2 | more strings. When all strings have been found, the script terminates with return code 0, 3 | if TIMEOUT_AFTER_SECONDS seconds have passed it will terminate with return code 1. 4 | 5 | Only used in GitHub action 'run-application'. 6 | """ 7 | 8 | import sys 9 | import json 10 | import time 11 | import argparse 12 | 13 | TIMEOUT_AFTER_SECONDS = 10 14 | 15 | 16 | def wait_until_output(file_to_scan, strings_to_match): 17 | """Reads a file line by line repeatedly and searches the first occurrence of one or 18 | more strings, returns True if all have been found in less than TIMEOUT_AFTER_SECONDS, 19 | else False. 20 | """ 21 | strings_to_match = set(strings_to_match) 22 | 23 | for _ in range(TIMEOUT_AFTER_SECONDS): 24 | with open(file_to_scan) as f: 25 | for line in f: 26 | for curr_str in list(strings_to_match): 27 | if curr_str in line: 28 | strings_to_match.remove(curr_str) 29 | if not strings_to_match: 30 | return True 31 | time.sleep(1) 32 | 33 | # not all strings found, return error 34 | return False 35 | 36 | 37 | def main(): 38 | parser = argparse.ArgumentParser() 39 | parser.add_argument('file', help='path to file to read') 40 | parser.add_argument('searchstrings', help='a JSON string or array of strings') 41 | args = parser.parse_args() 42 | 43 | file_to_scan = args.file 44 | strings_to_match = json.loads(args.searchstrings) 45 | if not isinstance(strings_to_match, list): 46 | strings_to_match = [strings_to_match] 47 | 48 | ret = wait_until_output(file_to_scan, strings_to_match) 49 | 50 | if not ret: 51 | sys.exit(1) 52 | 53 | 54 | if __name__ == '__main__': 55 | main() 56 | -------------------------------------------------------------------------------- /go.mod: -------------------------------------------------------------------------------- 1 | module github.com/lovoo/goka 2 | 3 | go 1.23.0 4 | 5 | require ( 6 | github.com/IBM/sarama v1.45.1 7 | github.com/go-stack/stack v1.8.1 8 | github.com/gorilla/mux v1.8.0 9 | github.com/hashicorp/go-multierror v1.1.1 10 | github.com/stretchr/testify v1.10.0 11 | github.com/syndtr/goleveldb v1.0.0 12 | go.uber.org/mock v0.5.2 13 | golang.org/x/sync v0.11.0 14 | gopkg.in/redis.v5 v5.2.9 15 | gopkg.in/yaml.v2 v2.4.0 16 | ) 17 | 18 | require ( 19 | github.com/davecgh/go-spew v1.1.1 // indirect 20 | github.com/eapache/go-resiliency v1.7.0 // indirect 21 | github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3 // indirect 22 | github.com/eapache/queue v1.1.0 // indirect 23 | github.com/golang/snappy v0.0.4 // indirect 24 | github.com/hashicorp/errwrap v1.1.0 // indirect 25 | github.com/hashicorp/go-uuid v1.0.3 // indirect 26 | github.com/jcmturner/aescts/v2 v2.0.0 // indirect 27 | github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect 28 | github.com/jcmturner/gofork v1.7.6 // indirect 29 | github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect 30 | github.com/jcmturner/rpc/v2 v2.0.3 // indirect 31 | github.com/klauspost/compress v1.17.11 // indirect 32 | github.com/kr/text v0.2.0 // indirect 33 | github.com/pierrec/lz4/v4 v4.1.22 // indirect 34 | github.com/pmezard/go-difflib v1.0.0 // indirect 35 | github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect 36 | github.com/rogpeppe/go-internal v1.9.0 // indirect 37 | golang.org/x/crypto v0.37.0 // indirect 38 | golang.org/x/mod v0.18.0 // indirect 39 | golang.org/x/net v0.39.0 // indirect 40 | golang.org/x/tools v0.22.0 // indirect 41 | gopkg.in/yaml.v3 v3.0.1 // indirect 42 | ) 43 | -------------------------------------------------------------------------------- /header_test.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "testing" 5 | ) 6 | 7 | func TestHeaders_Merged(t *testing.T) { 8 | h1 := Headers{ 9 | "key1": []byte("val1"), 10 | } 11 | h2 := Headers{ 12 | "key1": []byte("val1b"), 13 | "key2": []byte("val2"), 14 | } 15 | merged := h1.Merged(h2) 16 | 17 | if len(h1) != 1 || string(h1["key1"]) != "val1" { 18 | t.Errorf("Merged failed: receiver was modified") 19 | } 20 | 21 | if len(h2) != 2 || string(h2["key1"]) != "val1b" || string(h2["key2"]) != "val2" { 22 | t.Errorf("Merged failed: argument was modified") 23 | } 24 | 25 | if len(merged) != 2 { 26 | t.Errorf("Merged failed: expected %d keys, but found %d", 2, len(merged)) 27 | } 28 | 29 | if string(merged["key1"]) != "val1b" { 30 | t.Errorf("Merged failed: expected %q for key %q, but found %q", 31 | "val1b", "key1", string(merged["key1"])) 32 | } 33 | 34 | if string(merged["key2"]) != "val2" { 35 | t.Errorf("Merged failed: expected %q for key %q, but found %q", 36 | "val2", "key2", string(merged["key2"])) 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /headers.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "github.com/IBM/sarama" 5 | ) 6 | 7 | // Headers represents custom message headers with a convenient interface. 8 | type Headers map[string][]byte 9 | 10 | // HeadersFromSarama converts sarama headers to goka's type. 11 | func HeadersFromSarama(saramaHeaders []*sarama.RecordHeader) Headers { 12 | headers := Headers{} 13 | for _, rec := range saramaHeaders { 14 | headers[string(rec.Key)] = rec.Value 15 | } 16 | return headers 17 | } 18 | 19 | // Merged returns a new instance with all headers merged. Later keys override earlier ones. 20 | // Handles a nil receiver and nil arguments without panics. 21 | // If all headers are empty, nil is returned to allow using directly in emit functions. 22 | func (h Headers) Merged(headersList ...Headers) Headers { 23 | // optimize for headerless processors. 24 | if len(h) == 0 && allEmpty(headersList...) { 25 | return nil 26 | } 27 | 28 | merged := Headers{} 29 | for k, v := range h { 30 | merged[k] = v 31 | } 32 | 33 | for _, headers := range headersList { 34 | for k, v := range headers { 35 | merged[k] = v 36 | } 37 | } 38 | 39 | if len(merged) == 0 { 40 | return nil 41 | } 42 | 43 | return merged 44 | } 45 | 46 | // ToSarama converts the headers to a slice of sarama.RecordHeader. 47 | // If called on a nil receiver returns nil. 48 | func (h Headers) ToSarama() []sarama.RecordHeader { 49 | if h == nil { 50 | return nil 51 | } 52 | 53 | recordHeaders := make([]sarama.RecordHeader, 0, len(h)) 54 | for key, value := range h { 55 | recordHeaders = append(recordHeaders, 56 | sarama.RecordHeader{ 57 | Key: []byte(key), 58 | Value: value, 59 | }) 60 | } 61 | return recordHeaders 62 | } 63 | 64 | // ToSaramaPtr converts the headers to a slice of pointers to sarama.RecordHeader. 65 | // If called on a nil receiver returns nil. 66 | func (h Headers) ToSaramaPtr() []*sarama.RecordHeader { 67 | if h == nil { 68 | return nil 69 | } 70 | 71 | recordHeaders := make([]*sarama.RecordHeader, 0, len(h)) 72 | for key, value := range h { 73 | recordHeaders = append(recordHeaders, 74 | &sarama.RecordHeader{ 75 | Key: []byte(key), 76 | Value: value, 77 | }) 78 | } 79 | return recordHeaders 80 | } 81 | 82 | func allEmpty(headersList ...Headers) bool { 83 | for _, headers := range headersList { 84 | if len(headers) != 0 { 85 | return false 86 | } 87 | } 88 | return true 89 | } 90 | -------------------------------------------------------------------------------- /integrationtest/view_test.go: -------------------------------------------------------------------------------- 1 | package integrationtest 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | 7 | "github.com/lovoo/goka" 8 | "github.com/lovoo/goka/codec" 9 | "github.com/lovoo/goka/tester" 10 | "github.com/stretchr/testify/require" 11 | ) 12 | 13 | func TestView(t *testing.T) { 14 | t.Run("uninitialized_get", func(t *testing.T) { 15 | gkt := tester.New(t) 16 | 17 | // create an empty view on table "test" 18 | view, err := goka.NewView(nil, "test", new(codec.String), goka.WithViewTester(gkt)) 19 | require.NoError(t, err) 20 | 21 | // try to get a value 22 | val, err := view.Get("key") 23 | // --> must fail, because the view is not running yet 24 | require.Error(t, err) 25 | require.Nil(t, val) 26 | 27 | // start the view 28 | ctx, cancel := context.WithCancel(context.Background()) 29 | done := make(chan struct{}) 30 | go func() { 31 | defer close(done) 32 | if err := view.Run(ctx); err != nil { 33 | panic(err) 34 | } 35 | }() 36 | 37 | // set some value (this will wait internally until the view is running) 38 | gkt.SetTableValue("test", "key", "value") 39 | 40 | // try to get some non-existent key 41 | val, err = view.Get("not-existent") 42 | require.NoError(t, err) 43 | require.Nil(t, val) 44 | require.NoError(t, nil) 45 | 46 | // get the value we set earlier 47 | val, err = view.Get("key") 48 | require.NoError(t, err) 49 | require.Equal(t, "value", val.(string)) 50 | require.NoError(t, nil) 51 | 52 | // get all the keys from table "test" 53 | keys := gkt.GetTableKeys("test") 54 | // at the moment we only have one key "key" 55 | require.Equal(t, []string{"key"}, keys) 56 | 57 | // set a second key 58 | gkt.SetTableValue("test", "key2", "value") 59 | keys = gkt.GetTableKeys("test") 60 | require.Equal(t, []string{"key", "key2"}, keys) 61 | 62 | // stop the view and wait for it to finish up 63 | cancel() 64 | <-done 65 | }) 66 | } 67 | -------------------------------------------------------------------------------- /iterator.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "github.com/lovoo/goka/storage" 5 | ) 6 | 7 | // Iterator allows one to iterate over the keys of a view. 8 | type Iterator interface { 9 | // Next advances the iterator to the next KV-pair. Err should be called 10 | // after Next returns false to check whether the iteration finished 11 | // from exhaustion or was aborted due to an error. 12 | Next() bool 13 | // Err returns the error that stopped the iteration if any. 14 | Err() error 15 | // Return the key of the current item 16 | Key() string 17 | // Return the value of the current item 18 | // This value is already decoded with the view's codec (or nil, if it's nil) 19 | Value() (interface{}, error) 20 | // Release the iterator. After release, the iterator is not usable anymore 21 | Release() 22 | // Seek moves the iterator to the begining of a key-value pair sequence that 23 | // is greater or equal to the given key. It returns whether at least one of 24 | // such key-value pairs exist. If true is returned, Key/Value must be called 25 | // immediately to get the first item. Calling Next immediately after a successful 26 | // seek will effectively skip an item in the iterator. 27 | Seek(key string) bool 28 | } 29 | 30 | type iterator struct { 31 | iter storage.Iterator 32 | codec Codec 33 | } 34 | 35 | // Next advances the iterator to the next key. 36 | func (i *iterator) Next() bool { 37 | return i.iter.Next() 38 | } 39 | 40 | // Key returns the current key. 41 | func (i *iterator) Key() string { 42 | return string(i.iter.Key()) 43 | } 44 | 45 | // Value returns the current value decoded by the codec of the storage. 46 | func (i *iterator) Value() (interface{}, error) { 47 | data, err := i.iter.Value() 48 | if err != nil { 49 | return nil, err 50 | } else if data == nil { 51 | return nil, nil 52 | } 53 | return i.codec.Decode(data) 54 | } 55 | 56 | // Err returns the possible iteration error. 57 | func (i *iterator) Err() error { 58 | return i.iter.Err() 59 | } 60 | 61 | // Releases releases the iterator. The iterator is not usable anymore after calling Release. 62 | func (i *iterator) Release() { 63 | i.iter.Release() 64 | } 65 | 66 | func (i *iterator) Seek(key string) bool { 67 | return i.iter.Seek([]byte(key)) 68 | } 69 | -------------------------------------------------------------------------------- /iterator_test.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "os" 5 | "testing" 6 | 7 | "github.com/lovoo/goka/codec" 8 | "github.com/lovoo/goka/storage" 9 | "github.com/stretchr/testify/require" 10 | "github.com/syndtr/goleveldb/leveldb" 11 | ) 12 | 13 | func TestIterator(t *testing.T) { 14 | tmpdir, err := os.MkdirTemp("", "goka_storage_TestIterator") 15 | require.NoError(t, err) 16 | 17 | db, err := leveldb.OpenFile(tmpdir, nil) 18 | require.NoError(t, err) 19 | 20 | st, err := storage.New(db) 21 | require.NoError(t, err) 22 | 23 | kv := map[string]string{ 24 | "key-1": "val-1", 25 | "key-2": "val-2", 26 | "key-3": "val-3", 27 | } 28 | 29 | for k, v := range kv { 30 | require.NoError(t, st.Set(k, []byte(v))) 31 | } 32 | 33 | require.NoError(t, st.SetOffset(777)) 34 | 35 | iter, err := st.Iterator() 36 | require.NoError(t, err) 37 | 38 | it := &iterator{ 39 | iter: storage.NewMultiIterator([]storage.Iterator{iter}), 40 | codec: new(codec.String), 41 | } 42 | defer it.Release() 43 | count := 0 44 | 45 | // accessing iterator before Next should only return nils 46 | val, err := it.Value() 47 | require.True(t, val == nil) 48 | require.NoError(t, err) 49 | 50 | for it.Next() { 51 | count++ 52 | key := it.Key() 53 | expected, ok := kv[key] 54 | if !ok { 55 | t.Fatalf("unexpected key from iterator: %s", key) 56 | } 57 | 58 | val, err := it.Value() 59 | require.NoError(t, err) 60 | require.Equal(t, val.(string), expected) 61 | } 62 | 63 | if err := it.Err(); err != nil { 64 | t.Fatalf("unexpected iteration error: %v", err) 65 | } 66 | 67 | require.Equal(t, len(kv), count) 68 | } 69 | -------------------------------------------------------------------------------- /logger.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "fmt" 5 | "log" 6 | "os" 7 | "strings" 8 | 9 | "github.com/IBM/sarama" 10 | ) 11 | 12 | var defaultLogger = &std{ 13 | log: log.New(os.Stderr, "", log.LstdFlags), 14 | } 15 | 16 | // Logger is the interface Goka and its subpackages use for logging. 17 | type Logger interface { 18 | // Print will simply print the params 19 | Print(...interface{}) 20 | 21 | // Print will simply print the params 22 | Println(...interface{}) 23 | 24 | // Printf will be used for informational messages. These can be thought of 25 | // having an 'Info'-level in a structured logger. 26 | Printf(string, ...interface{}) 27 | } 28 | 29 | type logger interface { 30 | Logger 31 | // Debugf is used for debugging messages, mostly for debugging goka itself. 32 | // It is turned off unless goka is initialized 33 | Debugf(string, ...interface{}) 34 | // PrefixedLogger returns a logger that prefixes all messages with passed prefix 35 | Prefix(string) logger 36 | 37 | CurrentPrefix() string 38 | 39 | StackPrefix(prefix string) logger 40 | } 41 | 42 | // std bridges the logger calls to the standard library log. 43 | type std struct { 44 | log Logger 45 | debug bool 46 | prefixPath []string 47 | prefix string 48 | } 49 | 50 | func (s *std) Print(msgs ...interface{}) { 51 | s.log.Print(msgs...) 52 | } 53 | 54 | func (s *std) Println(msgs ...interface{}) { 55 | s.log.Print(msgs...) 56 | } 57 | 58 | func (s *std) Printf(msg string, args ...interface{}) { 59 | s.log.Printf(fmt.Sprintf("%s%s", s.prefix, msg), args...) 60 | } 61 | 62 | func (s *std) Debugf(msg string, args ...interface{}) { 63 | if s.debug { 64 | s.log.Printf(fmt.Sprintf("%s%s", s.prefix, msg), args...) 65 | } 66 | } 67 | 68 | func (s *std) Prefix(prefix string) logger { 69 | return s.StackPrefix(prefix).(*std) 70 | } 71 | 72 | // Default returns the standard library logger 73 | func DefaultLogger() Logger { 74 | return defaultLogger 75 | } 76 | 77 | // Debug enables or disables debug logging using the global logger. 78 | // The goka debugging setting is applied to any custom loggers in goka components (Processors, Views, Emitters). 79 | func Debug(gokaDebug, saramaDebug bool) { 80 | defaultLogger.debug = gokaDebug 81 | if saramaDebug { 82 | SetSaramaLogger((&std{log: defaultLogger, debug: true}).Prefix("Sarama")) 83 | } 84 | } 85 | 86 | func SetSaramaLogger(logger Logger) { 87 | sarama.Logger = logger 88 | } 89 | 90 | // newLogger creates a new goka logger 91 | func wrapLogger(l Logger, debug bool) logger { 92 | return &std{ 93 | log: l, 94 | debug: debug, 95 | } 96 | } 97 | 98 | func (s *std) CurrentPrefix() string { 99 | return s.prefix 100 | } 101 | 102 | func (s *std) StackPrefix(prefix string) logger { 103 | var prefPath []string 104 | // append existing path 105 | prefPath = append(prefPath, s.prefixPath...) 106 | 107 | // if new is not empty, append to path 108 | if prefix != "" { 109 | prefPath = append(prefPath, prefix) 110 | } 111 | 112 | // make new prefix 113 | newPrefix := strings.Join(prefPath, " > ") 114 | if newPrefix != "" { 115 | newPrefix = "[" + newPrefix + "] " 116 | } 117 | 118 | return &std{ 119 | log: s.log, 120 | prefixPath: prefPath, 121 | prefix: newPrefix, 122 | debug: s.debug, 123 | } 124 | } 125 | -------------------------------------------------------------------------------- /mockbuilder.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "errors" 5 | "fmt" 6 | "hash" 7 | "testing" 8 | 9 | "github.com/lovoo/goka/storage" 10 | "go.uber.org/mock/gomock" 11 | ) 12 | 13 | var ( 14 | errProducerBuilder error = errors.New("building producer failed on purpose") 15 | ) 16 | 17 | type builderMock struct { 18 | ctrl *gomock.Controller 19 | st storage.Storage 20 | mst *MockStorage 21 | tmgr *MockTopicManager 22 | consumerGroup *MockConsumerGroup 23 | producer *MockProducer 24 | client *MockClient 25 | admin *MockClusterAdmin 26 | } 27 | 28 | func newBuilderMock(ctrl *gomock.Controller) *builderMock { 29 | return &builderMock{ 30 | ctrl: ctrl, 31 | mst: NewMockStorage(ctrl), 32 | tmgr: NewMockTopicManager(ctrl), 33 | producer: NewMockProducer(ctrl), 34 | client: NewMockClient(ctrl), 35 | admin: NewMockClusterAdmin(ctrl), 36 | } 37 | } 38 | 39 | func (bm *builderMock) createProcessorOptions(consBuilder SaramaConsumerBuilder, groupBuilder ConsumerGroupBuilder) []ProcessorOption { 40 | return []ProcessorOption{ 41 | WithStorageBuilder(bm.getStorageBuilder()), 42 | WithTopicManagerBuilder(bm.getTopicManagerBuilder()), 43 | WithProducerBuilder(bm.getProducerBuilder()), 44 | WithConsumerGroupBuilder(groupBuilder), 45 | WithConsumerSaramaBuilder(consBuilder), 46 | } 47 | } 48 | 49 | func (bm *builderMock) getStorageBuilder() storage.Builder { 50 | return func(topic string, partition int32) (storage.Storage, error) { 51 | if bm.st != nil { 52 | return bm.st, nil 53 | } 54 | return bm.mst, nil 55 | } 56 | } 57 | 58 | func (bm *builderMock) getTopicManagerBuilder() TopicManagerBuilder { 59 | return func([]string) (TopicManager, error) { 60 | return bm.tmgr, nil 61 | } 62 | } 63 | 64 | func (bm *builderMock) getProducerBuilder() ProducerBuilder { 65 | return func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { 66 | return bm.producer, nil 67 | } 68 | } 69 | 70 | func (bm *builderMock) getErrorProducerBuilder() ProducerBuilder { 71 | return func(brokers []string, clientID string, hasher func() hash.Hash32) (Producer, error) { 72 | return nil, errProducerBuilder 73 | } 74 | } 75 | 76 | func (bm *builderMock) useMemoryStorage() { 77 | bm.st = storage.NewMemory() 78 | } 79 | 80 | func errStorageBuilder() storage.Builder { 81 | return func(topic string, partition int32) (storage.Storage, error) { 82 | return nil, fmt.Errorf("error returned by errStorageBuilder") 83 | } 84 | } 85 | 86 | func defaultSaramaAutoConsumerMock(t *testing.T) *MockAutoConsumer { 87 | return NewMockAutoConsumer(t, DefaultConfig()) 88 | } 89 | -------------------------------------------------------------------------------- /mockcontroller.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "fmt" 5 | 6 | "go.uber.org/mock/gomock" 7 | ) 8 | 9 | type gomockPanicker struct { 10 | reporter gomock.TestReporter 11 | } 12 | 13 | func (gm *gomockPanicker) Errorf(format string, args ...interface{}) { 14 | gm.reporter.Errorf(format, args...) 15 | } 16 | func (gm *gomockPanicker) Fatalf(format string, args ...interface{}) { 17 | defer panic(fmt.Sprintf(format, args...)) 18 | gm.reporter.Fatalf(format, args...) 19 | } 20 | 21 | // NewMockController returns a *gomock.Controller using a wrapped testing.T (or whatever) 22 | // which panics on a Fatalf. This is necessary when using a mock in kafkamock. 23 | // Otherwise it will freeze on an unexpected call. 24 | func NewMockController(t gomock.TestReporter) *gomock.Controller { 25 | return gomock.NewController(&gomockPanicker{reporter: t}) 26 | } 27 | -------------------------------------------------------------------------------- /multierr/errgroup.go: -------------------------------------------------------------------------------- 1 | package multierr 2 | 3 | import ( 4 | "context" 5 | "sync" 6 | 7 | "github.com/hashicorp/go-multierror" 8 | "golang.org/x/sync/errgroup" 9 | ) 10 | 11 | // ErrGroup implements a group of parallel running tasks allowing to 12 | // wait for all routines to terminate, as well as error handling. 13 | type ErrGroup struct { 14 | *errgroup.Group 15 | mutex sync.Mutex 16 | err *multierror.Error 17 | } 18 | 19 | // NewErrGroup creates a new ErrGroup using passed context. 20 | func NewErrGroup(ctx context.Context) (*ErrGroup, context.Context) { 21 | g, ctx := errgroup.WithContext(ctx) 22 | return &ErrGroup{Group: g}, ctx 23 | } 24 | 25 | // Wait blocks until all goroutines of the error group have terminated and returns 26 | // the accumulated errors. 27 | func (g *ErrGroup) Wait() *multierror.Error { 28 | g.Group.Wait() 29 | return g.err 30 | } 31 | 32 | // WaitChan returns a channel that is closed after the error group terminates, possibly 33 | // containing the error 34 | func (g *ErrGroup) WaitChan() <-chan *multierror.Error { 35 | errs := make(chan *multierror.Error, 1) 36 | go func() { 37 | defer close(errs) 38 | errs <- g.Wait() 39 | }() 40 | return errs 41 | } 42 | 43 | // Go starts a new goroutine. Termination of all functions can be checked via 44 | // Wait or WaitChan. 45 | // The ErrGroup closes the internal context when the first go-routine returns with an error. 46 | func (g *ErrGroup) Go(f func() error) { 47 | g.Group.Go(func() error { 48 | if err := f(); err != nil { 49 | g.mutex.Lock() 50 | defer g.mutex.Unlock() 51 | g.err = multierror.Append(g.err, err) 52 | return err 53 | } 54 | return nil 55 | }) 56 | } 57 | -------------------------------------------------------------------------------- /multierr/errgroup_test.go: -------------------------------------------------------------------------------- 1 | package multierr 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "testing" 7 | "time" 8 | 9 | "github.com/stretchr/testify/require" 10 | ) 11 | 12 | func TestErrGroup_Go(t *testing.T) { 13 | bctx := context.Background() 14 | 15 | // no errors 16 | g, ctx := NewErrGroup(bctx) 17 | g.Go(func() error { return nil }) 18 | errs := g.Wait() 19 | err := errs.ErrorOrNil() 20 | require.NoError(t, err) 21 | require.Error(t, ctx.Err()) 22 | require.Contains(t, ctx.Err().Error(), "context canceled") 23 | 24 | // with one error 25 | g, ctx = NewErrGroup(bctx) 26 | g.Go(func() error { return fmt.Errorf("some error") }) 27 | errs = g.Wait() 28 | err = errs.ErrorOrNil() 29 | require.Error(t, err) 30 | require.Contains(t, err.Error(), "some error") 31 | require.Error(t, ctx.Err()) 32 | require.Contains(t, ctx.Err().Error(), "context canceled") 33 | 34 | // with one error 35 | g, ctx = NewErrGroup(bctx) 36 | g.Go(func() error { return fmt.Errorf("some error") }) 37 | g.Go(func() error { return fmt.Errorf("some error2") }) 38 | errs = g.Wait() 39 | err = errs.ErrorOrNil() 40 | require.Error(t, err) 41 | require.Contains(t, err.Error(), "some error") 42 | require.Contains(t, err.Error(), "some error2") 43 | require.Error(t, ctx.Err()) 44 | require.Contains(t, ctx.Err().Error(), "context canceled") 45 | } 46 | 47 | func TestErrGroup_Empty(t *testing.T) { 48 | ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) 49 | defer cancel() 50 | errg, errgCtx := NewErrGroup(ctx) 51 | 52 | require.NoError(t, errg.Wait().ErrorOrNil()) 53 | select { 54 | case <-errgCtx.Done(): 55 | default: 56 | t.Errorf("context of errgroup was not cancelled after err group terminated") 57 | } 58 | 59 | select { 60 | case <-ctx.Done(): 61 | t.Errorf("context timed out") 62 | default: 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /multierr/errors.go: -------------------------------------------------------------------------------- 1 | package multierr 2 | 3 | import ( 4 | "fmt" 5 | "sync" 6 | ) 7 | 8 | // Errors represent a list of errors triggered during the execution of a goka view/processor. 9 | // Normally, the first error leads to stopping the processor/view, but during shutdown, more errors 10 | // might occur. 11 | // DEPRECATED. This will be removed one day, we migrated to the implementation in 12 | // github.com/hashicorp/go-multierror 13 | type Errors struct { 14 | errs []error 15 | m sync.Mutex 16 | } 17 | 18 | func (e *Errors) Collect(err error) *Errors { 19 | if err == nil { 20 | return e 21 | } 22 | e.m.Lock() 23 | e.errs = append(e.errs, err) 24 | e.m.Unlock() 25 | return e 26 | } 27 | 28 | func (e *Errors) Merge(o *Errors) *Errors { 29 | if o == nil { 30 | return e 31 | } 32 | 33 | // lock base 34 | e.m.Lock() 35 | defer e.m.Unlock() 36 | // lock other 37 | o.m.Lock() 38 | defer o.m.Unlock() 39 | 40 | e.errs = append(e.errs, o.errs...) 41 | return e 42 | } 43 | 44 | func (e *Errors) HasErrors() bool { 45 | return len(e.errs) > 0 46 | } 47 | 48 | func (e *Errors) Error() string { 49 | if !e.HasErrors() { 50 | return "" 51 | } 52 | if len(e.errs) == 1 { 53 | return e.errs[0].Error() 54 | } 55 | str := "Errors:\n" 56 | for _, err := range e.errs { 57 | str += fmt.Sprintf("\t* %s\n", err.Error()) 58 | } 59 | return str 60 | } 61 | 62 | func (e *Errors) NilOrError() error { 63 | if e.HasErrors() { 64 | return e 65 | } 66 | return nil 67 | } 68 | -------------------------------------------------------------------------------- /multierr/multiwait.go: -------------------------------------------------------------------------------- 1 | package multierr 2 | 3 | import ( 4 | "context" 5 | "sync" 6 | ) 7 | 8 | type MultiWait struct { 9 | ctx context.Context 10 | wg sync.WaitGroup 11 | } 12 | 13 | func NewMultiWait(ctx context.Context, num int) *MultiWait { 14 | mw := &MultiWait{ 15 | ctx: ctx, 16 | } 17 | mw.wg.Add(num) 18 | 19 | return mw 20 | } 21 | 22 | func (mw *MultiWait) Add(done <-chan struct{}) { 23 | go func() { 24 | select { 25 | case <-mw.ctx.Done(): 26 | case <-done: 27 | mw.wg.Done() 28 | } 29 | }() 30 | } 31 | 32 | func (mw *MultiWait) Done() <-chan struct{} { 33 | done := make(chan struct{}) 34 | go func() { 35 | defer close(done) 36 | mw.wg.Wait() 37 | }() 38 | return done 39 | } 40 | 41 | func (mw *MultiWait) Wait() bool { 42 | select { 43 | case <-mw.Done(): 44 | return true 45 | case <-mw.ctx.Done(): 46 | return false 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /once.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import "sync" 4 | 5 | type once struct { 6 | once sync.Once 7 | err error 8 | } 9 | 10 | // Do runs only once and always return the same error. 11 | func (o *once) Do(f func() error) error { 12 | o.once.Do(func() { o.err = f() }) 13 | return o.err 14 | } 15 | -------------------------------------------------------------------------------- /once_test.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "errors" 5 | "testing" 6 | 7 | "github.com/stretchr/testify/require" 8 | ) 9 | 10 | func TestOnce_Do(t *testing.T) { 11 | var o once 12 | 13 | err := o.Do(func() error { return errors.New("some error") }) 14 | require.Error(t, err) 15 | 16 | err2 := o.Do(func() error { return nil }) 17 | require.Error(t, err2) 18 | require.Equal(t, err2, err) 19 | } 20 | -------------------------------------------------------------------------------- /options_test.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "fmt" 5 | "testing" 6 | 7 | "github.com/lovoo/goka/storage" 8 | "github.com/stretchr/testify/require" 9 | ) 10 | 11 | func nullStorageBuilder() storage.Builder { 12 | return func(topic string, partition int32) (storage.Storage, error) { 13 | return &storage.Null{}, nil 14 | } 15 | } 16 | 17 | func newMockOptions(t *testing.T) *poptions { 18 | opts := new(poptions) 19 | err := opts.applyOptions(new(GroupGraph)) 20 | require.Error(t, err) 21 | 22 | opts.builders.storage = nullStorageBuilder() 23 | err = opts.applyOptions(new(GroupGraph)) 24 | require.NoError(t, err) 25 | 26 | fmt.Printf("%+v\n", opts) 27 | return opts 28 | } 29 | -------------------------------------------------------------------------------- /producer.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "fmt" 5 | "sync" 6 | "time" 7 | 8 | "github.com/IBM/sarama" 9 | ) 10 | 11 | // Producer abstracts the kafka producer 12 | type Producer interface { 13 | // Emit sends a message to topic. 14 | Emit(topic string, key string, value []byte) *Promise 15 | EmitWithHeaders(topic string, key string, value []byte, headers Headers) *Promise 16 | Close() error 17 | } 18 | 19 | type producer struct { 20 | producer sarama.AsyncProducer 21 | wg sync.WaitGroup 22 | } 23 | 24 | // NewProducer creates new kafka producer for passed brokers. 25 | func NewProducer(brokers []string, config *sarama.Config) (Producer, error) { 26 | aprod, err := sarama.NewAsyncProducer(brokers, config) 27 | if err != nil { 28 | return nil, fmt.Errorf("Failed to start Sarama producer: %v", err) 29 | } 30 | 31 | p := producer{ 32 | producer: aprod, 33 | } 34 | 35 | p.run() 36 | 37 | return &p, nil 38 | } 39 | 40 | // Close stops the producer and waits for the Success/Error channels to drain. 41 | // Emitting to a closing/closed producer results in write-to-closed-channel panic 42 | func (p *producer) Close() error { 43 | // do an async close to get the rest of the success/error messages to avoid 44 | // leaving unfinished promises. 45 | p.producer.AsyncClose() 46 | 47 | // wait for the channels to drain 48 | done := make(chan struct{}) 49 | go func() { 50 | p.wg.Wait() 51 | close(done) 52 | }() 53 | 54 | select { 55 | case <-done: 56 | case <-time.NewTimer(60 * time.Second).C: 57 | } 58 | 59 | return nil 60 | } 61 | 62 | // Emit emits a key-value pair to topic and returns a Promise that 63 | // can be checked for errors asynchronously 64 | func (p *producer) Emit(topic string, key string, value []byte) *Promise { 65 | promise := NewPromise() 66 | 67 | p.producer.Input() <- &sarama.ProducerMessage{ 68 | Topic: topic, 69 | Key: sarama.StringEncoder(key), 70 | Value: sarama.ByteEncoder(value), 71 | Metadata: promise, 72 | } 73 | return promise 74 | } 75 | 76 | // EmitWithHeaders emits a key-value pair with headers to topic and returns a Promise that 77 | // can be checked for errors asynchronously 78 | func (p *producer) EmitWithHeaders(topic string, key string, value []byte, headers Headers) *Promise { 79 | promise := NewPromise() 80 | 81 | p.producer.Input() <- &sarama.ProducerMessage{ 82 | Topic: topic, 83 | Key: sarama.StringEncoder(key), 84 | Value: sarama.ByteEncoder(value), 85 | Metadata: promise, 86 | Headers: headers.ToSarama(), 87 | } 88 | return promise 89 | } 90 | 91 | // resolve or reject a promise in the message's metadata on Success or Error 92 | func (p *producer) run() { 93 | p.wg.Add(2) 94 | go func() { 95 | defer p.wg.Done() 96 | for { 97 | err, ok := <-p.producer.Errors() 98 | 99 | // channel closed, the producer is stopping 100 | if !ok { 101 | return 102 | } 103 | err.Msg.Metadata.(*Promise).finish(nil, err) 104 | } 105 | }() 106 | 107 | go func() { 108 | defer p.wg.Done() 109 | for { 110 | msg, ok := <-p.producer.Successes() 111 | // channel closed, the producer is stopping 112 | if !ok { 113 | return 114 | } 115 | msg.Metadata.(*Promise).finish(msg, nil) 116 | } 117 | }() 118 | } 119 | -------------------------------------------------------------------------------- /producer_test.go: -------------------------------------------------------------------------------- 1 | // +build goka 2 | 3 | package goka 4 | 5 | import ( 6 | "log" 7 | "sync/atomic" 8 | "testing" 9 | "time" 10 | ) 11 | 12 | // This tests how a producer behaves when being shutdown to make sure, 13 | // no promises stay unfinished. 14 | // To run the test, get a local kafka-container running (e.g. go to 15 | // examples-directory and do `make restart`), then run the tests with 16 | // `go test -v github.com/lovoo/goka/kafka/ -tags=kafka` 17 | func TestProducerError(t *testing.T) { 18 | cfg := DefaultConfig() 19 | p, err := NewProducer([]string{"localhost:9092"}, cfg) 20 | 21 | if err != nil { 22 | t.Fatalf("error creating producer: %v", err) 23 | } 24 | 25 | var ( 26 | promises []*Promise 27 | donePromises int64 28 | emitted int64 29 | done = make(chan bool) 30 | ) 31 | 32 | go func() { 33 | defer func() { 34 | recover() 35 | }() 36 | defer close(done) 37 | for { 38 | promise := p.Emit("test", "test", []byte{}) 39 | promise.Then(func(err error) { 40 | atomic.AddInt64(&donePromises, 1) 41 | if err != nil { 42 | log.Printf("error producing message: %v", err) 43 | } 44 | }) 45 | promises = append(promises, promise) 46 | emitted++ 47 | time.Sleep(20 * time.Millisecond) 48 | } 49 | }() 50 | 51 | // let it run for 1 second 52 | time.Sleep(1000 * time.Millisecond) 53 | 54 | // close the producer 55 | err = p.Close() 56 | if err != nil { 57 | log.Printf("Error closing producer: %v", err) 58 | } 59 | // wait for the promises to be 60 | <-done 61 | 62 | if len(promises) != int(emitted) { 63 | t.Errorf("Promises/Emits do not match: promises: %d, emitted. %d", len(promises), emitted) 64 | } 65 | if len(promises) != int(donePromises) { 66 | t.Errorf("Promises/Done promises do not match: promises: %d, done. %d", len(promises), donePromises) 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /promise.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "sync" 5 | 6 | "github.com/IBM/sarama" 7 | ) 8 | 9 | // Promise as in https://en.wikipedia.org/wiki/Futures_and_promises 10 | type Promise struct { 11 | sync.Mutex 12 | err error 13 | msg *sarama.ProducerMessage 14 | finished bool 15 | 16 | callbacks []func(msg *sarama.ProducerMessage, err error) 17 | } 18 | 19 | // PromiseFinisher finishes a promise 20 | type PromiseFinisher func(msg *sarama.ProducerMessage, err error) *Promise 21 | 22 | // NewPromise creates a new Promise 23 | func NewPromise() *Promise { 24 | return new(Promise) 25 | } 26 | 27 | // NewPromiseWithFinisher creates a new Promise and a separate finish method. 28 | // This is necessary if the promise is used outside of goka package. 29 | func NewPromiseWithFinisher() (*Promise, PromiseFinisher) { 30 | p := new(Promise) 31 | return p, p.finish 32 | } 33 | 34 | // execute all callbacks conveniently 35 | // The caller needs to lock! 36 | func (p *Promise) executeCallbacks() { 37 | // already resolved 38 | if p.finished { 39 | return 40 | } 41 | for _, s := range p.callbacks { 42 | s(p.msg, p.err) 43 | } 44 | // mark as finished 45 | p.finished = true 46 | } 47 | 48 | // Then chains a callback to the Promise 49 | func (p *Promise) Then(callback func(err error)) *Promise { 50 | return p.ThenWithMessage(func(_ *sarama.ProducerMessage, err error) { 51 | callback(err) 52 | }) 53 | } 54 | 55 | // ThenWithMessage chains a callback to the Promise 56 | func (p *Promise) ThenWithMessage(callback func(msg *sarama.ProducerMessage, err error)) *Promise { 57 | p.Lock() 58 | defer p.Unlock() 59 | 60 | // promise already run, call the callback immediately 61 | if p.finished { 62 | callback(p.msg, p.err) 63 | // append it to the subscribers otherwise 64 | } else { 65 | p.callbacks = append(p.callbacks, callback) 66 | } 67 | return p 68 | } 69 | 70 | // Finish finishes the promise by executing all callbacks and saving the message/error for late subscribers 71 | func (p *Promise) finish(msg *sarama.ProducerMessage, err error) *Promise { 72 | p.Lock() 73 | defer p.Unlock() 74 | 75 | p.err = err 76 | p.msg = msg 77 | 78 | p.executeCallbacks() 79 | return p 80 | } 81 | -------------------------------------------------------------------------------- /promise_test.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "errors" 5 | "testing" 6 | 7 | "github.com/stretchr/testify/require" 8 | ) 9 | 10 | func TestPromise_thenBeforeFinish(t *testing.T) { 11 | p := new(Promise) 12 | 13 | var promiseErr error 14 | p.Then(func(err error) { 15 | promiseErr = err 16 | }) 17 | 18 | p.finish(nil, errors.New("test")) 19 | 20 | require.Equal(t, "test", promiseErr.Error()) 21 | 22 | // repeating finish won't change result 23 | p.finish(nil, errors.New("test-whatever")) 24 | 25 | require.Equal(t, "test", promiseErr.Error()) 26 | } 27 | 28 | func TestPromise_thenAfterFinish(t *testing.T) { 29 | p := new(Promise) 30 | 31 | var promiseErr error 32 | p.finish(nil, errors.New("test")) 33 | p.Then(func(err error) { 34 | promiseErr = err 35 | }) 36 | 37 | require.Equal(t, "test", promiseErr.Error()) 38 | } 39 | -------------------------------------------------------------------------------- /proxy.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "github.com/lovoo/goka/storage" 5 | ) 6 | 7 | type storageProxy struct { 8 | storage.Storage 9 | topic Stream 10 | partition int32 11 | stateless bool 12 | update UpdateCallback 13 | 14 | openedOnce once 15 | closedOnce once 16 | } 17 | 18 | func (s *storageProxy) Open() error { 19 | if s == nil { 20 | return nil 21 | } 22 | return s.openedOnce.Do(s.Storage.Open) 23 | } 24 | 25 | func (s *storageProxy) Close() error { 26 | if s == nil { 27 | return nil 28 | } 29 | return s.closedOnce.Do(s.Storage.Close) 30 | } 31 | 32 | func (s *storageProxy) Update(ctx UpdateContext, k string, v []byte) error { 33 | return s.update(ctx, s, k, v) 34 | } 35 | 36 | func (s *storageProxy) Stateless() bool { 37 | return s.stateless 38 | } 39 | 40 | func (s *storageProxy) MarkRecovered() error { 41 | return s.Storage.MarkRecovered() 42 | } 43 | -------------------------------------------------------------------------------- /proxy_test.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/IBM/sarama" 7 | "github.com/lovoo/goka/storage" 8 | ) 9 | 10 | type nullProxy struct{} 11 | 12 | func (p *nullProxy) Add(topic string, offset int64) error { return nil } 13 | func (p *nullProxy) Remove(topic string) error { return nil } 14 | func (p *nullProxy) AddGroup() {} 15 | func (p *nullProxy) Stop() {} 16 | 17 | func TestUpdateWithHeaders(t *testing.T) { 18 | s := storageProxy{ 19 | update: func(ctx UpdateContext, s storage.Storage, key string, value []byte) error { 20 | if len(ctx.Headers()) == 0 { 21 | t.Errorf("Missing headers") 22 | return nil 23 | } 24 | if string(ctx.Headers()["key"]) != "value" { 25 | t.Errorf("Key missmatch. Expected %q. Found: %q", "key", string(ctx.Headers()["key"])) 26 | } 27 | return nil 28 | }, 29 | } 30 | _ = s.Update(&DefaultUpdateContext{headers: []*sarama.RecordHeader{{Key: []byte("key"), Value: []byte("value")}}}, "", nil) 31 | } 32 | -------------------------------------------------------------------------------- /signal_test.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/require" 7 | ) 8 | 9 | func TestSignal_SetState(t *testing.T) { 10 | sig := NewSignal(0, 1, 2) 11 | require.True(t, sig.IsState(0)) 12 | require.False(t, sig.IsState(1)) 13 | 14 | sig.SetState(1) 15 | require.True(t, sig.IsState(1)) 16 | require.False(t, sig.IsState(0)) 17 | 18 | defer func() { 19 | err := recover() 20 | if err == nil { 21 | t.Fatalf("Expected panic, which didn't occur") 22 | } 23 | }() 24 | 25 | // set some invalid state, this will panic 26 | sig.SetState(3) 27 | } 28 | 29 | func TestSignal_Wait(t *testing.T) { 30 | sig := NewSignal(0, 1, 2) 31 | 32 | <-sig.WaitForState(0) 33 | // should continue right now since 34 | 35 | var ( 36 | done = make(chan struct{}) 37 | hasState bool 38 | ) 39 | go func() { 40 | defer close(done) 41 | <-sig.WaitForState(1) 42 | hasState = true 43 | }() 44 | 45 | require.False(t, hasState) 46 | sig.SetState(1) 47 | // wait for the goroutine to catchup with the state 48 | <-done 49 | require.True(t, hasState) 50 | } 51 | 52 | func TestSignalWaitMin(t *testing.T) { 53 | sig := NewSignal(0, 1, 2) 54 | 55 | var ( 56 | done = make(chan struct{}) 57 | hasState bool 58 | ) 59 | go func() { 60 | defer close(done) 61 | <-sig.WaitForStateMin(1) 62 | hasState = true 63 | }() 64 | 65 | require.False(t, hasState) 66 | sig.SetState(2) 67 | // wait for the goroutine to catchup with the state 68 | <-done 69 | require.True(t, hasState) 70 | } 71 | -------------------------------------------------------------------------------- /simple_backoff.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "sync" 5 | "time" 6 | ) 7 | 8 | // NewSimpleBackoff returns a simple backoff waiting the 9 | // specified duration longer each iteration until reset. 10 | func NewSimpleBackoff(step time.Duration, max time.Duration) Backoff { 11 | return &simpleBackoff{ 12 | step: step, 13 | max: max, 14 | } 15 | } 16 | 17 | type simpleBackoff struct { 18 | sync.Mutex 19 | 20 | current time.Duration 21 | step time.Duration 22 | max time.Duration 23 | } 24 | 25 | func (b *simpleBackoff) Reset() { 26 | b.Lock() 27 | defer b.Unlock() 28 | b.current = time.Duration(0) 29 | } 30 | 31 | func (b *simpleBackoff) Duration() time.Duration { 32 | b.Lock() 33 | defer b.Unlock() 34 | value := b.current 35 | 36 | if (b.current + b.step) <= b.max { 37 | b.current += b.step 38 | } 39 | return value 40 | } 41 | -------------------------------------------------------------------------------- /simple_backoff_test.go: -------------------------------------------------------------------------------- 1 | package goka 2 | 3 | import ( 4 | "testing" 5 | "time" 6 | 7 | "github.com/stretchr/testify/require" 8 | ) 9 | 10 | func TestSimpleBackoff(t *testing.T) { 11 | t.Run("simple progression", func(t *testing.T) { 12 | backoff := NewSimpleBackoff(time.Second, 10*time.Second) 13 | for i := 0; i < 10; i++ { 14 | require.Equal(t, backoff.Duration(), time.Duration(i)*time.Second) 15 | } 16 | 17 | // it doesn't go higher than the max 18 | require.Equal(t, backoff.Duration(), 10*time.Second) 19 | require.Equal(t, backoff.Duration(), 10*time.Second) 20 | }) 21 | t.Run("reset", func(t *testing.T) { 22 | backoff := NewSimpleBackoff(time.Second, 10*time.Second) 23 | 24 | require.Equal(t, time.Duration(0), backoff.Duration()) 25 | backoff.Duration() 26 | require.True(t, backoff.Duration() != 0) 27 | backoff.Reset() 28 | require.Equal(t, time.Duration(0), backoff.Duration()) 29 | }) 30 | } 31 | -------------------------------------------------------------------------------- /storage/append.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | "fmt" 5 | "io" 6 | "os" 7 | "path/filepath" 8 | ) 9 | 10 | type file struct { 11 | file io.WriteCloser 12 | recovered bool 13 | 14 | bytesWritten int64 15 | } 16 | 17 | // NewFile retuns a new on-disk storage. 18 | func NewFile(path string, part int32) (Storage, error) { 19 | if err := os.MkdirAll(path, os.ModePerm); err != nil { 20 | return nil, fmt.Errorf("error creating storage directory: %v", err) 21 | } 22 | 23 | f, err := os.OpenFile(filepath.Join(path, fmt.Sprintf("part-%d", part)), os.O_CREATE|os.O_RDWR|os.O_APPEND, os.ModePerm) 24 | if err != nil { 25 | return nil, err 26 | } 27 | 28 | return &file{file: f}, nil 29 | } 30 | 31 | func (f *file) Recovered() bool { 32 | return f.recovered 33 | } 34 | 35 | func (f *file) MarkRecovered() error { 36 | f.recovered = true 37 | return nil 38 | } 39 | 40 | func (f *file) Has(key string) (bool, error) { 41 | return false, nil 42 | } 43 | 44 | func (f *file) Get(key string) ([]byte, error) { 45 | return nil, nil 46 | } 47 | 48 | func (f *file) Set(key string, val []byte) error { 49 | num, err := f.file.Write(val) 50 | if err != nil { 51 | return err 52 | } 53 | 54 | f.bytesWritten += int64(num) 55 | 56 | if _, err := f.file.Write([]byte("\n")); err != nil { 57 | return err 58 | } 59 | 60 | return nil 61 | } 62 | 63 | func (f *file) Delete(string) error { 64 | return nil 65 | } 66 | 67 | func (f *file) GetOffset(def int64) (int64, error) { 68 | return def, nil 69 | } 70 | 71 | func (f *file) SetOffset(val int64) error { 72 | return nil 73 | } 74 | 75 | func (f *file) Iterator() (Iterator, error) { 76 | return new(NullIter), nil 77 | } 78 | 79 | func (f *file) IteratorWithRange(start, limit []byte) (Iterator, error) { 80 | return new(NullIter), nil 81 | } 82 | 83 | func (f *file) Open() error { 84 | return nil 85 | } 86 | 87 | func (f *file) Close() error { 88 | return f.file.Close() 89 | } 90 | -------------------------------------------------------------------------------- /storage/builders.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | "fmt" 5 | "path/filepath" 6 | 7 | "github.com/syndtr/goleveldb/leveldb" 8 | "github.com/syndtr/goleveldb/leveldb/opt" 9 | ) 10 | 11 | // Builder creates a local storage (a persistent cache) for a topic 12 | // table. Builder creates one storage for each partition of the topic. 13 | type Builder func(topic string, partition int32) (Storage, error) 14 | 15 | // DefaultBuilder builds a LevelDB storage with default configuration. 16 | // The database will be stored in the given path. 17 | func DefaultBuilder(path string) Builder { 18 | return func(topic string, partition int32) (Storage, error) { 19 | fp := filepath.Join(path, fmt.Sprintf("%s.%d", topic, partition)) 20 | db, err := leveldb.OpenFile(fp, nil) 21 | if err != nil { 22 | return nil, fmt.Errorf("error opening leveldb: %v", err) 23 | } 24 | return New(db) 25 | } 26 | } 27 | 28 | // BuilderWithOptions builds LevelDB storage with the given options and 29 | // in the given path. 30 | func BuilderWithOptions(path string, opts *opt.Options) Builder { 31 | return func(topic string, partition int32) (Storage, error) { 32 | fp := filepath.Join(path, fmt.Sprintf("%s.%d", topic, partition)) 33 | db, err := leveldb.OpenFile(fp, opts) 34 | if err != nil { 35 | return nil, fmt.Errorf("error opening leveldb: %v", err) 36 | } 37 | return New(db) 38 | } 39 | } 40 | 41 | // MemoryBuilder builds in-memory storage. 42 | func MemoryBuilder() Builder { 43 | return func(topic string, partition int32) (Storage, error) { 44 | return NewMemory(), nil 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /storage/iterator.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | ldbiter "github.com/syndtr/goleveldb/leveldb/iterator" 5 | ) 6 | 7 | // iterator wraps an Iterator implementation and handles the value decoding and 8 | // offset key skipping. 9 | type iterator struct { 10 | iter ldbiter.Iterator 11 | } 12 | 13 | func (i *iterator) Next() bool { 14 | next := i.iter.Next() 15 | if string(i.iter.Key()) == offsetKey { 16 | next = i.iter.Next() 17 | } 18 | 19 | return next 20 | } 21 | 22 | func (i *iterator) Err() error { 23 | return i.iter.Error() 24 | } 25 | 26 | func (i *iterator) Key() []byte { 27 | return i.iter.Key() 28 | } 29 | 30 | func (i *iterator) Value() ([]byte, error) { 31 | data := i.iter.Value() 32 | if data == nil { 33 | return nil, nil 34 | } 35 | 36 | return data, nil 37 | } 38 | 39 | func (i *iterator) Release() { 40 | i.iter.Release() 41 | } 42 | 43 | func (i *iterator) Seek(key []byte) bool { 44 | return i.iter.Seek(key) 45 | } 46 | -------------------------------------------------------------------------------- /storage/iterator_test.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | "os" 5 | "testing" 6 | 7 | "github.com/syndtr/goleveldb/leveldb" 8 | ) 9 | 10 | func TestIterator(t *testing.T) { 11 | tmpdir, err := os.MkdirTemp("", "goka_storage_TestIterator") 12 | assertNil(t, err) 13 | 14 | db, err := leveldb.OpenFile(tmpdir, nil) 15 | assertNil(t, err) 16 | 17 | st, err := New(db) 18 | assertNil(t, err) 19 | 20 | kv := map[string]string{ 21 | "key-1": "val-1", 22 | "key-2": "val-2", 23 | "key-3": "val-3", 24 | } 25 | 26 | for k, v := range kv { 27 | assertNil(t, st.Set(k, []byte(v))) 28 | } 29 | 30 | assertNil(t, st.SetOffset(777)) 31 | 32 | iter, err := st.Iterator() 33 | assertNil(t, err) 34 | defer iter.Release() 35 | count := 0 36 | 37 | // accessing iterator before Next should only return nils 38 | val, err := iter.Value() 39 | assertTrue(t, val == nil) 40 | assertNil(t, err) 41 | 42 | for iter.Next() { 43 | count++ 44 | key := string(iter.Key()) 45 | expected, ok := kv[key] 46 | if !ok { 47 | t.Fatalf("unexpected key from iterator: %s", key) 48 | } 49 | 50 | val, err := iter.Value() 51 | assertNil(t, err) 52 | assertEqual(t, expected, string(val)) 53 | } 54 | assertEqual(t, count, len(kv)) 55 | } 56 | -------------------------------------------------------------------------------- /storage/leveldb_test.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | "fmt" 5 | "os" 6 | "testing" 7 | 8 | "github.com/stretchr/testify/require" 9 | "github.com/syndtr/goleveldb/leveldb" 10 | ) 11 | 12 | var ( 13 | keys []string 14 | numKeys = 100 15 | numWrites = 200000 16 | ) 17 | 18 | func init() { 19 | for i := 0; i < numKeys; i++ { 20 | keys = append(keys, fmt.Sprintf("key-%d", i)) 21 | } 22 | } 23 | 24 | func BenchmarkStateStorage_unbatched(b *testing.B) { 25 | tmpdir, err := os.MkdirTemp("", "benchmark_statestorage_unbatched") 26 | require.NoError(b, err) 27 | 28 | db, err := leveldb.OpenFile(tmpdir, nil) 29 | require.NoError(b, err) 30 | 31 | storage, err := New(db) 32 | require.NoError(b, err) 33 | require.NoError(b, storage.MarkRecovered()) 34 | b.ResetTimer() 35 | for i := 0; i < b.N*numWrites; i++ { 36 | storage.Set(keys[i%len(keys)], []byte(fmt.Sprintf("value-%d", i))) 37 | } 38 | storage.Close() 39 | } 40 | 41 | func BenchmarkStateStorage_transactioned(b *testing.B) { 42 | tmpdir, err := os.MkdirTemp("", "benchmark_statestorage_transactioned") 43 | require.NoError(b, err) 44 | 45 | db, err := leveldb.OpenFile(tmpdir, nil) 46 | require.NoError(b, err) 47 | 48 | storage, err := New(db) 49 | require.NoError(b, err) 50 | b.ResetTimer() 51 | for i := 0; i < b.N*numWrites; i++ { 52 | storage.Set(keys[i%len(keys)], []byte(fmt.Sprintf("value-%d", i))) 53 | } 54 | require.NoError(b, storage.MarkRecovered()) 55 | storage.Close() 56 | } 57 | -------------------------------------------------------------------------------- /storage/memory_test.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "testing" 7 | "time" 8 | 9 | "github.com/lovoo/goka/multierr" 10 | "github.com/stretchr/testify/require" 11 | ) 12 | 13 | func TestMemoryStorage(t *testing.T) { 14 | t.Run("concurrent", func(t *testing.T) { 15 | mem := NewMemory() 16 | defer mem.Close() 17 | 18 | ctx, cancel := context.WithCancel(context.Background()) 19 | errg, ctx := multierr.NewErrGroup(ctx) 20 | 21 | // setter 22 | errg.Go(func() (err error) { 23 | defer func() { 24 | if r := recover(); r != nil { 25 | err = fmt.Errorf("panic: %v", r) 26 | } 27 | }() 28 | for i := 0; ; i++ { 29 | select { 30 | case <-ctx.Done(): 31 | return nil 32 | default: 33 | } 34 | mem.Set(fmt.Sprintf("%d", i%5), []byte(fmt.Sprintf("%d", i))) 35 | } 36 | }) 37 | 38 | // getter 39 | errg.Go(func() (err error) { 40 | defer func() { 41 | if r := recover(); r != nil { 42 | err = fmt.Errorf("panic: %v", r) 43 | } 44 | }() 45 | 46 | for i := 0; ; i++ { 47 | select { 48 | case <-ctx.Done(): 49 | return nil 50 | default: 51 | } 52 | mem.Get(fmt.Sprintf("%d", i%5)) 53 | } 54 | }) 55 | 56 | // get offset 57 | errg.Go(func() (err error) { 58 | defer func() { 59 | if r := recover(); r != nil { 60 | err = fmt.Errorf("panic: %v", r) 61 | } 62 | }() 63 | 64 | for i := 0; ; i++ { 65 | select { 66 | case <-ctx.Done(): 67 | return nil 68 | default: 69 | } 70 | mem.GetOffset(0) 71 | } 72 | }) 73 | 74 | // set offset 75 | errg.Go(func() (err error) { 76 | defer func() { 77 | if r := recover(); r != nil { 78 | err = fmt.Errorf("panic: %v", r) 79 | } 80 | }() 81 | 82 | for i := 0; ; i++ { 83 | select { 84 | case <-ctx.Done(): 85 | return nil 86 | default: 87 | } 88 | mem.SetOffset(123) 89 | } 90 | }) 91 | 92 | time.Sleep(1 * time.Second) 93 | cancel() 94 | 95 | require.NoError(t, errg.Wait().ErrorOrNil()) 96 | }) 97 | } 98 | -------------------------------------------------------------------------------- /storage/merge_iterator.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | "bytes" 5 | "container/heap" 6 | ) 7 | 8 | type iterHeap []Iterator 9 | 10 | func (h iterHeap) Len() int { 11 | return len(h) 12 | } 13 | 14 | func (h iterHeap) Less(i, j int) bool { 15 | return bytes.Compare(h[i].Key(), h[j].Key()) == -1 16 | } 17 | 18 | func (h iterHeap) Swap(i, j int) { 19 | h[i], h[j] = h[j], h[i] 20 | } 21 | 22 | func (h *iterHeap) Push(x interface{}) { 23 | *h = append(*h, x.(Iterator)) 24 | } 25 | 26 | func (h *iterHeap) Pop() interface{} { 27 | dref := *h 28 | x := dref[len(dref)-1] 29 | *h = dref[:len(dref)-1] 30 | return x 31 | } 32 | 33 | type mergeIterator struct { 34 | key []byte 35 | value []byte 36 | err error 37 | 38 | heap iterHeap 39 | iters []Iterator 40 | } 41 | 42 | // NewMultiIterator returns an Iterator that iterates over the given subiterators. 43 | // Iteration happens in lexicographical order given that the subiterators also 44 | // return values in order. 45 | func NewMultiIterator(iters []Iterator) Iterator { 46 | miter := &mergeIterator{ 47 | iters: iters, 48 | heap: make([]Iterator, 0, len(iters)), 49 | } 50 | 51 | miter.buildHeap(func(i Iterator) bool { return i.Next() }) 52 | 53 | return miter 54 | } 55 | 56 | func (m *mergeIterator) buildHeap(hasValue func(i Iterator) bool) { 57 | m.heap = m.heap[:0] 58 | 59 | for _, iter := range m.iters { 60 | if !hasValue(iter) { 61 | if m.err = iter.Err(); m.err != nil { 62 | return 63 | } 64 | 65 | continue 66 | } 67 | 68 | heap.Push(&m.heap, iter) 69 | } 70 | } 71 | 72 | // Key returns the current key. Caller should not keep references to the 73 | // buffer or modify its contents. 74 | func (m *mergeIterator) Key() []byte { 75 | return m.key 76 | } 77 | 78 | // Value returns the current value. Caller should not keep references to the 79 | // buffer or modify its contents. 80 | func (m *mergeIterator) Value() ([]byte, error) { 81 | return m.value, nil 82 | } 83 | 84 | // Seek moves the iterator to the beginning of a key-value pair sequence that 85 | // is greater or equal to the given key. It returns whether at least one 86 | // such key-value pairs exist. 87 | func (m *mergeIterator) Seek(key []byte) bool { 88 | if m.err != nil { 89 | return false 90 | } 91 | 92 | m.buildHeap(func(i Iterator) bool { return i.Seek(key) }) 93 | 94 | return m.err == nil && len(m.heap) > 0 95 | } 96 | 97 | // Next advances the iterator to the next key-value pair. If there is no next 98 | // pair, false is returned. Error should be checked after receiving false by 99 | // calling Error(). 100 | func (m *mergeIterator) Next() bool { 101 | if m.err != nil || len(m.heap) == 0 { 102 | return false 103 | } 104 | 105 | iter := heap.Pop(&m.heap).(Iterator) 106 | 107 | // cache the values as the underlying iterator might reuse its buffers on 108 | // call to Next 109 | m.key = append(m.key[:0], iter.Key()...) 110 | val, err := iter.Value() 111 | if err != nil { 112 | m.err = err 113 | return false 114 | } 115 | m.value = append(m.value[:0], val...) 116 | 117 | if iter.Next() { 118 | heap.Push(&m.heap, iter) 119 | } else if m.err = iter.Err(); m.err != nil { 120 | return false 121 | } 122 | 123 | return true 124 | } 125 | 126 | // Err returns the possible iteration error. 127 | func (m *mergeIterator) Err() error { 128 | return m.err 129 | } 130 | 131 | // Release frees up the resources used by the iterator. This will also release 132 | // the subiterators. 133 | func (m *mergeIterator) Release() { 134 | for i := range m.iters { 135 | m.iters[i].Release() 136 | } 137 | 138 | m.iters = nil 139 | m.heap = nil 140 | m.key = nil 141 | m.value = nil 142 | m.err = nil 143 | } 144 | -------------------------------------------------------------------------------- /storage/null.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | // Null storage discards everything that it is given. This can be useful for 4 | // debugging. 5 | type Null struct { 6 | recovered bool 7 | } 8 | 9 | // NewNull returns a new Null storage. 10 | func NewNull() Storage { 11 | return new(Null) 12 | } 13 | 14 | // MarkRecovered does nothing. 15 | func (n *Null) MarkRecovered() error { 16 | return nil 17 | } 18 | 19 | // Recovered returns whether the storage has recovered. 20 | func (n *Null) Recovered() bool { 21 | return n.recovered 22 | } 23 | 24 | // Has returns false as in key not found. 25 | func (n *Null) Has(key string) (bool, error) { 26 | return false, nil 27 | } 28 | 29 | // Get returns nil values. 30 | func (n *Null) Get(key string) ([]byte, error) { 31 | return nil, nil 32 | } 33 | 34 | // Set will do nothing and doesn't error. 35 | func (n *Null) Set(key string, val []byte) error { 36 | return nil 37 | } 38 | 39 | // Delete does nothing and doesn't error. 40 | func (n *Null) Delete(string) error { 41 | return nil 42 | } 43 | 44 | // GetOffset returns the default offset given to it. 45 | func (n *Null) GetOffset(def int64) (int64, error) { 46 | return def, nil 47 | } 48 | 49 | // SetOffset does nothing and doesn't error. 50 | func (n *Null) SetOffset(val int64) error { 51 | return nil 52 | } 53 | 54 | // Iterator returns an Iterator that is immediately exhausted. 55 | func (n *Null) Iterator() (Iterator, error) { 56 | return new(NullIter), nil 57 | } 58 | 59 | // IteratorWithRange returns an Iterator that is immediately exhausted. 60 | func (n *Null) IteratorWithRange(start, limit []byte) (Iterator, error) { 61 | return new(NullIter), nil 62 | } 63 | 64 | // Open does nothing and doesn't error. 65 | func (n *Null) Open() error { 66 | return nil 67 | } 68 | 69 | // Close does nothing and doesn't error 70 | func (n *Null) Close() error { 71 | return nil 72 | } 73 | 74 | // NullIter is an iterator which is immediately exhausted. 75 | type NullIter struct{} 76 | 77 | // Next returns always false. 78 | func (ni *NullIter) Next() bool { 79 | return false 80 | } 81 | 82 | func (*NullIter) Err() error { 83 | return nil 84 | } 85 | 86 | // Key returns always nil. 87 | func (ni *NullIter) Key() []byte { 88 | return nil 89 | } 90 | 91 | // Value returns always a nil value and no errors. 92 | func (ni *NullIter) Value() ([]byte, error) { 93 | return nil, nil 94 | } 95 | 96 | // Release does nothing. 97 | func (ni *NullIter) Release() {} 98 | 99 | // Seek do nothing 100 | func (ni *NullIter) Seek(key []byte) bool { return false } 101 | -------------------------------------------------------------------------------- /storage/redis/builders.go: -------------------------------------------------------------------------------- 1 | package redis 2 | 3 | import ( 4 | "errors" 5 | "fmt" 6 | 7 | "github.com/lovoo/goka/storage" 8 | 9 | redis "gopkg.in/redis.v5" 10 | ) 11 | 12 | // RedisBuilder builds redis storage. 13 | func RedisBuilder(client *redis.Client, namespace string) storage.Builder { 14 | return func(topic string, partition int32) (storage.Storage, error) { 15 | if namespace == "" { 16 | return nil, errors.New("missing namespace to redis storage") 17 | } 18 | return New(client, fmt.Sprintf("%s:%s:%d", namespace, topic, partition)) 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /storage/redis/redis_test.go: -------------------------------------------------------------------------------- 1 | package redis 2 | 3 | import ( 4 | "runtime/debug" 5 | "testing" 6 | ) 7 | 8 | func Test_redisIterator(t *testing.T) { 9 | assertRedisIterator(t, &redisIterator{ 10 | current: 0, 11 | keys: []string{ 12 | "key1", 13 | "val1", 14 | offsetKey, 15 | "123", 16 | "key2", 17 | "val2", 18 | }, 19 | }) 20 | assertRedisIterator(t, &redisIterator{ 21 | current: 0, 22 | keys: []string{ 23 | offsetKey, 24 | "123", 25 | "key1", 26 | "val1", 27 | "key2", 28 | "val2", 29 | }, 30 | }) 31 | assertRedisIterator(t, &redisIterator{ 32 | current: 0, 33 | keys: []string{ 34 | "key1", 35 | "val1", 36 | "key2", 37 | "val2", 38 | offsetKey, 39 | "123", 40 | }, 41 | }) 42 | } 43 | 44 | func assertRedisIterator(t *testing.T, it *redisIterator) { 45 | // the iterator contract implies we must call `Next()` first 46 | it.Next() 47 | assertRedisIteratorKey(t, it, "key1", "val1") 48 | 49 | it.Next() 50 | assertRedisIteratorKey(t, it, "key2", "val2") 51 | 52 | it.Next() 53 | if !it.exhausted() { 54 | t.Fatalf("Expected iterator to be exhausted in %s", string(debug.Stack())) 55 | } 56 | } 57 | 58 | func assertRedisIteratorKey(t *testing.T, it *redisIterator, expectedKey string, expectedValue string) { 59 | if it.exhausted() { 60 | t.Fatalf("Did not expect iterator to be exhausted in %s", string(debug.Stack())) 61 | } 62 | 63 | actualKey := string(it.Key()) 64 | if actualKey != expectedKey { 65 | t.Fatalf("Expected iterator key to be '%s', but was '%s' in %s", expectedKey, actualKey, string(debug.Stack())) 66 | } 67 | 68 | actualValue, _ := it.Value() 69 | if string(actualValue) != expectedValue { 70 | t.Fatalf("Expected iterator value to be '%s', but was '%s' in %s", expectedValue, actualValue, string(debug.Stack())) 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /storage/test.go: -------------------------------------------------------------------------------- 1 | package storage 2 | 3 | import ( 4 | "reflect" 5 | "regexp" 6 | "runtime/debug" 7 | "strings" 8 | "testing" 9 | ) 10 | 11 | func assertNil(t *testing.T, actual interface{}) { 12 | value := reflect.ValueOf(actual) 13 | if value.IsValid() { 14 | if !value.IsNil() { 15 | t.Fatalf("Expected value to be nil, but was not nil in %s", string(debug.Stack())) 16 | } 17 | } 18 | } 19 | 20 | func assertNotNil(t *testing.T, actual interface{}) { 21 | value := reflect.ValueOf(actual) 22 | if !value.IsValid() || value.IsNil() { 23 | t.Fatalf("Expected value to be not nil, but was nil in %s", string(debug.Stack())) 24 | } 25 | } 26 | 27 | func assertTrue(t *testing.T, value bool) { 28 | if !value { 29 | t.Fatalf("Expected value to be true, but was false in %s", string(debug.Stack())) 30 | } 31 | } 32 | func assertFalse(t *testing.T, value bool) { 33 | if value { 34 | t.Fatalf("Expected value to be false, but was true in %s", string(debug.Stack())) 35 | } 36 | } 37 | 38 | func assertEqual(t *testing.T, actual, expected interface{}) { 39 | if !reflect.DeepEqual(actual, expected) { 40 | t.Fatalf("Expected values were not equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) 41 | } 42 | } 43 | 44 | func panicAssertEqual(t *testing.T, expected interface{}) { 45 | if expected == nil { 46 | panic("can't pass nil to panicAssertEqual") 47 | } 48 | if actual := recover(); actual != nil { 49 | if !reflect.DeepEqual(actual, expected) { 50 | t.Fatalf("Expected values were equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) 51 | } 52 | } else { 53 | t.Errorf("panic expected") 54 | t.FailNow() 55 | } 56 | 57 | } 58 | 59 | func assertNotEqual(t *testing.T, actual, expected interface{}) { 60 | if reflect.DeepEqual(actual, expected) { 61 | t.Fatalf("Expected values were equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) 62 | } 63 | } 64 | 65 | func assertFuncEqual(t *testing.T, actual, expected interface{}) { 66 | if !(reflect.ValueOf(actual).Pointer() == reflect.ValueOf(expected).Pointer()) { 67 | t.Fatalf("Expected functions were equal.\nactual=%#v\nexpected=%#v in %s", actual, expected, string(debug.Stack())) 68 | } 69 | } 70 | 71 | func assertError(t *testing.T, actual error, reg *regexp.Regexp) { 72 | if actual == nil || reg == nil { 73 | t.Fatalf("Error or regexp is nil.\nactual=%#v\nregexp=%#v in %s", actual, reg, string(debug.Stack())) 74 | } 75 | if !reg.MatchString(actual.(error).Error()) { 76 | t.Fatalf("Expected but got.\nactual=%#v\nregexp=%#v in %s", actual, reg, string(debug.Stack())) 77 | } 78 | } 79 | 80 | func assertStringContains(t *testing.T, actual string, contains string) { 81 | if !strings.Contains(actual, contains) { 82 | t.Fatalf("Expected string to contain substring \nactual=%#v\nexpected=%#v in %s", actual, contains, string(debug.Stack())) 83 | } 84 | } 85 | 86 | func panicAssertStringContains(t *testing.T, s string) { 87 | if r := recover(); r != nil { 88 | err := r.(error) 89 | assertStringContains(t, err.Error(), s) 90 | } else { 91 | t.Errorf("panic expected") 92 | t.FailNow() 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /systemtest/emitter_disconnect_test.go: -------------------------------------------------------------------------------- 1 | package systemtest 2 | 3 | import ( 4 | "errors" 5 | "fmt" 6 | "log" 7 | "sync" 8 | "sync/atomic" 9 | "syscall" 10 | "testing" 11 | "time" 12 | 13 | "github.com/lovoo/goka" 14 | "github.com/lovoo/goka/codec" 15 | "github.com/stretchr/testify/require" 16 | ) 17 | 18 | func TestEmitter_KafkaDisconnect(t *testing.T) { 19 | brokers := initSystemTest(t) 20 | topic := goka.Stream(fmt.Sprintf("goka_systemtest_emitter_disconnect-%d", time.Now().Unix())) 21 | 22 | tmgr, err := goka.DefaultTopicManagerBuilder(brokers) 23 | require.NoError(t, err) 24 | require.NoError(t, tmgr.EnsureStreamExists(string(topic), 10)) 25 | 26 | cfg := goka.DefaultConfig() 27 | 28 | fi := NewFIProxy() 29 | cfg.Net.Proxy.Enable = true 30 | cfg.Net.Proxy.Dialer = fi 31 | 32 | // get it faster over with 33 | cfg.Producer.Retry.Max = 1 34 | cfg.Producer.Retry.Backoff = 0 35 | 36 | em, err := goka.NewEmitter(brokers, topic, new(codec.Int64), 37 | goka.WithEmitterProducerBuilder(goka.ProducerBuilderWithConfig(cfg)), 38 | ) 39 | require.NoError(t, err) 40 | var ( 41 | i int64 42 | success int64 43 | ) 44 | 45 | done := make(chan struct{}) 46 | go func() { 47 | defer close(done) 48 | var closeOnce sync.Once 49 | stop := make(chan struct{}) 50 | for { 51 | select { 52 | case <-stop: 53 | return 54 | default: 55 | } 56 | 57 | prom, err := em.Emit(fmt.Sprintf("key-%d", i%20), i) 58 | if err != nil { 59 | if errors.Is(err, goka.ErrEmitterAlreadyClosed) { 60 | return 61 | } 62 | log.Printf("error emitting: %v", err) 63 | } 64 | prom.Then(func(err error) { 65 | if err != nil { 66 | log.Printf("error emitting (async): %v", err) 67 | closeOnce.Do(func() { 68 | close(stop) 69 | }) 70 | return 71 | } 72 | if err == nil { 73 | atomic.AddInt64(&success, 1) 74 | } 75 | }) 76 | time.Sleep(10 * time.Millisecond) 77 | i++ 78 | } 79 | }() 80 | 81 | pollTimed(t, "emitter emitted something successfully", func() bool { 82 | return atomic.LoadInt64(&success) > 0 83 | }) 84 | 85 | fi.SetWriteError(syscall.EPIPE) 86 | <-done 87 | require.NoError(t, em.Finish()) 88 | } 89 | -------------------------------------------------------------------------------- /systemtest/emitter_offset_test.go: -------------------------------------------------------------------------------- 1 | package systemtest 2 | 3 | import ( 4 | "fmt" 5 | "log" 6 | "sync/atomic" 7 | "testing" 8 | "time" 9 | 10 | "github.com/IBM/sarama" 11 | "github.com/lovoo/goka" 12 | "github.com/lovoo/goka/codec" 13 | "github.com/stretchr/testify/require" 14 | ) 15 | 16 | // TestEmitterOffset is a simple brute force test that ensure 17 | // that the offsets, generated by kafka producers are monotonically increasing. 18 | // That is required to get rid of the former functionality "storeNewestOffset", that would ignore 19 | // older offsets. 20 | func TestEmitterOffset(t *testing.T) { 21 | var topic goka.Stream = goka.Stream(fmt.Sprintf("%s-%d", "goka-systemtest-emitter-offset", time.Now().Unix())) 22 | 23 | brokers := initSystemTest(t) 24 | 25 | tmc := goka.NewTopicManagerConfig() 26 | tmc.Table.Replication = 1 27 | cfg := goka.DefaultConfig() 28 | tm, err := goka.TopicManagerBuilderWithConfig(cfg, tmc)(brokers) 29 | require.NoError(t, err) 30 | tm.EnsureStreamExists(string(topic), 1) 31 | 32 | var lastOffset int64 33 | 34 | emitter, err := goka.NewEmitter(brokers, topic, new(codec.Int64)) 35 | if err != nil { 36 | t.Fatalf("error creating emitter: %v", err) 37 | } 38 | 39 | for i := 0; i < 100000; i++ { 40 | prom, err := emitter.Emit(fmt.Sprintf("%d", i), int64(i)) 41 | if err != nil { 42 | log.Fatalf("error emitting: %v", err) 43 | } 44 | prom.ThenWithMessage(func(msg *sarama.ProducerMessage, err error) { 45 | if err != nil { 46 | log.Fatalf("error emitting message: %v", err) 47 | } 48 | oldOffset := atomic.SwapInt64(&lastOffset, msg.Offset) 49 | if msg.Offset < oldOffset { 50 | log.Fatalf("offsets appeared in wrong order new=%d, old=%d", msg.Offset, oldOffset) 51 | } 52 | }) 53 | } 54 | 55 | defer func() { 56 | if err := emitter.Finish(); err != nil { 57 | log.Fatalf("error closing emitter: %v", err) 58 | } 59 | }() 60 | } 61 | -------------------------------------------------------------------------------- /systemtest/env.go: -------------------------------------------------------------------------------- 1 | package systemtest 2 | 3 | import ( 4 | "os" 5 | "strings" 6 | "testing" 7 | ) 8 | 9 | // Checks if the env-variable to activate system test is set and returns a broker 10 | // If system tests are not activated, will skip the test 11 | func initSystemTest(t *testing.T) []string { 12 | if _, isIntegration := os.LookupEnv("GOKA_SYSTEMTEST"); !isIntegration { 13 | t.Skip("*** skip integration test ***") 14 | } 15 | if brokers, ok := os.LookupEnv("GOKA_SYSTEMTEST_BROKERS"); ok { 16 | return strings.Split(brokers, ",") 17 | } 18 | return []string{"localhost:9092"} 19 | } 20 | -------------------------------------------------------------------------------- /systemtest/multitopic_test.go: -------------------------------------------------------------------------------- 1 | package systemtest 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "log" 7 | "math/rand" 8 | "testing" 9 | "time" 10 | 11 | "github.com/lovoo/goka" 12 | "github.com/lovoo/goka/codec" 13 | "github.com/lovoo/goka/multierr" 14 | "github.com/stretchr/testify/require" 15 | ) 16 | 17 | // Tests a processor with multiple input topics. Random values are emitted to random topics, the values are accumulated 18 | // for a single key and checked for correctness after emitting a couple of messages. 19 | // This is a regression/showcase test for https://github.com/lovoo/goka/issues/332 20 | func TestMultiTopics(t *testing.T) { 21 | brokers := initSystemTest(t) 22 | var ( 23 | group goka.Group = goka.Group(fmt.Sprintf("%s-%d", "goka-systemtest-multitopic", time.Now().Unix())) 24 | table = goka.GroupTable(group) 25 | inputStreams []goka.Stream 26 | ) 27 | 28 | for i := 0; i < 5; i++ { 29 | inputStreams = append(inputStreams, goka.Stream(fmt.Sprintf("%s-input-%d", string(group), i))) 30 | } 31 | 32 | tmc := goka.NewTopicManagerConfig() 33 | tmc.Table.Replication = 1 34 | tmc.Stream.Replication = 1 35 | cfg := goka.DefaultConfig() 36 | tm, err := goka.TopicManagerBuilderWithConfig(cfg, tmc)(brokers) 37 | require.NoError(t, err) 38 | 39 | for _, inStream := range inputStreams { 40 | err = tm.EnsureStreamExists(string(inStream), 1) 41 | require.NoError(t, err) 42 | } 43 | // let the cluster create it 44 | time.Sleep(5 * time.Second) 45 | 46 | proc, err := goka.NewProcessor(brokers, 47 | goka.DefineGroup( 48 | group, 49 | goka.Inputs(inputStreams, new(codec.Int64), func(ctx goka.Context, msg interface{}) { 50 | var oldVal int64 51 | 52 | if val := ctx.Value(); val != nil { 53 | oldVal = val.(int64) 54 | } 55 | 56 | // accumulate with old value 57 | ctx.SetValue(msg.(int64) + oldVal) 58 | }), 59 | goka.Persist(new(codec.Int64)), 60 | ), 61 | goka.WithTopicManagerBuilder(goka.TopicManagerBuilderWithTopicManagerConfig(tmc)), 62 | ) 63 | require.NoError(t, err) 64 | 65 | view, err := goka.NewView(brokers, table, new(codec.Int64)) 66 | require.NoError(t, err) 67 | 68 | var emitters []*goka.Emitter 69 | 70 | for _, input := range inputStreams { 71 | emitter, err := goka.NewEmitter(brokers, input, new(codec.Int64)) 72 | require.NoError(t, err) 73 | emitters = append(emitters, emitter) 74 | } 75 | 76 | ctx, cancel := context.WithCancel(context.Background()) 77 | defer cancel() 78 | errg, ctx := multierr.NewErrGroup(ctx) 79 | 80 | errg.Go(func() error { 81 | return proc.Run(ctx) 82 | }) 83 | errg.Go(func() error { 84 | return view.Run(ctx) 85 | }) 86 | 87 | log.Printf("waiting for processor/view to be running") 88 | pollTimed(t, "proc and view are recovered", proc.Recovered, view.Recovered) 89 | log.Printf("...done") 90 | 91 | var sum int64 92 | for i := int64(0); i < 100; i++ { 93 | value := rand.Int63n(100) 94 | // emit to random emitters in sync 95 | err := emitters[rand.Intn(len(emitters))].EmitSync("key", value) 96 | require.NoError(t, err) 97 | // ... and batched 98 | prom, err := emitters[rand.Intn(len(emitters))].Emit("key", value) 99 | require.NoError(t, err) 100 | prom.Then(func(err error) { 101 | require.NoError(t, err) 102 | }) 103 | 104 | // accumulate what we have sent so far 105 | sum += (value * 2) 106 | } 107 | 108 | for _, emitter := range emitters { 109 | require.NoError(t, emitter.Finish()) 110 | } 111 | 112 | // poll the view and the processor until we're sure that we have 113 | pollTimed(t, "all messages have been transferred", 114 | func() bool { 115 | value, err := view.Get("key") 116 | require.NoError(t, err) 117 | return value != nil && value.(int64) == sum 118 | }, 119 | func() bool { 120 | value, err := proc.Get("key") 121 | require.NoError(t, err) 122 | return value != nil && value.(int64) == sum 123 | }, 124 | ) 125 | 126 | // stop everything and wait until it's shut down 127 | cancel() 128 | require.NoError(t, errg.Wait().ErrorOrNil()) 129 | } 130 | -------------------------------------------------------------------------------- /systemtest/proc_disconnect_test.go: -------------------------------------------------------------------------------- 1 | package systemtest 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "io" 7 | "log" 8 | "testing" 9 | "time" 10 | 11 | "github.com/lovoo/goka" 12 | "github.com/lovoo/goka/codec" 13 | "github.com/lovoo/goka/multierr" 14 | "github.com/stretchr/testify/require" 15 | ) 16 | 17 | func TestProcessorShutdown_KafkaDisconnect(t *testing.T) { 18 | brokers := initSystemTest(t) 19 | var ( 20 | topic = goka.Stream(fmt.Sprintf("goka_systemtest_proc_shutdown_disconnect-%d", time.Now().Unix())) 21 | join = goka.Stream(fmt.Sprintf("goka_systemtest_proc_shutdown_disconnect-%d-join", time.Now().Unix())) 22 | group = goka.Group(topic) 23 | ) 24 | 25 | ctx, cancel := context.WithCancel(context.Background()) 26 | defer cancel() 27 | 28 | errg, ctx := multierr.NewErrGroup(ctx) 29 | 30 | tmgr, err := goka.DefaultTopicManagerBuilder(brokers) 31 | require.NoError(t, err) 32 | require.NoError(t, tmgr.EnsureStreamExists(string(topic), 10)) 33 | require.NoError(t, tmgr.EnsureTableExists(string(join), 10)) 34 | 35 | // emit values 36 | errg.Go(func() error { 37 | em, err := goka.NewEmitter(brokers, topic, new(codec.Int64)) 38 | require.NoError(t, err) 39 | defer em.Finish() 40 | var i int64 41 | for { 42 | select { 43 | case <-ctx.Done(): 44 | return nil 45 | default: 46 | } 47 | 48 | prom, err := em.Emit(fmt.Sprintf("key-%d", i%20), i) 49 | require.NoError(t, err) 50 | prom.Then(func(err error) { 51 | require.NoError(t, err) 52 | }) 53 | time.Sleep(100 * time.Millisecond) 54 | i++ 55 | } 56 | }) 57 | 58 | cfg := goka.DefaultConfig() 59 | 60 | fi := NewFIProxy() 61 | cfg.Net.Proxy.Enable = true 62 | cfg.Net.Proxy.Dialer = fi 63 | 64 | proc, err := goka.NewProcessor(brokers, 65 | goka.DefineGroup( 66 | group, 67 | goka.Input(topic, new(codec.Int64), func(ctx goka.Context, msg interface{}) { 68 | if val := ctx.Value(); val != nil { 69 | ctx.SetValue(val.(int64) + msg.(int64)) 70 | } else { 71 | ctx.SetValue(msg) 72 | } 73 | }), 74 | goka.Join(goka.Table(join), new(codec.String)), 75 | goka.Persist(new(codec.Int64)), 76 | ), 77 | goka.WithConsumerGroupBuilder(goka.ConsumerGroupBuilderWithConfig(cfg)), 78 | goka.WithProducerBuilder(goka.ProducerBuilderWithConfig(cfg)), 79 | goka.WithConsumerSaramaBuilder(goka.SaramaConsumerBuilderWithConfig(cfg)), 80 | ) 81 | require.NoError(t, err) 82 | 83 | errg.Go(func() error { 84 | return proc.Run(ctx) 85 | }) 86 | pollTimed(t, "proc running", proc.Recovered, func() bool { 87 | if val, _ := proc.Get("key-15"); val != nil && val.(int64) > 0 { 88 | return true 89 | } 90 | return false 91 | }) 92 | 93 | log.Printf("disconnecting consumer-group") 94 | fi.SetReadError(io.EOF) 95 | fi.SetWriteError(io.ErrClosedPipe) 96 | err = errg.Wait().ErrorOrNil() 97 | 98 | require.Error(t, err) 99 | } 100 | -------------------------------------------------------------------------------- /systemtest/proxy.go: -------------------------------------------------------------------------------- 1 | package systemtest 2 | 3 | import ( 4 | "net" 5 | "sync" 6 | ) 7 | 8 | // FIProxy is a fault injecting proxy hooked into the sarama-config 9 | // to proxy connections to kafka and inject connection loss etc. 10 | type FIProxy struct { 11 | m sync.RWMutex 12 | 13 | readErr error 14 | writeErr error 15 | 16 | conns map[string]*Conn 17 | } 18 | 19 | type Conn struct { 20 | net.Conn 21 | fip *FIProxy 22 | } 23 | 24 | func (c *Conn) Close() error { 25 | defer c.fip.removeConn(c.Conn) 26 | return c.Conn.Close() 27 | } 28 | 29 | func (c *Conn) Read(b []byte) (int, error) { 30 | c.fip.m.RLock() 31 | defer c.fip.m.RUnlock() 32 | 33 | if c.fip.readErr != nil { 34 | return 0, c.fip.readErr 35 | } 36 | return c.Conn.Read(b) 37 | } 38 | 39 | func (c *Conn) Write(b []byte) (int, error) { 40 | c.fip.m.RLock() 41 | defer c.fip.m.RUnlock() 42 | if c.fip.writeErr != nil { 43 | return 0, c.fip.writeErr 44 | } 45 | return c.Conn.Write(b) 46 | } 47 | 48 | func NewFIProxy() *FIProxy { 49 | return &FIProxy{ 50 | conns: make(map[string]*Conn), 51 | } 52 | } 53 | 54 | func (fip *FIProxy) Dial(network, addr string) (c net.Conn, err error) { 55 | fip.m.Lock() 56 | defer fip.m.Unlock() 57 | 58 | conn, err := net.Dial(network, addr) 59 | if err != nil { 60 | return nil, err 61 | } 62 | 63 | wrappedConn := &Conn{ 64 | Conn: conn, 65 | fip: fip, 66 | } 67 | key := conn.LocalAddr().String() 68 | 69 | fip.conns[key] = wrappedConn 70 | return wrappedConn, err 71 | } 72 | 73 | func (fip *FIProxy) removeConn(c net.Conn) { 74 | fip.m.Lock() 75 | defer fip.m.Unlock() 76 | 77 | delete(fip.conns, c.LocalAddr().String()) 78 | } 79 | 80 | func (fip *FIProxy) SetReadError(err error) { 81 | fip.m.Lock() 82 | defer fip.m.Unlock() 83 | fip.readErr = err 84 | } 85 | 86 | func (fip *FIProxy) SetWriteError(err error) { 87 | fip.m.Lock() 88 | defer fip.m.Unlock() 89 | fip.writeErr = err 90 | } 91 | 92 | func (fip *FIProxy) ResetErrors() { 93 | fip.m.Lock() 94 | defer fip.m.Unlock() 95 | fip.readErr = nil 96 | fip.writeErr = nil 97 | } 98 | 99 | func (fip *FIProxy) String() string { 100 | return "Fault Injecting Proxy (FIP)" 101 | } 102 | -------------------------------------------------------------------------------- /systemtest/topicmanager_test.go: -------------------------------------------------------------------------------- 1 | package systemtest 2 | 3 | import ( 4 | "crypto/rand" 5 | "encoding/hex" 6 | "strings" 7 | "testing" 8 | "time" 9 | 10 | "github.com/IBM/sarama" 11 | "github.com/lovoo/goka" 12 | "github.com/stretchr/testify/require" 13 | ) 14 | 15 | func TestTopicManagerCreate(t *testing.T) { 16 | brokers := initSystemTest(t) 17 | 18 | cfg := sarama.NewConfig() 19 | cfg.Version = sarama.V0_11_0_0 20 | 21 | tm, err := goka.TopicManagerBuilderWithConfig(cfg, goka.NewTopicManagerConfig())(brokers) 22 | require.NoError(t, err) 23 | 24 | err = tm.EnsureTopicExists("test10", 4, 2, nil) 25 | require.NoError(t, err) 26 | } 27 | 28 | // Tests the topic manager with sarama version v11 --> so it will test topic configuration using 29 | // the sarama.ClusterAdmin 30 | func TestTopicManager_v11(t *testing.T) { 31 | brokers := initSystemTest(t) 32 | 33 | cfg := sarama.NewConfig() 34 | cfg.Version = sarama.V0_11_0_0 35 | tmc := goka.NewTopicManagerConfig() 36 | tmc.Table.Replication = 1 37 | tmc.MismatchBehavior = goka.TMConfigMismatchBehaviorFail 38 | 39 | tm, err := goka.TopicManagerBuilderWithConfig(cfg, tmc)(brokers) 40 | require.NoError(t, err) 41 | 42 | client, _ := sarama.NewClient(brokers, cfg) 43 | admin, _ := sarama.NewClusterAdminFromClient(client) 44 | 45 | t.Run("ensure-new-stream", func(t *testing.T) { 46 | topic := newTopicName() 47 | 48 | // delete topic, ignore error if it does not exist 49 | admin.DeleteTopic(topic) 50 | 51 | err := tm.EnsureStreamExists(topic, 10) 52 | require.NoError(t, err) 53 | time.Sleep(1 * time.Second) 54 | // trying to create the same is fine 55 | err = tm.EnsureStreamExists(topic, 10) 56 | require.NoError(t, err) 57 | time.Sleep(1 * time.Second) 58 | // partitions changed - error 59 | err = tm.EnsureStreamExists(topic, 11) 60 | require.Error(t, err) 61 | }) 62 | 63 | t.Run("list-partitions", func(t *testing.T) { 64 | var ( 65 | topic = newTopicName() 66 | partitions []int32 67 | err error 68 | ) 69 | _, err = tm.Partitions(topic) 70 | require.Error(t, err) 71 | require.True(t, strings.Contains(err.Error(), "requested topic was not found")) 72 | require.Equal(t, 0, len(partitions)) 73 | 74 | tm.EnsureTableExists(topic, 123) 75 | time.Sleep(1 * time.Second) 76 | partitions, err = tm.Partitions(topic) 77 | require.NoError(t, err) 78 | require.Equal(t, 123, len(partitions)) 79 | }) 80 | 81 | t.Run("non-existent", func(t *testing.T) { 82 | // topic does not exist 83 | partitions, err := tm.Partitions("non-existent-topic") 84 | require.True(t, len(partitions) == 0, "expected no partitions, was", partitions) 85 | require.Error(t, err) 86 | }) 87 | } 88 | 89 | func newTopicName() string { 90 | topicBytes := make([]byte, 4) 91 | rand.Read(topicBytes) 92 | return hex.EncodeToString(topicBytes) 93 | } 94 | -------------------------------------------------------------------------------- /systemtest/view_reconnect_test.go: -------------------------------------------------------------------------------- 1 | package systemtest 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "io" 7 | "testing" 8 | "time" 9 | 10 | "github.com/lovoo/goka" 11 | "github.com/lovoo/goka/codec" 12 | "github.com/lovoo/goka/multierr" 13 | "github.com/stretchr/testify/require" 14 | ) 15 | 16 | // Tests the following scenario: 17 | // A view started with `WithViewAutoReconnect` should still return values even after losing connection to kafka. 18 | // Therefore we start a view on a topic fed by an emitter, the view proxies through the FIProxy and loses connection 19 | // after recovering. The values are still be served/returned 20 | func TestView_Reconnect(t *testing.T) { 21 | topic := fmt.Sprintf("goka_systemtest_view_reconnect_test-%d", time.Now().Unix()) 22 | brokers := initSystemTest(t) 23 | 24 | ctx, cancel := context.WithCancel(context.Background()) 25 | defer cancel() 26 | 27 | errg, ctx := multierr.NewErrGroup(ctx) 28 | 29 | tmgr, err := goka.DefaultTopicManagerBuilder(brokers) 30 | require.NoError(t, err) 31 | require.NoError(t, tmgr.EnsureStreamExists(topic, 10)) 32 | 33 | errg.Go(func() error { 34 | em, err := goka.NewEmitter(brokers, goka.Stream(topic), new(codec.Int64)) 35 | if err != nil { 36 | return err 37 | } 38 | defer em.Finish() 39 | var i int64 40 | for { 41 | select { 42 | case <-ctx.Done(): 43 | return nil 44 | default: 45 | } 46 | 47 | require.NoError(t, em.EmitSync("key", i)) 48 | time.Sleep(10 * time.Millisecond) 49 | i++ 50 | } 51 | }) 52 | 53 | cfg := goka.DefaultConfig() 54 | 55 | fi := NewFIProxy() 56 | cfg.Net.Proxy.Enable = true 57 | cfg.Net.Proxy.Dialer = fi 58 | 59 | // we'll use a view on the stream. 60 | view, err := goka.NewView(brokers, goka.Table(topic), new(codec.Int64), 61 | goka.WithViewAutoReconnect(), 62 | goka.WithViewConsumerSaramaBuilder(goka.SaramaConsumerBuilderWithConfig(cfg)), 63 | goka.WithViewTopicManagerBuilder(goka.TopicManagerBuilderWithConfig(cfg, goka.NewTopicManagerConfig())), 64 | ) 65 | require.NoError(t, err) 66 | 67 | // Start view and wait for it to be recovered 68 | errg.Go(func() error { 69 | return view.Run(ctx) 70 | }) 71 | pollTimed(t, "view-recovered", view.Recovered) 72 | 73 | val := func() int64 { 74 | val, err := view.Get("key") 75 | require.NoError(t, err) 76 | if val == nil { 77 | return 0 78 | } 79 | return val.(int64) 80 | } 81 | 82 | pollTimed(t, "wait-first-value", func() bool { 83 | return val() > 0 84 | }) 85 | firstVal := val() 86 | 87 | time.Sleep(500 * time.Millisecond) 88 | 89 | // kill kafka connection 90 | fi.SetReadError(io.EOF) 91 | pollTimed(t, "view-reconnecting", func() bool { 92 | return view.CurrentState() == goka.ViewStateConnecting 93 | }) 94 | 95 | // the view still should have gotten the update before the EOF 96 | secondVal := val() 97 | require.True(t, secondVal > firstVal) 98 | 99 | // let some time pass -> the value should not have updated 100 | time.Sleep(500 * time.Millisecond) 101 | require.True(t, val() == secondVal) 102 | 103 | // connect kafka again, wait until it's running -> the value should have changed 104 | fi.ResetErrors() 105 | pollTimed(t, "view-running", func() bool { 106 | return view.CurrentState() == goka.ViewStateRunning 107 | }) 108 | pollTimed(t, "value-propagated", func() bool { 109 | return val() > secondVal 110 | }) 111 | 112 | // shut everything down 113 | cancel() 114 | require.NoError(t, errg.Wait().ErrorOrNil()) 115 | } 116 | -------------------------------------------------------------------------------- /tester/client.go: -------------------------------------------------------------------------------- 1 | package tester 2 | 3 | type client struct { 4 | clientID string 5 | consumerGroup *consumerGroup 6 | consumer *consumerMock 7 | } 8 | 9 | func (c *client) waitStartup() { 10 | if c.consumerGroup != nil { 11 | c.consumerGroup.waitRunning() 12 | } 13 | 14 | c.consumer.waitRequiredConsumersStartup() 15 | } 16 | 17 | func (c *client) requireConsumer(topic string) { 18 | c.consumer.requirePartConsumer(topic) 19 | } 20 | 21 | func (c *client) catchup() int { 22 | var catchup int 23 | if c.consumerGroup != nil { 24 | catchup += c.consumerGroup.catchupAndWait() 25 | } 26 | 27 | catchup += c.consumer.catchup() 28 | 29 | return catchup 30 | } 31 | -------------------------------------------------------------------------------- /tester/consumer.go: -------------------------------------------------------------------------------- 1 | package tester 2 | 3 | import ( 4 | "fmt" 5 | "sync" 6 | "time" 7 | 8 | "github.com/IBM/sarama" 9 | ) 10 | 11 | type consumerMock struct { 12 | sync.RWMutex 13 | tester *Tester 14 | requiredTopics map[string]bool 15 | partConsumers map[string]*partConsumerMock 16 | } 17 | 18 | func newConsumerMock(tt *Tester) *consumerMock { 19 | return &consumerMock{ 20 | tester: tt, 21 | requiredTopics: make(map[string]bool), 22 | partConsumers: make(map[string]*partConsumerMock), 23 | } 24 | } 25 | 26 | func (cm *consumerMock) catchup() int { 27 | cm.RLock() 28 | defer cm.RUnlock() 29 | var catchup int 30 | for _, pc := range cm.partConsumers { 31 | catchup += pc.catchup() 32 | } 33 | return catchup 34 | } 35 | 36 | func (cm *consumerMock) Topics() ([]string, error) { 37 | cm.tester.mCodecs.RLock() 38 | defer cm.tester.mCodecs.RUnlock() 39 | 40 | var topics []string 41 | 42 | for topic := range cm.tester.codecs { 43 | topics = append(topics, topic) 44 | } 45 | return topics, nil 46 | } 47 | 48 | func (cm *consumerMock) Partitions(topic string) ([]int32, error) { 49 | return []int32{0}, nil 50 | } 51 | 52 | func (cm *consumerMock) ConsumePartition(topic string, partition int32, offset int64) (sarama.PartitionConsumer, error) { 53 | cm.Lock() 54 | defer cm.Unlock() 55 | if _, exists := cm.partConsumers[topic]; exists { 56 | return nil, fmt.Errorf("Got duplicate consume partition for topic %s", topic) 57 | } 58 | cons := &partConsumerMock{ 59 | hwm: offset, 60 | queue: cm.tester.getOrCreateQueue(topic), 61 | messages: make(chan *sarama.ConsumerMessage), 62 | errors: make(chan *sarama.ConsumerError), 63 | closer: func() error { 64 | cm.Lock() 65 | defer cm.Unlock() 66 | if _, exists := cm.partConsumers[topic]; !exists { 67 | return fmt.Errorf("partition consumer seems already closed") 68 | } 69 | delete(cm.partConsumers, topic) 70 | return nil 71 | }, 72 | } 73 | 74 | cm.partConsumers[topic] = cons 75 | 76 | return cons, nil 77 | } 78 | func (cm *consumerMock) HighWaterMarks() map[string]map[int32]int64 { 79 | return nil 80 | } 81 | func (cm *consumerMock) Close() error { 82 | return nil 83 | } 84 | 85 | func (cm *consumerMock) Pause(topicPartitions map[string][]int32) {} 86 | 87 | func (cm *consumerMock) Resume(topicPartitions map[string][]int32) {} 88 | 89 | func (cm *consumerMock) PauseAll() {} 90 | 91 | func (cm *consumerMock) ResumeAll() {} 92 | 93 | func (cm *consumerMock) waitRequiredConsumersStartup() { 94 | doCheck := func() bool { 95 | cm.RLock() 96 | defer cm.RUnlock() 97 | 98 | for topic := range cm.requiredTopics { 99 | _, ok := cm.partConsumers[topic] 100 | if !ok { 101 | return false 102 | } 103 | } 104 | return true 105 | } 106 | for !doCheck() { 107 | time.Sleep(50 * time.Millisecond) 108 | } 109 | } 110 | 111 | func (cm *consumerMock) requirePartConsumer(topic string) { 112 | cm.requiredTopics[topic] = true 113 | } 114 | 115 | type partConsumerMock struct { 116 | hwm int64 117 | closer func() error 118 | messages chan *sarama.ConsumerMessage 119 | errors chan *sarama.ConsumerError 120 | queue *queue 121 | } 122 | 123 | func (pcm *partConsumerMock) catchup() int { 124 | var numCatchup int 125 | for _, msg := range pcm.queue.messagesFromOffset(pcm.hwm) { 126 | pcm.messages <- &sarama.ConsumerMessage{ 127 | Headers: msg.headers.ToSaramaPtr(), 128 | Key: []byte(msg.key), 129 | Value: msg.value, 130 | Topic: pcm.queue.topic, 131 | Partition: 0, 132 | Offset: msg.offset, 133 | } 134 | 135 | // we'll send a nil that is being ignored by the partition_table to make sure the other message 136 | // really went through the channel 137 | pcm.messages <- nil 138 | numCatchup++ 139 | pcm.hwm = msg.offset + 1 140 | } 141 | 142 | return numCatchup 143 | } 144 | 145 | func (pcm *partConsumerMock) Close() error { 146 | close(pcm.messages) 147 | close(pcm.errors) 148 | return pcm.closer() 149 | } 150 | 151 | func (pcm *partConsumerMock) AsyncClose() { 152 | go pcm.Close() 153 | } 154 | 155 | func (pcm *partConsumerMock) Messages() <-chan *sarama.ConsumerMessage { 156 | return pcm.messages 157 | } 158 | 159 | func (pcm *partConsumerMock) Errors() <-chan *sarama.ConsumerError { 160 | return pcm.errors 161 | } 162 | 163 | func (pcm *partConsumerMock) HighWaterMarkOffset() int64 { 164 | return pcm.queue.Hwm() 165 | } 166 | 167 | func (pcm *partConsumerMock) Pause() {} 168 | 169 | func (pcm *partConsumerMock) Resume() {} 170 | 171 | func (pcm *partConsumerMock) IsPaused() bool { 172 | return false 173 | } 174 | -------------------------------------------------------------------------------- /tester/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | 3 | This package provides a kafka mock that allows integration testing of goka processors. 4 | 5 | Usage 6 | 7 | Simply append a tester option when creating the processor for testing. 8 | Usually it makes sense to move the processor creation to a function that accepts 9 | extra options. That way the test can use exactly the same processor setup. 10 | 11 | // creates the processor defining its group graph 12 | func createProcessor(brokers []string, options ...goka.ProcessorOption) *goka.Processor{ 13 | return goka.NewProcessor(brokers, goka.DefineGroup("group", 14 | // some group definitions 15 | options..., 16 | ), 17 | ) 18 | } 19 | 20 | In the main function we would run the processor like this: 21 | func main(){ 22 | proc := createProcessor([]string{"broker1:9092"}) 23 | proc.Run(ctx.Background()) 24 | } 25 | And in the unit test something like: 26 | func TestProcessor(t *testing.T){ 27 | // create tester 28 | tester := tester.New(t) 29 | // create the processor 30 | proc := createProcessor(nil, goka.WithTester(tester)) 31 | 32 | // .. do extra initialization if necessary 33 | 34 | go proc.Run(ctx.Background()) 35 | 36 | // execute the actual test 37 | tester.Consume("input-topic", "key", "value") 38 | 39 | value := tester.TableValue("group-table", "key") 40 | if value != expected{ 41 | t.Fatalf("got unexpected table value") 42 | } 43 | } 44 | 45 | See https://github.com/lovoo/goka/tree/master/examples/testing for a full example 46 | */ 47 | package tester 48 | -------------------------------------------------------------------------------- /tester/producer.go: -------------------------------------------------------------------------------- 1 | package tester 2 | 3 | import ( 4 | "github.com/lovoo/goka" 5 | ) 6 | 7 | // emitHandler abstracts a function that allows to overwrite kafkamock's Emit function to 8 | // simulate producer errors 9 | type emitHandler func(topic string, key string, value []byte, options ...EmitOption) *goka.Promise 10 | 11 | type producerMock struct { 12 | emitter emitHandler 13 | } 14 | 15 | func newProducerMock(emitter emitHandler) *producerMock { 16 | return &producerMock{ 17 | emitter: emitter, 18 | } 19 | } 20 | 21 | // Emit emits messages to arbitrary topics. 22 | // The mock simply forwards the emit to the KafkaMock which takes care of queueing calls 23 | // to handled topics or putting the emitted messages in the emitted-messages-list 24 | func (p *producerMock) EmitWithHeaders(topic string, key string, value []byte, headers goka.Headers) *goka.Promise { 25 | return p.emitter(topic, key, value, WithHeaders(headers)) 26 | } 27 | 28 | // Emit emits messages to arbitrary topics. 29 | // The mock simply forwards the emit to the KafkaMock which takes care of queueing calls 30 | // to handled topics or putting the emitted messages in the emitted-messages-list 31 | func (p *producerMock) Emit(topic string, key string, value []byte) *goka.Promise { 32 | return p.emitter(topic, key, value) 33 | } 34 | 35 | // Close closes the producer mock 36 | // No action required in the mock. 37 | func (p *producerMock) Close() error { 38 | logger.Printf("Closing producer mock") 39 | return nil 40 | } 41 | 42 | // flushingProducer wraps the producer and 43 | // waits for all consumers after the Emit. 44 | type flushingProducer struct { 45 | tester *Tester 46 | producer goka.Producer 47 | } 48 | 49 | // Emit using the underlying producer 50 | func (e *flushingProducer) EmitWithHeaders(topic string, key string, value []byte, headers goka.Headers) *goka.Promise { 51 | prom := e.producer.EmitWithHeaders(topic, key, value, headers) 52 | e.tester.waitForClients() 53 | return prom 54 | } 55 | 56 | // Emit using the underlying producer 57 | func (e *flushingProducer) Emit(topic string, key string, value []byte) *goka.Promise { 58 | prom := e.producer.Emit(topic, key, value) 59 | e.tester.waitForClients() 60 | return prom 61 | } 62 | 63 | // Close using the underlying producer 64 | func (e *flushingProducer) Close() error { 65 | return e.producer.Close() 66 | } 67 | -------------------------------------------------------------------------------- /tester/queue.go: -------------------------------------------------------------------------------- 1 | package tester 2 | 3 | import ( 4 | "sync" 5 | 6 | "github.com/lovoo/goka" 7 | ) 8 | 9 | type message struct { 10 | offset int64 11 | key string 12 | value []byte 13 | headers goka.Headers 14 | } 15 | 16 | type queue struct { 17 | sync.Mutex 18 | topic string 19 | messages []*message 20 | hwm int64 21 | } 22 | 23 | func newQueue(topic string) *queue { 24 | 25 | return &queue{ 26 | topic: topic, 27 | } 28 | } 29 | 30 | func (q *queue) Hwm() int64 { 31 | q.Lock() 32 | defer q.Unlock() 33 | 34 | hwm := q.hwm 35 | return hwm 36 | } 37 | 38 | func (q *queue) push(key string, value []byte, headers goka.Headers) int64 { 39 | q.Lock() 40 | defer q.Unlock() 41 | offset := q.hwm 42 | q.messages = append(q.messages, &message{ 43 | offset: offset, 44 | key: key, 45 | value: value, 46 | headers: headers, 47 | }) 48 | q.hwm++ 49 | return offset 50 | } 51 | 52 | func (q *queue) message(offset int) *message { 53 | q.Lock() 54 | defer q.Unlock() 55 | return q.messages[offset] 56 | } 57 | 58 | func (q *queue) messagesFromOffset(offset int64) []*message { 59 | q.Lock() 60 | defer q.Unlock() 61 | return q.messages[offset:] 62 | } 63 | 64 | func (q *queue) size() int { 65 | q.Lock() 66 | defer q.Unlock() 67 | return len(q.messages) 68 | } 69 | 70 | // QueueTracker tracks message offsets for each topic for convenient 71 | // 'expect message x to be in topic y' in unit tests 72 | type QueueTracker struct { 73 | t T 74 | topic string 75 | nextOffset int64 76 | tester *Tester 77 | } 78 | 79 | func newQueueTracker(tester *Tester, t T, topic string) *QueueTracker { 80 | return &QueueTracker{ 81 | t: t, 82 | topic: topic, 83 | tester: tester, 84 | nextOffset: tester.getOrCreateQueue(topic).hwm, 85 | } 86 | } 87 | 88 | // Next returns the next message since the last time this 89 | // function was called (or MoveToEnd) 90 | // It uses the known codec for the topic to decode the message 91 | func (mt *QueueTracker) Next() (string, interface{}, bool) { 92 | _, key, msg, hasNext := mt.NextWithHeaders() 93 | return key, msg, hasNext 94 | } 95 | 96 | // NextWithHeaders returns the next message since the last time this 97 | // function was called (or MoveToEnd). This includes headers 98 | // It uses the known codec for the topic to decode the message 99 | func (mt *QueueTracker) NextWithHeaders() (goka.Headers, string, interface{}, bool) { 100 | headers, key, msgRaw, hasNext := mt.NextRawWithHeaders() 101 | 102 | if !hasNext { 103 | return headers, key, msgRaw, hasNext 104 | } 105 | 106 | decoded, err := mt.tester.codecForTopic(mt.topic).Decode(msgRaw) 107 | if err != nil { 108 | mt.t.Fatalf("Error decoding message: %v", err) 109 | } 110 | return headers, key, decoded, true 111 | } 112 | 113 | // NextRaw returns the next message similar to Next(), but without the decoding 114 | func (mt *QueueTracker) NextRaw() (string, []byte, bool) { 115 | _, key, value, hasNext := mt.NextRawWithHeaders() 116 | return key, value, hasNext 117 | } 118 | 119 | // NextRawWithHeaders returns the next message similar to Next(), but without the decoding 120 | func (mt *QueueTracker) NextRawWithHeaders() (goka.Headers, string, []byte, bool) { 121 | q := mt.tester.getOrCreateQueue(mt.topic) 122 | if int(mt.nextOffset) >= q.size() { 123 | return nil, "", nil, false 124 | } 125 | msg := q.message(int(mt.nextOffset)) 126 | 127 | mt.nextOffset++ 128 | return msg.headers, msg.key, msg.value, true 129 | } 130 | 131 | // Seek moves the index pointer of the queue tracker to passed offset 132 | func (mt *QueueTracker) Seek(offset int64) { 133 | mt.nextOffset = offset 134 | } 135 | 136 | // Hwm returns the tracked queue's hwm value 137 | func (mt *QueueTracker) Hwm() int64 { 138 | return mt.tester.getOrCreateQueue(mt.topic).Hwm() 139 | } 140 | 141 | // NextOffset returns the tracker's next offset 142 | func (mt *QueueTracker) NextOffset() int64 { 143 | return mt.nextOffset 144 | } 145 | -------------------------------------------------------------------------------- /tester/tester_test.go: -------------------------------------------------------------------------------- 1 | package tester 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/lovoo/goka" 7 | "github.com/lovoo/goka/codec" 8 | "github.com/stretchr/testify/require" 9 | ) 10 | 11 | type failingT struct { 12 | failed bool 13 | } 14 | 15 | func (ft *failingT) Errorf(format string, args ...interface{}) { 16 | ft.failed = true 17 | } 18 | 19 | func (ft *failingT) Fatalf(format string, args ...interface{}) { 20 | ft.failed = true 21 | } 22 | 23 | func (ft *failingT) Fatal(a ...interface{}) { 24 | ft.failed = true 25 | } 26 | 27 | func TestTester(t *testing.T) { 28 | var ft failingT 29 | gkt := New(&ft) 30 | 31 | goka.NewProcessor(nil, 32 | goka.DefineGroup("", goka.Input("input", new(codec.Int64), func(ctx goka.Context, msg interface{}) {})), 33 | goka.WithTester(gkt)) 34 | 35 | // make sure that an invalid group graph fails the test 36 | require.True(t, ft.failed) 37 | } 38 | -------------------------------------------------------------------------------- /tester/topic_manager.go: -------------------------------------------------------------------------------- 1 | package tester 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/IBM/sarama" 7 | ) 8 | 9 | // MockTopicManager mimicks the behavior of the real topic manager 10 | type MockTopicManager struct { 11 | tt *Tester 12 | } 13 | 14 | // NewMockTopicManager creates a new topic manager mock 15 | func NewMockTopicManager(tt *Tester, defaultNumPartitions int, defaultReplFactor int) *MockTopicManager { 16 | return &MockTopicManager{ 17 | tt: tt, 18 | } 19 | } 20 | 21 | // EnsureTableExists ensures a table exists 22 | func (tm *MockTopicManager) EnsureTableExists(topic string, npar int) error { 23 | if npar != 1 { 24 | return fmt.Errorf("Mock only supports 1 partition") 25 | } 26 | tm.tt.getOrCreateQueue(topic) 27 | return nil 28 | } 29 | 30 | // EnsureStreamExists ensures a stream exists 31 | func (tm *MockTopicManager) EnsureStreamExists(topic string, npar int) error { 32 | tm.tt.getOrCreateQueue(topic) 33 | return nil 34 | } 35 | 36 | // EnsureTopicExists ensures a topic exists 37 | func (tm *MockTopicManager) EnsureTopicExists(topic string, npar, rfactor int, config map[string]string) error { 38 | tm.tt.getOrCreateQueue(topic) 39 | return nil 40 | } 41 | 42 | // Partitions returns all partitions for a topic 43 | func (tm *MockTopicManager) Partitions(topic string) ([]int32, error) { 44 | return []int32{0}, nil 45 | } 46 | 47 | // GetOffset returns the offset closest to the passed time (or exactly time, if the offsets are empty) 48 | func (tm *MockTopicManager) GetOffset(topicName string, partitionID int32, time int64) (int64, error) { 49 | topic := tm.tt.getOrCreateQueue(topicName) 50 | 51 | switch time { 52 | case sarama.OffsetOldest: 53 | return 0, nil 54 | case sarama.OffsetNewest: 55 | return topic.Hwm(), nil 56 | default: 57 | // always return from oldest, because the tester does not store 58 | // timestamps 59 | return 0, nil 60 | } 61 | } 62 | 63 | // Close has no action on the mock 64 | func (tm *MockTopicManager) Close() error { 65 | return nil 66 | } 67 | -------------------------------------------------------------------------------- /tools/tools.go: -------------------------------------------------------------------------------- 1 | // tools.go 2 | 3 | //go:build tools 4 | 5 | package tools 6 | 7 | import _ "go.uber.org/mock/mockgen" 8 | -------------------------------------------------------------------------------- /web/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/lovoo/goka/e3997d4ef01a22bb53b6d3728350b2e70ea44aba/web/README.md -------------------------------------------------------------------------------- /web/actions/action.go: -------------------------------------------------------------------------------- 1 | package actions 2 | 3 | import ( 4 | "context" 5 | "sync" 6 | "time" 7 | ) 8 | 9 | type action struct { 10 | sync.RWMutex 11 | ctx context.Context 12 | name string 13 | actor Actor 14 | cancel context.CancelFunc 15 | done chan struct{} 16 | started time.Time 17 | finished time.Time 18 | runErr error 19 | } 20 | 21 | func (a *action) Name() string { 22 | return a.name 23 | } 24 | 25 | // IsRunning returns whether the action is currently running. 26 | func (a *action) IsRunning() bool { 27 | a.RLock() 28 | defer a.RUnlock() 29 | 30 | // if there's a "done"-channel and it's not closed, the action is running 31 | if a.done != nil { 32 | select { 33 | case <-a.done: 34 | return false 35 | default: 36 | return true 37 | } 38 | } 39 | return false 40 | } 41 | 42 | func (a *action) Description() string { 43 | return a.actor.Description() 44 | } 45 | 46 | // StartTime returns a rfc3339 format of the start time or "not started" if it was not started yet 47 | func (a *action) StartTime() string { 48 | a.RLock() 49 | defer a.RUnlock() 50 | if a.started.IsZero() { 51 | return "not started" 52 | } 53 | return a.started.Format(time.RFC3339) 54 | } 55 | 56 | // FinishedTime returns a rfc3339 format of the finish time or "not finished" if it was not finished yet. 57 | func (a *action) FinishedTime() string { 58 | a.RLock() 59 | defer a.RUnlock() 60 | if a.finished.IsZero() { 61 | return "not finished" 62 | } 63 | return a.finished.Format(time.RFC3339) 64 | } 65 | 66 | // Error returns the error of the last invocation 67 | func (a *action) Error() error { 68 | 69 | a.RLock() 70 | defer a.RUnlock() 71 | return a.runErr 72 | } 73 | 74 | // Start starts the action in a separate goroutine. 75 | // If Start is called while the action is running, `Stop` will be called first. 76 | func (a *action) Start(value string) { 77 | // stop the action (noop if it's not running) 78 | a.Stop() 79 | 80 | a.Lock() 81 | defer a.Unlock() 82 | 83 | a.started = time.Now() 84 | a.finished = time.Time{} 85 | ctx, cancel := context.WithCancel(context.Background()) 86 | a.ctx, a.cancel = ctx, cancel 87 | done := make(chan struct{}) 88 | a.done = done 89 | go func() { 90 | defer cancel() 91 | err := a.actor.RunAction(ctx, value) 92 | defer close(done) 93 | 94 | a.Lock() 95 | a.finished = time.Now() 96 | a.runErr = err 97 | a.Unlock() 98 | 99 | }() 100 | } 101 | 102 | // Stop stops the action. It waits for the action to be completed 103 | func (a *action) Stop() { 104 | a.RLock() 105 | done := a.done 106 | if a.cancel != nil { 107 | a.cancel() 108 | } 109 | a.RUnlock() 110 | 111 | // if there was something to wait on, let's wait. 112 | if done != nil { 113 | <-done 114 | } 115 | } 116 | -------------------------------------------------------------------------------- /web/actions/action_test.go: -------------------------------------------------------------------------------- 1 | package actions 2 | 3 | import ( 4 | "context" 5 | "log" 6 | "testing" 7 | "time" 8 | 9 | "github.com/stretchr/testify/require" 10 | ) 11 | 12 | var actionRuntime = 10 * time.Millisecond 13 | 14 | func TestActionStart(t *testing.T) { 15 | var run int64 16 | actor := FuncActor("sleep", func(ctx context.Context, value string) error { 17 | select { 18 | case <-ctx.Done(): 19 | log.Printf("ctx done") 20 | return ctx.Err() 21 | case <-time.After(actionRuntime): 22 | run++ 23 | } 24 | return nil 25 | }) 26 | 27 | t.Run("no-run", func(t *testing.T) { 28 | a := &action{ 29 | name: "test", 30 | actor: actor, 31 | } 32 | 33 | require.Equal(t, "test", a.Name()) 34 | require.Equal(t, "sleep", a.Description()) 35 | require.True(t, a.Error() == nil) 36 | require.False(t, a.IsRunning()) 37 | require.Equal(t, "not started", a.StartTime()) 38 | require.Equal(t, "not finished", a.FinishedTime()) 39 | }) 40 | 41 | t.Run("stop-only", func(t *testing.T) { 42 | a := &action{ 43 | name: "test", 44 | actor: actor, 45 | } 46 | 47 | a.Stop() 48 | }) 49 | 50 | t.Run("start-stop", func(t *testing.T) { 51 | run = 0 52 | a := &action{ 53 | name: "test", 54 | actor: actor, 55 | } 56 | 57 | // start and check it's running 58 | a.Start("") 59 | require.True(t, a.IsRunning()) 60 | require.NotEqual(t, a.StartTime(), "not started") 61 | require.Equal(t, "not finished", a.FinishedTime()) 62 | 63 | a.Stop() 64 | require.False(t, a.IsRunning()) 65 | require.NotEqual(t, a.StartTime(), "not started") 66 | // it's finished 67 | require.NotEqual(t, a.FinishedTime(), "not finished") 68 | require.True(t, a.Error() != nil) 69 | require.Equal(t, context.Canceled.Error(), a.Error().Error()) 70 | require.Equal(t, int64(0), run) 71 | }) 72 | 73 | t.Run("start-finish", func(t *testing.T) { 74 | run = 0 75 | a := &action{ 76 | name: "test", 77 | actor: actor, 78 | } 79 | 80 | // start and check it's running 81 | a.Start("") 82 | time.Sleep(actionRuntime * 2) 83 | require.False(t, a.IsRunning()) 84 | require.NotEqual(t, a.StartTime(), "not started") 85 | require.NotEqual(t, a.FinishedTime(), "not finished") 86 | require.True(t, a.Error() == nil) 87 | require.Equal(t, int64(1), run) 88 | }) 89 | 90 | t.Run("start-restart-finish", func(t *testing.T) { 91 | run = 0 92 | a := &action{ 93 | name: "test", 94 | actor: actor, 95 | } 96 | 97 | // start + stop immediately 98 | a.Start("") 99 | a.Stop() 100 | 101 | // start and keep it running 102 | a.Start("") 103 | time.Sleep(actionRuntime * 2) 104 | 105 | a.Start("") 106 | time.Sleep(actionRuntime * 2) 107 | require.False(t, a.IsRunning()) 108 | require.NotEqual(t, a.StartTime(), "not started") 109 | require.NotEqual(t, a.FinishedTime(), "not finished") 110 | require.True(t, a.Error() == nil) 111 | require.Equal(t, int64(2), run) 112 | }) 113 | } 114 | -------------------------------------------------------------------------------- /web/actions/actions.go: -------------------------------------------------------------------------------- 1 | package actions 2 | 3 | import "context" 4 | 5 | type funcActor struct { 6 | description string 7 | actor func(ctx context.Context, value string) error 8 | } 9 | 10 | func FuncActor(description string, actor func(ctx context.Context, value string) error) Actor { 11 | return &funcActor{ 12 | description: description, 13 | actor: actor, 14 | } 15 | } 16 | 17 | func (fa *funcActor) RunAction(ctx context.Context, value string) error { 18 | return fa.actor(ctx, value) 19 | } 20 | func (fa *funcActor) Description() string { 21 | return fa.description 22 | } 23 | -------------------------------------------------------------------------------- /web/actions/option.go: -------------------------------------------------------------------------------- 1 | package actions 2 | 3 | import "github.com/lovoo/goka" 4 | 5 | // Option is a function that applies a configuration to the server. 6 | type Option func(s *Server) 7 | 8 | // WithLogger sets the logger to use. By default, it logs to the standard 9 | // library logger. 10 | func WithLogger(l goka.Logger) Option { 11 | return func(s *Server) { 12 | s.log = l 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /web/actions/server.go: -------------------------------------------------------------------------------- 1 | package actions 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "net/http" 7 | "sort" 8 | "strings" 9 | "sync" 10 | 11 | "github.com/lovoo/goka" 12 | "github.com/lovoo/goka/web/templates" 13 | 14 | "github.com/gorilla/mux" 15 | ) 16 | 17 | type Actor interface { 18 | RunAction(ctx context.Context, value string) error 19 | Description() string 20 | } 21 | 22 | // Server is a provides HTTP routes for querying the group table. 23 | type Server struct { 24 | log goka.Logger 25 | m sync.RWMutex 26 | 27 | basePath string 28 | loader templates.Loader 29 | actions map[string]*action 30 | } 31 | 32 | // NewServer creates a server with the given options. 33 | func NewServer(basePath string, router *mux.Router, opts ...Option) *Server { 34 | srv := &Server{ 35 | log: goka.DefaultLogger(), 36 | basePath: basePath, 37 | loader: &templates.EmbedLoader{}, 38 | actions: make(map[string]*action), 39 | } 40 | 41 | for _, opt := range opts { 42 | opt(srv) 43 | } 44 | 45 | sub := router.PathPrefix(basePath).Subrouter() 46 | sub.HandleFunc("/", srv.index) 47 | sub.HandleFunc("", srv.index) 48 | sub.HandleFunc("/start/{action:.*}", srv.startAction).Methods("POST") 49 | sub.HandleFunc("/stop/{action:.*}", srv.stopAction).Methods("POST") 50 | 51 | return srv 52 | } 53 | 54 | func (s *Server) startAction(w http.ResponseWriter, r *http.Request) { 55 | 56 | actionName := mux.Vars(r)["action"] 57 | action := s.actions[actionName] 58 | switch { 59 | case action == nil: 60 | s.redirect(w, r, fmt.Sprintf("Action '%s' not found", actionName)) 61 | 62 | case action.IsRunning(): 63 | s.redirect(w, r, "action already running.") 64 | default: 65 | action.Start(r.FormValue("value")) 66 | s.redirect(w, r, "") 67 | } 68 | } 69 | 70 | func (s *Server) stopAction(w http.ResponseWriter, r *http.Request) { 71 | actionName := mux.Vars(r)["action"] 72 | action := s.actions[actionName] 73 | switch { 74 | case action == nil: 75 | s.redirect(w, r, fmt.Sprintf("Action '%s' not found", actionName)) 76 | 77 | case !action.IsRunning(): 78 | s.redirect(w, r, "action is not running.") 79 | default: 80 | action.Stop() 81 | s.redirect(w, r, "") 82 | } 83 | } 84 | 85 | func (s *Server) redirect(w http.ResponseWriter, r *http.Request, errMessage string) { 86 | var path = s.basePath 87 | if errMessage != "" { 88 | path += "?error=" + errMessage 89 | } 90 | 91 | http.Redirect(w, r, path, http.StatusFound) 92 | } 93 | 94 | func (s *Server) BasePath() string { 95 | return s.basePath 96 | } 97 | 98 | func (s *Server) sortedActions() []*action { 99 | s.m.RLock() 100 | defer s.m.RUnlock() 101 | var actions []*action 102 | for _, action := range s.actions { 103 | actions = append(actions, action) 104 | } 105 | sort.Slice(actions, func(i, j int) bool { 106 | return strings.Compare(actions[i].name, actions[j].name) < 0 107 | }) 108 | return actions 109 | } 110 | 111 | // AttachSource attaches a new source to the query server. 112 | func (s *Server) AttachAction(name string, actor Actor) error { 113 | s.m.Lock() 114 | defer s.m.Unlock() 115 | if _, exists := s.actions[name]; exists { 116 | return fmt.Errorf("source with name '%s' is already attached", name) 117 | } 118 | s.actions[name] = &action{ 119 | name: name, 120 | actor: actor, 121 | } 122 | return nil 123 | } 124 | 125 | func (s *Server) AttachFuncAction(name string, description string, actor func(ctx context.Context, value string) error) error { 126 | return s.AttachAction(name, FuncActor(description, actor)) 127 | } 128 | 129 | func (s *Server) executeTemplate(w http.ResponseWriter, params map[string]interface{}) { 130 | tmpl, err := templates.LoadTemplates(append(templates.BaseTemplates, "web/templates/actions/index.go.html")...) 131 | if err != nil { 132 | http.Error(w, err.Error(), http.StatusInternalServerError) 133 | return 134 | } 135 | 136 | params["menu_title"] = "menu title" 137 | params["base_path"] = s.basePath 138 | 139 | if err := tmpl.Execute(w, params); err != nil { 140 | s.log.Printf("error executing query template: %v", err) 141 | } 142 | } 143 | 144 | func (s *Server) index(w http.ResponseWriter, r *http.Request) { 145 | params := map[string]interface{}{ 146 | "page_title": "Actions", 147 | "actions": s.sortedActions(), 148 | "error": r.URL.Query()["error"], 149 | } 150 | 151 | s.executeTemplate(w, params) 152 | } 153 | -------------------------------------------------------------------------------- /web/index/index.go: -------------------------------------------------------------------------------- 1 | package index 2 | 3 | import ( 4 | "net/http" 5 | "sync" 6 | 7 | "github.com/gorilla/mux" 8 | "github.com/lovoo/goka" 9 | "github.com/lovoo/goka/web/templates" 10 | ) 11 | 12 | var baseTemplates = append(templates.BaseTemplates) // "web/templates/monitor/menu.go.html" 13 | 14 | type component struct { 15 | Name string 16 | BasePath string 17 | } 18 | 19 | type Server struct { 20 | log goka.Logger 21 | m sync.RWMutex 22 | 23 | basePath string 24 | components []*component 25 | } 26 | 27 | type ComponentPathProvider interface { 28 | BasePath() string 29 | } 30 | 31 | func NewServer(basePath string, router *mux.Router) *Server { 32 | srv := &Server{ 33 | log: goka.DefaultLogger(), 34 | basePath: basePath, 35 | } 36 | 37 | sub := router.PathPrefix(basePath).Subrouter() 38 | sub.HandleFunc("/", srv.index) 39 | 40 | return srv 41 | } 42 | 43 | func (s *Server) AddComponent(pathProvider ComponentPathProvider, name string) { 44 | s.components = append(s.components, &component{ 45 | Name: name, 46 | BasePath: pathProvider.BasePath(), 47 | }) 48 | } 49 | 50 | func (s *Server) index(w http.ResponseWriter, r *http.Request) { 51 | tmpl, err := templates.LoadTemplates(append(baseTemplates, "web/templates/index/index.go.html")...) 52 | if err != nil { 53 | http.Error(w, err.Error(), http.StatusInternalServerError) 54 | return 55 | } 56 | 57 | params := map[string]interface{}{ 58 | "base_path": s.basePath, 59 | "components": s.components, 60 | } 61 | 62 | if err := tmpl.Execute(w, params); err != nil { 63 | s.log.Printf("error rendering index template: %v", err) 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /web/monitor/option.go: -------------------------------------------------------------------------------- 1 | package monitor 2 | 3 | import "github.com/lovoo/goka" 4 | 5 | // Option is a function that applies a configuration to the server. 6 | type Option func(s *Server) 7 | 8 | // WithLogger sets the logger to use. By default, it logs to standard out. 9 | func WithLogger(l goka.Logger) Option { 10 | return func(s *Server) { 11 | s.log = l 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /web/query/option.go: -------------------------------------------------------------------------------- 1 | package query 2 | 3 | import "github.com/lovoo/goka" 4 | 5 | // Option is a function that applies a configuration to the server. 6 | type Option func(s *Server) 7 | 8 | // WithHumanizer sets the humanizer to use for the queried values. By default, 9 | // the value will be printed out as JSON. 10 | func WithHumanizer(h Humanizer) Option { 11 | return func(s *Server) { 12 | s.humanizer = h 13 | } 14 | } 15 | 16 | // WithLogger sets the logger to use. By default, it logs to the standard 17 | // library logger. 18 | func WithLogger(l goka.Logger) Option { 19 | return func(s *Server) { 20 | s.log = l 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /web/templates/actions/index.go.html: -------------------------------------------------------------------------------- 1 | {{define "content"}} 2 |
3 |
4 |
5 |

Actions

6 | {{if .error}} 7 | 11 | {{end}} {{if .actions}} 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | {{$base_path := .base_path}} {{range $index, $action := .actions}} 23 | 24 | 30 | 31 | 32 | 35 | 50 | 51 | 52 | 53 | {{end}} 54 | 55 | {{else}} 56 |
No actions defined.
57 | {{end}} 58 | 59 | 60 | 61 | {{end}} -------------------------------------------------------------------------------- /web/templates/common/base.go.html: -------------------------------------------------------------------------------- 1 | {{define "base"}} 2 | 3 | {{template "head" .}} 4 | 5 | 6 | {{template "menu" .}} 7 | {{template "content" .}} 8 | 9 | 10 | {{end}} 11 | -------------------------------------------------------------------------------- /web/templates/common/head.go.html: -------------------------------------------------------------------------------- 1 | {{define "head"}} 2 | 3 | 4 | 5 | 6 | {{.page_title}} 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | {{end}} -------------------------------------------------------------------------------- /web/templates/common/menu.go.html: -------------------------------------------------------------------------------- 1 | {{define "menu"}} 2 | 14 | {{ end }} -------------------------------------------------------------------------------- /web/templates/index/index.go.html: -------------------------------------------------------------------------------- 1 | {{define "content"}} 2 |
3 |
4 |
5 | {{range $index, $component := .components}} 6 |
7 |
8 |
9 |

{{$component.Name}} 10 |

11 | 12 |
13 |
14 |
15 | {{end}} 16 |
17 |
18 |
19 | {{end}} 20 | -------------------------------------------------------------------------------- /web/templates/loader.go: -------------------------------------------------------------------------------- 1 | package templates 2 | 3 | import ( 4 | "embed" 5 | "fmt" 6 | "path/filepath" 7 | ) 8 | 9 | var ( 10 | // DefaultLoader is the default loader to be used with LoadTemplates. 11 | DefaultLoader = &EmbedLoader{} 12 | 13 | // templateFiles embeds all the files in the templates directory. 14 | //go:embed * 15 | templateFiles embed.FS 16 | ) 17 | 18 | // Loader is an interface for a template loader. 19 | type Loader interface { 20 | LoadTemplates(filename string) (string, error) 21 | } 22 | 23 | // EmbedLoader loads templates from the templates directory. 24 | type EmbedLoader struct{} 25 | 26 | // NewEmbedLoader creates a new EmbedLoader. 27 | func NewEmbedLoader() *EmbedLoader { 28 | return &EmbedLoader{} 29 | } 30 | 31 | // LoadTemplates reads and returns the file's content. 32 | func (l *EmbedLoader) LoadTemplates(filename string) (string, error) { 33 | directory := filepath.Base(filepath.Dir(filename)) 34 | file := filepath.Base(filename) 35 | fileBytes, err := templateFiles.ReadFile(directory + "/" + file) 36 | if err != nil { 37 | return "", fmt.Errorf("error reading and returning contents of embedded file: %v", err) 38 | } 39 | return string(fileBytes), nil 40 | } 41 | -------------------------------------------------------------------------------- /web/templates/monitor/index.go.html: -------------------------------------------------------------------------------- 1 | {{define "content"}} 2 |
3 |
4 |
5 |

Processors

6 | {{$base_path := .base_path}} 7 | {{range $index, $processor := .processors}} 8 |
9 |
10 | 14 |
15 | {{ with $graph := $processor.Graph }} 16 |
    17 | {{range $edgeIdx, $edge := $graph.InputStreams}} 18 |
  • INPUT: {{$edge.Topic}} → {{printf "%T" $edge.Codec}}
  • 19 | {{end}} 20 | {{range $edgeIdx, $edge := $graph.JointTables}} 21 |
  • JOINT: {{$edge.Topic}} → {{printf "%T" $edge.Codec}}
  • 22 | {{end}} 23 | {{range $edgeIdx, $edge := $graph.LookupTables}} 24 |
  • LOOKUP: {{$edge.Topic}} → {{printf "%T" $edge.Codec}}
  • 25 | {{end}} 26 | {{with $groupTable := $graph.GroupTable}} 27 |
  • GROUPTABLE: {{$groupTable.Topic}} → {{printf "%T" $groupTable.Codec}}
  • 28 | {{end}} 29 | {{with $loop := $graph.LoopStream}} 30 |
  • LOOP: {{$loop.Topic}} → {{printf "%T" $loop.Codec}}
  • 31 | {{end}} 32 | {{range $edgeIdx, $edge := $graph.OutputStreams}} 33 |
  • OUTPUT: {{$edge.Topic}} → {{printf "%T" $edge.Codec}}
  • 34 | {{end}} 35 |
36 | {{ end }} 37 |
38 |
39 |
40 | {{end}} 41 |
42 |
43 | 44 |
45 |
46 |

Views

47 | {{$base_path := .base_path}} 48 | {{range $index, $view := .views}} 49 |
50 |
51 |
52 |

{{$view.Topic}} 53 |

54 |
55 |
56 |
57 |
58 |
59 | {{end}} 60 |
61 |
62 |
63 | {{end}} 64 | -------------------------------------------------------------------------------- /web/templates/monitor/menu.go.html: -------------------------------------------------------------------------------- 1 | {{define "menu_content"}} 2 | {{range $index, $proc := .processors}} 3 |
  • {{$proc.Graph.Group}} (processor)
  • 4 | {{end}} 5 | {{range $index, $proc := .views}} 6 |
  • {{$proc.Topic}} (view)
  • 7 | {{end}} 8 | {{end}} 9 | -------------------------------------------------------------------------------- /web/templates/query/index.go.html: -------------------------------------------------------------------------------- 1 | {{define "content"}} 2 |
    3 |
    4 |
    5 |
    6 |

    Query Keys

    7 | {{if .error}} 8 | 12 | {{end}} 13 | {{if .warning}} 14 | 18 | {{end}} 19 | {{if .sources}} 20 |
    21 |
    22 |
    23 | 24 | 30 |
    31 | 32 |
    33 | 34 |
    35 |
    36 | 37 | {{else}} 38 | 39 | {{end}} 40 |
    41 |
    42 | {{if .value}} 43 |
    44 |
    45 |
    46 |

    {{.key}}

    47 |
    48 |
    49 |
    {{.value}}
    50 |
    51 |
    52 | {{end}} 53 |
    54 |
    55 | {{end}} 56 | -------------------------------------------------------------------------------- /web/templates/utils.go: -------------------------------------------------------------------------------- 1 | package templates 2 | 3 | import ( 4 | "fmt" 5 | "html/template" 6 | ) 7 | 8 | // BaseTemplates represents all partial templates used to build the base of the 9 | // site. 10 | var BaseTemplates = []string{ 11 | "web/templates/common/base.go.html", 12 | "web/templates/common/head.go.html", 13 | "web/templates/common/menu.go.html", 14 | } 15 | 16 | // LoadTemplates loads and associates the given templates. 17 | func LoadTemplates(filenames ...string) (*template.Template, error) { 18 | files := make(map[string]string) 19 | 20 | for _, name := range filenames { 21 | content, err := DefaultLoader.LoadTemplates(name) 22 | if err != nil { 23 | return nil, fmt.Errorf("error retrieving %s: %v", name, err) 24 | } 25 | files[name] = content 26 | } 27 | 28 | tmpl := template.New("base") 29 | for name, content := range files { 30 | if _, err := tmpl.Parse(content); err != nil { 31 | return nil, fmt.Errorf("error parsing %s: %v", name, err) 32 | } 33 | } 34 | 35 | return tmpl, nil 36 | } 37 | --------------------------------------------------------------------------------
    ActionStatusStarted/Finishederror 
    25 | {{$action.Name}} 26 | 27 |
    28 | {{$action.Description}} 29 |
    {{if $action.IsRunning}}running{{else}}not running{{end}} Started: {{$action.StartTime}}
    Finished: {{$action.FinishedTime}}
    33 | {{if $action.Error}} {{$action.Error.Error}} {{end}} 34 | 36 | {{if not $action.IsRunning}} 37 |
    38 |
    39 | 40 | 41 |
    42 | 43 |
    44 | {{else}} 45 |
    46 | 47 |
    48 | {{end}} 49 |