├── sh ├── .gitignore ├── build-librdkafka.sh ├── init-outbox.sh └── soak.sh ├── .gitignore ├── metric ├── metric.go ├── meter_test.go └── meter.go ├── examples ├── reaper_secure.yaml └── reaper.yaml ├── .travis.yml ├── go.mod ├── Makefile ├── battery_test.go ├── neli.go ├── event_test.go ├── LICENSE ├── battery.go ├── cmd ├── goharvest_example │ └── example_main.go ├── reaper │ └── reaper_main.go └── pump │ └── pump_main.go ├── db.go ├── db_mock_test.go ├── stasher ├── stasher_doc_test.go ├── stasher.go └── statsher_test.go ├── event.go ├── int ├── faulty_kafka_test.go └── harvest_int_test.go ├── kafka.go ├── kafka_mock_test.go ├── postgres.go ├── goharvest_doc_test.go ├── config_test.go ├── config.go ├── postgres_test.go ├── go.sum ├── harvest.go ├── README.md └── harvest_test.go /sh/.gitignore: -------------------------------------------------------------------------------- 1 | librdkafka 2 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .vscode 2 | .DS_Store 3 | *.cer 4 | *.pem 5 | /bin 6 | /log* 7 | -------------------------------------------------------------------------------- /metric/metric.go: -------------------------------------------------------------------------------- 1 | // Package metric contains data structures for working with metrics. 2 | package metric 3 | -------------------------------------------------------------------------------- /sh/build-librdkafka.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | cd $(dirname $0) 4 | 5 | set -e 6 | 7 | if [ -d librdkafka ]; then 8 | cd librdkafka 9 | git pull 10 | cd .. 11 | else 12 | git clone https://github.com/edenhill/librdkafka.git 13 | fi 14 | 15 | cd librdkafka 16 | ./configure --prefix /usr 17 | make 18 | sudo make install 19 | rm -rf librdkafka 20 | -------------------------------------------------------------------------------- /sh/init-outbox.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | cat </dev/null | awk -v RS= -F: '/^# File/,/^# Finished Make data base/ {if ($$1 !~ "^[#.]") {print $$1}}' | sort | egrep -v -e '^[^[:alnum:]]' -e '^$@$$' 31 | 32 | FORCE: 33 | -------------------------------------------------------------------------------- /battery_test.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/assert" 7 | ) 8 | 9 | func TestEnqueue_concurrencyOf1(t *testing.T) { 10 | enqueued := make(chan OutboxRecord) 11 | b := newConcurrentBattery(1, 0, func(records chan OutboxRecord) { 12 | for rec := range records { 13 | enqueued <- rec 14 | } 15 | }) 16 | defer b.shutdown() 17 | 18 | rec := OutboxRecord{} 19 | assert.True(t, b.enqueue(rec)) 20 | assert.Equal(t, rec, <-enqueued) 21 | } 22 | 23 | func TestEnqueue_concurrencyOf2(t *testing.T) { 24 | enqueued := make(chan OutboxRecord) 25 | b := newConcurrentBattery(2, 0, func(records chan OutboxRecord) { 26 | for rec := range records { 27 | enqueued <- rec 28 | } 29 | }) 30 | defer b.shutdown() 31 | 32 | rec := OutboxRecord{} 33 | assert.True(t, b.enqueue(rec)) 34 | assert.Equal(t, rec, <-enqueued) 35 | } 36 | 37 | func TestEnqueue_afterDone(t *testing.T) { 38 | b := newConcurrentBattery(2, 0, func(records chan OutboxRecord) {}) 39 | b.await() 40 | 41 | assert.False(t, b.enqueue(OutboxRecord{})) 42 | b.stop() 43 | } 44 | -------------------------------------------------------------------------------- /neli.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import "github.com/obsidiandynamics/goneli" 4 | 5 | // NeliProvider is a factory for creating Neli instances. 6 | type NeliProvider func(config goneli.Config, barrier goneli.Barrier) (goneli.Neli, error) 7 | 8 | // StandardNeliProvider returns a factory for creating a conventional Neli instance, backed by the real client API. 9 | func StandardNeliProvider() NeliProvider { 10 | return func(config goneli.Config, barrier goneli.Barrier) (goneli.Neli, error) { 11 | return goneli.New(config, barrier) 12 | } 13 | } 14 | 15 | func configToNeli(hConfigMap KafkaConfigMap) goneli.KafkaConfigMap { 16 | return map[string]interface{}(hConfigMap) 17 | } 18 | 19 | func configToHarvest(nConfigMap goneli.KafkaConfigMap) KafkaConfigMap { 20 | return map[string]interface{}(nConfigMap) 21 | } 22 | 23 | func convertKafkaConsumerProvider(hProvider KafkaConsumerProvider) goneli.KafkaConsumerProvider { 24 | return func(conf *goneli.KafkaConfigMap) (goneli.KafkaConsumer, error) { 25 | hCfg := configToHarvest(*conf) 26 | return hProvider(&hCfg) 27 | } 28 | } 29 | 30 | func convertKafkaProducerProvider(hProvider KafkaProducerProvider) goneli.KafkaProducerProvider { 31 | return func(conf *goneli.KafkaConfigMap) (goneli.KafkaProducer, error) { 32 | hCfg := configToHarvest(*conf) 33 | return hProvider(&hCfg) 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /event_test.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/google/uuid" 7 | "github.com/obsidiandynamics/goharvest/metric" 8 | "github.com/stretchr/testify/assert" 9 | ) 10 | 11 | func TestLeaderAcquired_string(t *testing.T) { 12 | leaderID, _ := uuid.NewRandom() 13 | assert.Contains(t, LeaderAcquired{leaderID}.String(), "LeaderAcquired[") 14 | assert.Contains(t, LeaderAcquired{leaderID}.String(), leaderID.String()) 15 | } 16 | 17 | func TestLeaderAcquired_getter(t *testing.T) { 18 | leaderID, _ := uuid.NewRandom() 19 | e := LeaderAcquired{leaderID} 20 | assert.Equal(t, leaderID, e.LeaderID()) 21 | } 22 | 23 | func TestLeaderRefreshed_string(t *testing.T) { 24 | leaderID, _ := uuid.NewRandom() 25 | assert.Contains(t, LeaderRefreshed{leaderID}.String(), "LeaderRefreshed[") 26 | assert.Contains(t, LeaderRefreshed{leaderID}.String(), leaderID.String()) 27 | } 28 | 29 | func TestLeaderRefreshed_getter(t *testing.T) { 30 | leaderID, _ := uuid.NewRandom() 31 | e := LeaderRefreshed{leaderID} 32 | assert.Equal(t, leaderID, e.LeaderID()) 33 | } 34 | 35 | func TestLeaderRevoked_string(t *testing.T) { 36 | assert.Equal(t, "LeaderRevoked[]", LeaderRevoked{}.String()) 37 | } 38 | 39 | func TestLeaderFenced_string(t *testing.T) { 40 | assert.Equal(t, "LeaderFenced[]", LeaderFenced{}.String()) 41 | } 42 | 43 | func TestMeterStats_string(t *testing.T) { 44 | stats := metric.MeterStats{} 45 | assert.Contains(t, MeterRead{stats}.String(), "MeterRead[") 46 | assert.Contains(t, MeterRead{stats}.String(), stats.String()) 47 | } 48 | -------------------------------------------------------------------------------- /sh/soak.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | if [ "$SOAK_CMD" == "" ]; then 4 | echo "SOAK_CMD is not set" 5 | exit 1 6 | fi 7 | if [ "$SOAK_RUNS" == "" ]; then 8 | SOAK_RUNS=10 9 | fi 10 | if [ "$SOAK_INTERVAL" == "" ]; then 11 | SOAK_INTERVAL=0 12 | fi 13 | 14 | if [ "$SOAK_GITPULL" == "" ]; then 15 | SOAK_GITPULL=true 16 | fi 17 | 18 | GREEN='\033[0;32m' 19 | RED='\033[0;31m' 20 | YELLOW='\033[0;33m' 21 | CYAN='\033[0;36m' 22 | GREY='\033[0;90m' 23 | NC='\033[0m' 24 | 25 | echo -e "${GREY}SOAK_CMD: $SOAK_CMD${NC}" 26 | echo -e "${GREY}SOAK_RUNS: $SOAK_RUNS${NC}" 27 | echo -e "${GREY}SOAK_INTERVAL: $SOAK_INTERVAL${NC}" 28 | echo -e "${GREY}SOAK_GITPULL: $SOAK_GITPULL${NC}" 29 | 30 | cd $(dirname $0)/.. 31 | 32 | set -e 33 | cycle=1 34 | while [ true ]; do 35 | echo -e "${CYAN}=============================================================================================================${NC}" 36 | echo -e "${CYAN}Cycle ${cycle}${NC}" 37 | 38 | for run in $(seq 1 $SOAK_RUNS) 39 | do 40 | timestamp=$(date +%'F %H:%M:%S') 41 | echo -e "${GREEN}-------------------------------------------------------------------------------------------------------------${NC}" 42 | echo -e "${GREEN}${timestamp}: Starting run ${run}/${SOAK_RUNS}${NC}" 43 | echo -e "${GREEN}-------------------------------------------------------------------------------------------------------------${NC}" 44 | $SOAK_CMD 45 | 46 | sleep $SOAK_INTERVAL 47 | done 48 | 49 | if [ $SOAK_GITPULL == "true" ]; then 50 | git pull 51 | fi 52 | cycle=$(($cycle + 1)) 53 | done 54 | 55 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | BSD 3-Clause License 2 | 3 | Copyright (c) 2020, Obsidian Dynamics 4 | All rights reserved. 5 | 6 | Redistribution and use in source and binary forms, with or without 7 | modification, are permitted provided that the following conditions are met: 8 | 9 | 1. Redistributions of source code must retain the above copyright notice, this 10 | list of conditions and the following disclaimer. 11 | 12 | 2. Redistributions in binary form must reproduce the above copyright notice, 13 | this list of conditions and the following disclaimer in the documentation 14 | and/or other materials provided with the distribution. 15 | 16 | 3. Neither the name of the copyright holder nor the names of its 17 | contributors may be used to endorse or promote products derived from 18 | this software without specific prior written permission. 19 | 20 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 21 | AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 22 | IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE 23 | DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE 24 | FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL 25 | DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR 26 | SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 27 | CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, 28 | OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 29 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | -------------------------------------------------------------------------------- /battery.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "hash/fnv" 5 | ) 6 | 7 | type cell struct { 8 | records chan OutboxRecord 9 | done chan int 10 | } 11 | 12 | func (c cell) stop() { 13 | close(c.records) 14 | } 15 | 16 | func (c cell) await() { 17 | <-c.done 18 | } 19 | 20 | func (c cell) enqueue(rec OutboxRecord) bool { 21 | select { 22 | case <-c.done: 23 | return false 24 | case c.records <- rec: 25 | return true 26 | } 27 | } 28 | 29 | type cellHandler func(records chan OutboxRecord) 30 | 31 | func newCell(buffer int, handler cellHandler) cell { 32 | c := cell{ 33 | records: make(chan OutboxRecord), 34 | done: make(chan int), 35 | } 36 | go func() { 37 | defer close(c.done) 38 | handler(c.records) 39 | }() 40 | return c 41 | } 42 | 43 | type battery interface { 44 | stop() 45 | await() 46 | shutdown() 47 | enqueue(rec OutboxRecord) bool 48 | } 49 | 50 | type concurrentBattery []cell 51 | 52 | func (b *concurrentBattery) stop() { 53 | for _, c := range *b { 54 | c.stop() 55 | } 56 | } 57 | 58 | func (b *concurrentBattery) await() { 59 | for _, c := range *b { 60 | c.await() 61 | } 62 | } 63 | 64 | func (b *concurrentBattery) shutdown() { 65 | b.stop() 66 | b.await() 67 | } 68 | 69 | func (b *concurrentBattery) enqueue(rec OutboxRecord) bool { 70 | if length := len(*b); length > 1 { 71 | return (*b)[hash(rec.KafkaKey)%uint32(length)].enqueue(rec) 72 | } 73 | return (*b)[0].enqueue(rec) 74 | } 75 | 76 | func newConcurrentBattery(concurrency int, buffer int, handler cellHandler) *concurrentBattery { 77 | b := make(concurrentBattery, concurrency) 78 | for i := 0; i < concurrency; i++ { 79 | b[i] = newCell(buffer, handler) 80 | } 81 | return &b 82 | } 83 | 84 | func hash(str string) uint32 { 85 | algorithm := fnv.New32a() 86 | algorithm.Write([]byte(str)) 87 | return algorithm.Sum32() 88 | } 89 | -------------------------------------------------------------------------------- /cmd/goharvest_example/example_main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "database/sql" 5 | 6 | "github.com/obsidiandynamics/goharvest" 7 | "github.com/obsidiandynamics/libstdgo/scribe" 8 | scribelogrus "github.com/obsidiandynamics/libstdgo/scribe/logrus" 9 | "github.com/sirupsen/logrus" 10 | ) 11 | 12 | func main() { 13 | const dataSource = "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable" 14 | 15 | // Optional: Ensure the database table exists before we start harvesting. 16 | func() { 17 | db, err := sql.Open("postgres", dataSource) 18 | if err != nil { 19 | panic(err) 20 | } 21 | defer db.Close() 22 | 23 | _, err = db.Exec(` 24 | CREATE TABLE IF NOT EXISTS outbox ( 25 | id BIGSERIAL PRIMARY KEY, 26 | create_time TIMESTAMP WITH TIME ZONE NOT NULL, 27 | kafka_topic VARCHAR(249) NOT NULL, 28 | kafka_key VARCHAR(100) NOT NULL, -- pick your own key size 29 | kafka_value VARCHAR(10000), -- pick your own value size 30 | kafka_header_keys TEXT[] NOT NULL, 31 | kafka_header_values TEXT[] NOT NULL, 32 | leader_id UUID 33 | ) 34 | `) 35 | if err != nil { 36 | panic(err) 37 | } 38 | }() 39 | 40 | // Configure the harvester. It will use its own database connections under the hood. 41 | log := logrus.StandardLogger() 42 | log.SetLevel(logrus.DebugLevel) 43 | config := goharvest.Config{ 44 | BaseKafkaConfig: goharvest.KafkaConfigMap{ 45 | "bootstrap.servers": "localhost:9092", 46 | }, 47 | DataSource: dataSource, 48 | Scribe: scribe.New(scribelogrus.Bind()), 49 | } 50 | 51 | // Create a new harvester. 52 | harvest, err := goharvest.New(config) 53 | if err != nil { 54 | panic(err) 55 | } 56 | 57 | // Start it. 58 | err = harvest.Start() 59 | if err != nil { 60 | panic(err) 61 | } 62 | 63 | // Wait indefinitely for it to end. 64 | log.Fatal(harvest.Await()) 65 | } 66 | -------------------------------------------------------------------------------- /db.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "fmt" 5 | "time" 6 | 7 | "github.com/google/uuid" 8 | ) 9 | 10 | // KafkaHeader is a key-value tuple representing a single header entry. 11 | type KafkaHeader struct { 12 | Key string 13 | Value string 14 | } 15 | 16 | // String obtains a textual representation of a KafkaHeader. 17 | func (h KafkaHeader) String() string { 18 | return h.Key + ":" + h.Value 19 | } 20 | 21 | // KafkaHeaders is a slice of KafkaHeader tuples. 22 | type KafkaHeaders []KafkaHeader 23 | 24 | // OutboxRecord depicts a single entry in the outbox table. It can be used for both reading and writing operations. 25 | type OutboxRecord struct { 26 | ID int64 27 | CreateTime time.Time 28 | KafkaTopic string 29 | KafkaKey string 30 | KafkaValue *string 31 | KafkaHeaders KafkaHeaders 32 | LeaderID *uuid.UUID 33 | } 34 | 35 | // String is a convenience function that returns a pointer to the given str argument, for use with setting OutboxRecord.Value. 36 | func String(str string) *string { 37 | return &str 38 | } 39 | 40 | // String provides a textual representation of an OutboxRecord. 41 | func (rec OutboxRecord) String() string { 42 | return fmt.Sprint("OutboxRecord[ID=", rec.ID, 43 | ", CreateTime=", rec.CreateTime, 44 | ", KafkaTopic=", rec.KafkaTopic, 45 | ", KafkaKey=", rec.KafkaKey, 46 | ", KafkaValue=", rec.KafkaValue, 47 | ", KafkaHeaders=", rec.KafkaHeaders, 48 | ", LeaderID=", rec.LeaderID, "]") 49 | } 50 | 51 | // DatabaseBinding is an abstraction over the data access layer, allowing goharvest to use arbitrary database implementations. 52 | type DatabaseBinding interface { 53 | Mark(leaderID uuid.UUID, limit int) ([]OutboxRecord, error) 54 | Purge(id int64) (bool, error) 55 | Reset(id int64) (bool, error) 56 | Dispose() 57 | } 58 | 59 | // DatabaseBindingProvider is a factory for creating instances of a DatabaseBinding. 60 | type DatabaseBindingProvider func(dataSource string, outboxTable string) (DatabaseBinding, error) 61 | -------------------------------------------------------------------------------- /db_mock_test.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "github.com/google/uuid" 5 | "github.com/obsidiandynamics/libstdgo/concurrent" 6 | ) 7 | 8 | type dbMockFuncs struct { 9 | Mark func(m *dbMock, leaderID uuid.UUID, limit int) ([]OutboxRecord, error) 10 | Purge func(m *dbMock, id int64) (bool, error) 11 | Reset func(m *dbMock, id int64) (bool, error) 12 | Dispose func(m *dbMock) 13 | } 14 | 15 | type dbMockCounts struct { 16 | Mark, 17 | Purge, 18 | Reset, 19 | Dispose concurrent.AtomicCounter 20 | } 21 | 22 | type dbMock struct { 23 | markedRecords chan []OutboxRecord 24 | f dbMockFuncs 25 | c dbMockCounts 26 | } 27 | 28 | func (m *dbMock) Mark(leaderID uuid.UUID, limit int) ([]OutboxRecord, error) { 29 | defer m.c.Mark.Inc() 30 | return m.f.Mark(m, leaderID, limit) 31 | } 32 | 33 | func (m *dbMock) Purge(id int64) (bool, error) { 34 | defer m.c.Purge.Inc() 35 | return m.f.Purge(m, id) 36 | } 37 | 38 | func (m *dbMock) Reset(id int64) (bool, error) { 39 | defer m.c.Reset.Inc() 40 | return m.f.Reset(m, id) 41 | } 42 | 43 | func (m *dbMock) Dispose() { 44 | defer m.c.Dispose.Inc() 45 | m.f.Dispose(m) 46 | } 47 | 48 | func (m *dbMock) fillDefaults() { 49 | if m.markedRecords == nil { 50 | m.markedRecords = make(chan []OutboxRecord) 51 | } 52 | 53 | if m.f.Mark == nil { 54 | m.f.Mark = func(m *dbMock, leaderID uuid.UUID, limit int) ([]OutboxRecord, error) { 55 | select { 56 | case records := <-m.markedRecords: 57 | return records, nil 58 | default: 59 | return []OutboxRecord{}, nil 60 | } 61 | } 62 | } 63 | if m.f.Purge == nil { 64 | m.f.Purge = func(m *dbMock, id int64) (bool, error) { 65 | return true, nil 66 | } 67 | } 68 | if m.f.Reset == nil { 69 | m.f.Reset = func(m *dbMock, id int64) (bool, error) { 70 | return true, nil 71 | } 72 | } 73 | if m.f.Dispose == nil { 74 | m.f.Dispose = func(m *dbMock) {} 75 | } 76 | m.c.Mark = concurrent.NewAtomicCounter() 77 | m.c.Purge = concurrent.NewAtomicCounter() 78 | m.c.Reset = concurrent.NewAtomicCounter() 79 | m.c.Dispose = concurrent.NewAtomicCounter() 80 | } 81 | 82 | func mockDatabaseBindingProvider(m *dbMock) func(string, string) (DatabaseBinding, error) { 83 | return func(dataSource string, table string) (DatabaseBinding, error) { 84 | return m, nil 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /stasher/stasher_doc_test.go: -------------------------------------------------------------------------------- 1 | package stasher 2 | 3 | import ( 4 | "database/sql" 5 | "testing" 6 | 7 | "github.com/obsidiandynamics/goharvest" 8 | "github.com/obsidiandynamics/libstdgo/check" 9 | ) 10 | 11 | func Example() { 12 | db, err := sql.Open("postgres", "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable") 13 | if err != nil { 14 | panic(err) 15 | } 16 | defer db.Close() 17 | 18 | st := New("outbox") 19 | 20 | // Begin a transaction. 21 | tx, _ := db.Begin() 22 | defer tx.Rollback() 23 | 24 | // Update other database entities in transaction scope. 25 | // ... 26 | 27 | // Stash an outbox record for subsequent harvesting. 28 | err = st.Stash(tx, goharvest.OutboxRecord{ 29 | KafkaTopic: "my-app.topic", 30 | KafkaKey: "hello", 31 | KafkaValue: goharvest.String("world"), 32 | KafkaHeaders: goharvest.KafkaHeaders{ 33 | {Key: "applicationId", Value: "my-app"}, 34 | }, 35 | }) 36 | if err != nil { 37 | panic(err) 38 | } 39 | 40 | // Commit the transaction. 41 | tx.Commit() 42 | } 43 | 44 | func TestExample(t *testing.T) { 45 | check.RunTargetted(t, Example) 46 | } 47 | 48 | func Example_prepare() { 49 | db, err := sql.Open("postgres", "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable") 50 | if err != nil { 51 | panic(err) 52 | } 53 | defer db.Close() 54 | 55 | st := New("outbox") 56 | 57 | // Begin a transaction. 58 | tx, _ := db.Begin() 59 | defer tx.Rollback() 60 | 61 | // Update other database entities in transaction scope. 62 | // ... 63 | 64 | // Formulates a prepared statement that may be reused within the scope of the transaction. 65 | prestash, _ := st.Prepare(tx) 66 | 67 | // Publish a bunch of messages using the same prepared statement. 68 | for i := 0; i < 10; i++ { 69 | // Stash an outbox record for subsequent harvesting. 70 | err = prestash.Stash(goharvest.OutboxRecord{ 71 | KafkaTopic: "my-app.topic", 72 | KafkaKey: "hello", 73 | KafkaValue: goharvest.String("world"), 74 | KafkaHeaders: goharvest.KafkaHeaders{ 75 | {Key: "applicationId", Value: "my-app"}, 76 | }, 77 | }) 78 | if err != nil { 79 | panic(err) 80 | } 81 | } 82 | 83 | // Commit the transaction. 84 | tx.Commit() 85 | } 86 | 87 | func TestExample_prepare(t *testing.T) { 88 | check.RunTargetted(t, Example_prepare) 89 | } 90 | -------------------------------------------------------------------------------- /event.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/google/uuid" 7 | "github.com/obsidiandynamics/goharvest/metric" 8 | ) 9 | 10 | // EventHandler is a callback function for handling GoHarvest events. 11 | type EventHandler func(e Event) 12 | 13 | // Event encapsulates a GoHarvest event. 14 | type Event interface { 15 | fmt.Stringer 16 | } 17 | 18 | // LeaderAcquired is emitted upon successful acquisition of leader status. 19 | type LeaderAcquired struct { 20 | leaderID uuid.UUID 21 | } 22 | 23 | // String obtains a textual representation of the LeaderAcquired event. 24 | func (e LeaderAcquired) String() string { 25 | return fmt.Sprint("LeaderAcquired[leaderID=", e.leaderID, "]") 26 | } 27 | 28 | // LeaderID returns the local UUID of the elected leader. 29 | func (e LeaderAcquired) LeaderID() uuid.UUID { 30 | return e.leaderID 31 | } 32 | 33 | // LeaderRefreshed is emitted when a new leader ID is generated as a result of a remarking request. 34 | type LeaderRefreshed struct { 35 | leaderID uuid.UUID 36 | } 37 | 38 | // String obtains a textual representation of the LeaderRefreshed event. 39 | func (e LeaderRefreshed) String() string { 40 | return fmt.Sprint("LeaderRefreshed[leaderID=", e.leaderID, "]") 41 | } 42 | 43 | // LeaderID returns the local UUID of the elected leader. 44 | func (e LeaderRefreshed) LeaderID() uuid.UUID { 45 | return e.leaderID 46 | } 47 | 48 | // LeaderRevoked is emitted when the leader status has been revoked. 49 | type LeaderRevoked struct{} 50 | 51 | // String obtains a textual representation of the LeaderRevoked event. 52 | func (e LeaderRevoked) String() string { 53 | return fmt.Sprint("LeaderRevoked[]") 54 | } 55 | 56 | // LeaderFenced is emitted when the leader status has been revoked. 57 | type LeaderFenced struct{} 58 | 59 | // String obtains a textual representation of the LeaderFenced event. 60 | func (e LeaderFenced) String() string { 61 | return fmt.Sprint("LeaderFenced[]") 62 | } 63 | 64 | // MeterRead is emitted when the internal throughput Meter has been read. 65 | type MeterRead struct { 66 | stats metric.MeterStats 67 | } 68 | 69 | // String obtains a textual representation of the MeterRead event. 70 | func (e MeterRead) String() string { 71 | return fmt.Sprint("MeterRead[stats=", e.stats, "]") 72 | } 73 | 74 | // Stats embedded in the MeterRead event. 75 | func (e MeterRead) Stats() metric.MeterStats { 76 | return e.stats 77 | } 78 | -------------------------------------------------------------------------------- /cmd/reaper/reaper_main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "fmt" 6 | "io/ioutil" 7 | "os" 8 | 9 | "github.com/obsidiandynamics/goharvest" 10 | "github.com/obsidiandynamics/libstdgo/scribe" 11 | "gopkg.in/yaml.v2" 12 | 13 | scribelogrus "github.com/obsidiandynamics/libstdgo/scribe/logrus" 14 | logrus "github.com/sirupsen/logrus" 15 | ) 16 | 17 | func panicOnError(scr scribe.Scribe, err error) { 18 | if err != nil { 19 | scr.E()("Error: %v", err.Error()) 20 | panic(err) 21 | } 22 | } 23 | 24 | func main() { 25 | var configFile string 26 | flag.StringVar(&configFile, "f", "", "Configuration file (shorthand)") 27 | flag.StringVar(&configFile, "file", "", "Configuration file") 28 | flag.Parse() 29 | 30 | errorFunc := func(field string) { 31 | flag.PrintDefaults() 32 | panic(fmt.Errorf("required '-%s' has not been set", field)) 33 | } 34 | if configFile == "" { 35 | errorFunc("f") 36 | } 37 | 38 | lr := logrus.StandardLogger() 39 | lr.SetLevel(logrus.TraceLevel) 40 | scr := scribe.New(scribelogrus.Bind()) 41 | 42 | workDir, err := os.Getwd() 43 | panicOnError(scr, err) 44 | scr.I()("Starting GoHarvest Reaper") 45 | executable, err := os.Executable() 46 | panicOnError(scr, err) 47 | scr.I()("Executable: %s; working directory: %s", executable, workDir) 48 | 49 | cfgData, err := ioutil.ReadFile(configFile) 50 | panicOnError(scr, err) 51 | cfg, err := unmarshal(cfgData) 52 | panicOnError(scr, err) 53 | 54 | cfg.Harvest.Scribe = scr 55 | level, err := scribe.ParseLevelName(cfg.Logging.Level) 56 | panicOnError(scr, err) 57 | scr.SetEnabled(level.Level) 58 | 59 | h, err := goharvest.New(cfg.Harvest) 60 | panicOnError(scr, err) 61 | 62 | panicOnError(scr, h.Start()) 63 | panicOnError(scr, h.Await()) 64 | } 65 | 66 | type LoggingConfig struct { 67 | Level string `yaml:"level"` 68 | } 69 | 70 | func (l *LoggingConfig) setDefaults() { 71 | if l.Level == "" { 72 | l.Level = scribe.Levels[scribe.Debug].Name 73 | } 74 | } 75 | 76 | type ReaperConfig struct { 77 | Harvest goharvest.Config `yaml:"harvest"` 78 | Logging LoggingConfig `yaml:"logging"` 79 | } 80 | 81 | func (r *ReaperConfig) setDefaults() { 82 | r.Harvest.SetDefaults() 83 | r.Logging.setDefaults() 84 | } 85 | 86 | func unmarshal(in []byte) (ReaperConfig, error) { 87 | cfg := ReaperConfig{} 88 | err := yaml.UnmarshalStrict(in, &cfg) 89 | if err == nil { 90 | cfg.setDefaults() 91 | } 92 | return cfg, err 93 | } 94 | -------------------------------------------------------------------------------- /stasher/stasher.go: -------------------------------------------------------------------------------- 1 | // Package stasher is a helper for inserting records into an outbox table within transaction scope. 2 | package stasher 3 | 4 | import ( 5 | "database/sql" 6 | "fmt" 7 | 8 | "github.com/lib/pq" 9 | "github.com/obsidiandynamics/goharvest" 10 | ) 11 | 12 | // Stasher writes records into the outbox table. 13 | type Stasher interface { 14 | Stash(tx *sql.Tx, rec goharvest.OutboxRecord) error 15 | Prepare(tx *sql.Tx) (PreStash, error) 16 | } 17 | 18 | type stasher struct { 19 | query string 20 | } 21 | 22 | // New creates a new Stasher instance for the given outboxTable. 23 | func New(outboxTable string) Stasher { 24 | return &stasher{fmt.Sprintf(insertQueryTemplate, outboxTable)} 25 | } 26 | 27 | const insertQueryTemplate = ` 28 | -- insert query 29 | INSERT INTO %s (create_time, kafka_topic, kafka_key, kafka_value, kafka_header_keys, kafka_header_values) 30 | VALUES (NOW(), $1, $2, $3, $4, $5) 31 | ` 32 | 33 | // PreStash houses a prepared statement bound to the scope of a single transaction. 34 | type PreStash struct { 35 | stmt *sql.Stmt 36 | } 37 | 38 | // Prepare a statement for stashing records, where the latter is expected to be invoked multiple times from 39 | // a given transaction. 40 | func (s *stasher) Prepare(tx *sql.Tx) (PreStash, error) { 41 | stmt, err := tx.Prepare(s.query) 42 | return PreStash{stmt}, err 43 | } 44 | 45 | // Stash one record using the prepared statement. 46 | func (p PreStash) Stash(rec goharvest.OutboxRecord) error { 47 | headerKeys, headerValues := makeHeaders(rec) 48 | _, err := p.stmt.Exec(rec.KafkaTopic, rec.KafkaKey, rec.KafkaValue, pq.Array(headerKeys), pq.Array(headerValues)) 49 | return err 50 | } 51 | 52 | func makeHeaders(rec goharvest.OutboxRecord) ([]string, []string) { 53 | var headerKeys, headerValues []string 54 | if numHeaders := len(rec.KafkaHeaders); numHeaders > 0 { 55 | headerKeys = make([]string, numHeaders) 56 | headerValues = make([]string, numHeaders) 57 | for i, header := range rec.KafkaHeaders { 58 | headerKeys[i], headerValues[i] = header.Key, header.Value 59 | } 60 | } else { 61 | headerKeys, headerValues = []string{}, []string{} 62 | } 63 | return headerKeys, headerValues 64 | } 65 | 66 | // Stash one record within the given transaction scope. 67 | func (s *stasher) Stash(tx *sql.Tx, rec goharvest.OutboxRecord) error { 68 | headerKeys, headerValues := makeHeaders(rec) 69 | _, err := tx.Exec(s.query, rec.KafkaTopic, rec.KafkaKey, rec.KafkaValue, pq.Array(headerKeys), pq.Array(headerValues)) 70 | return err 71 | } 72 | -------------------------------------------------------------------------------- /int/faulty_kafka_test.go: -------------------------------------------------------------------------------- 1 | package int 2 | 3 | import ( 4 | "github.com/obsidiandynamics/goharvest" 5 | "github.com/obsidiandynamics/libstdgo/fault" 6 | "gopkg.in/confluentinc/confluent-kafka-go.v1/kafka" 7 | ) 8 | 9 | type ProducerFaultSpecs struct { 10 | OnProduce fault.Spec 11 | OnDelivery fault.Spec 12 | } 13 | 14 | func (specs ProducerFaultSpecs) build() producerFaults { 15 | return producerFaults{ 16 | onProduce: specs.OnProduce.Build(), 17 | onDelivery: specs.OnDelivery.Build(), 18 | } 19 | } 20 | 21 | func FaultyKafkaProducerProvider(realProvider goharvest.KafkaProducerProvider, specs ProducerFaultSpecs) goharvest.KafkaProducerProvider { 22 | return func(conf *goharvest.KafkaConfigMap) (goharvest.KafkaProducer, error) { 23 | real, err := realProvider(conf) 24 | if err != nil { 25 | return nil, err 26 | } 27 | return newFaultyProducer(real, specs.build()), nil 28 | } 29 | } 30 | 31 | type producerFaults struct { 32 | onProduce fault.Fault 33 | onDelivery fault.Fault 34 | } 35 | 36 | type faultyProducer struct { 37 | real goharvest.KafkaProducer 38 | faults producerFaults 39 | events chan kafka.Event 40 | } 41 | 42 | func newFaultyProducer(real goharvest.KafkaProducer, faults producerFaults) *faultyProducer { 43 | f := &faultyProducer{ 44 | real: real, 45 | faults: faults, 46 | events: make(chan kafka.Event), 47 | } 48 | 49 | go func() { 50 | defer close(f.events) 51 | 52 | for e := range real.Events() { 53 | switch ev := e.(type) { 54 | case *kafka.Message: 55 | if ev.TopicPartition.Error != nil { 56 | f.events <- e 57 | } else if err := f.faults.onDelivery.Try(); err != nil { 58 | rewrittenMessage := *ev 59 | rewrittenMessage.TopicPartition = kafka.TopicPartition{ 60 | Topic: ev.TopicPartition.Topic, 61 | Partition: ev.TopicPartition.Partition, 62 | Offset: ev.TopicPartition.Offset, 63 | Metadata: ev.TopicPartition.Metadata, 64 | Error: err, 65 | } 66 | f.events <- &rewrittenMessage 67 | } else { 68 | f.events <- e 69 | } 70 | default: 71 | f.events <- e 72 | } 73 | } 74 | }() 75 | 76 | return f 77 | } 78 | 79 | func (f *faultyProducer) Events() chan kafka.Event { 80 | return f.events 81 | } 82 | 83 | func (f *faultyProducer) Produce(msg *kafka.Message, deliveryChan chan kafka.Event) error { 84 | if err := f.faults.onProduce.Try(); err != nil { 85 | return err 86 | } 87 | return f.real.Produce(msg, deliveryChan) 88 | } 89 | 90 | func (f *faultyProducer) Close() { 91 | f.real.Close() 92 | } 93 | -------------------------------------------------------------------------------- /stasher/statsher_test.go: -------------------------------------------------------------------------------- 1 | package stasher 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/DATA-DOG/go-sqlmock" 7 | "github.com/lib/pq" 8 | "github.com/obsidiandynamics/goharvest" 9 | "github.com/stretchr/testify/require" 10 | ) 11 | 12 | const ( 13 | testTopic = "topic" 14 | testKey = "key" 15 | testValue = "value" 16 | testHeaderKey = "header-key" 17 | testHeaderValue = "header-value" 18 | testInsertQuery = "-- insert query" 19 | ) 20 | 21 | func TestStash_withHeaders(t *testing.T) { 22 | s := New("outbox") 23 | 24 | db, mock, err := sqlmock.New() 25 | require.Nil(t, err) 26 | 27 | mock.ExpectBegin() 28 | tx, err := db.Begin() 29 | require.Nil(t, err) 30 | 31 | mock.ExpectExec(testInsertQuery). 32 | WithArgs(testTopic, testKey, testValue, pq.Array([]string{testHeaderKey}), pq.Array([]string{testHeaderValue})). 33 | WillReturnResult(sqlmock.NewResult(-1, 1)) 34 | err = s.Stash(tx, goharvest.OutboxRecord{ 35 | KafkaTopic: testTopic, 36 | KafkaKey: testKey, 37 | KafkaValue: goharvest.String(testValue), 38 | KafkaHeaders: goharvest.KafkaHeaders{ 39 | {Key: testHeaderKey, Value: testHeaderValue}, 40 | }, 41 | }) 42 | require.Nil(t, err) 43 | 44 | require.Nil(t, mock.ExpectationsWereMet()) 45 | } 46 | 47 | func TestStash_withoutHeaders(t *testing.T) { 48 | s := New("outbox") 49 | 50 | db, mock, err := sqlmock.New() 51 | require.Nil(t, err) 52 | 53 | mock.ExpectBegin() 54 | tx, err := db.Begin() 55 | require.Nil(t, err) 56 | 57 | mock.ExpectExec(testInsertQuery). 58 | WithArgs(testTopic, testKey, testValue, pq.Array([]string{}), pq.Array([]string{})). 59 | WillReturnResult(sqlmock.NewResult(-1, 1)) 60 | err = s.Stash(tx, goharvest.OutboxRecord{ 61 | KafkaTopic: testTopic, 62 | KafkaKey: testKey, 63 | KafkaValue: goharvest.String(testValue), 64 | }) 65 | require.Nil(t, err) 66 | 67 | require.Nil(t, mock.ExpectationsWereMet()) 68 | } 69 | 70 | func TestStash_prepare(t *testing.T) { 71 | s := New("outbox") 72 | 73 | db, mock, err := sqlmock.New() 74 | require.Nil(t, err) 75 | 76 | mock.ExpectBegin() 77 | tx, err := db.Begin() 78 | require.Nil(t, err) 79 | 80 | mock.ExpectPrepare(testInsertQuery) 81 | prestash, err := s.Prepare(tx) 82 | require.Nil(t, err) 83 | require.NotNil(t, prestash) 84 | 85 | mock.ExpectExec(testInsertQuery). 86 | WithArgs(testTopic, testKey, testValue, pq.Array([]string{}), pq.Array([]string{})). 87 | WillReturnResult(sqlmock.NewResult(-1, 1)) 88 | err = prestash.Stash(goharvest.OutboxRecord{ 89 | KafkaTopic: testTopic, 90 | KafkaKey: testKey, 91 | KafkaValue: goharvest.String(testValue), 92 | }) 93 | require.Nil(t, err) 94 | 95 | require.Nil(t, mock.ExpectationsWereMet()) 96 | } 97 | -------------------------------------------------------------------------------- /metric/meter_test.go: -------------------------------------------------------------------------------- 1 | package metric 2 | 3 | import ( 4 | "testing" 5 | "time" 6 | 7 | "github.com/obsidiandynamics/libstdgo/check" 8 | "github.com/obsidiandynamics/libstdgo/scribe" 9 | "github.com/stretchr/testify/assert" 10 | "github.com/stretchr/testify/require" 11 | ) 12 | 13 | func wait(t *testing.T) check.Timesert { 14 | return check.Wait(t, 10*time.Second) 15 | } 16 | 17 | func TestMeterString(t *testing.T) { 18 | m := NewMeter("test-name", time.Second) 19 | str := m.String() 20 | require.Contains(t, str, "Meter[") 21 | require.Contains(t, str, m.name) 22 | } 23 | 24 | func TestMeterMaybeStats(t *testing.T) { 25 | m := NewMeter("test-name", time.Millisecond) 26 | m.Add(1) 27 | wait(t).UntilAsserted(func(t check.Tester) { 28 | s := m.MaybeStats() 29 | if assert.NotNil(t, s) { 30 | assert.Equal(t, "test-name", s.Name) 31 | assert.Equal(t, int64(1), s.TotalCount) 32 | assert.Equal(t, int64(1), s.IntervalCount) 33 | } 34 | }) 35 | 36 | m.Add(2) 37 | 38 | wait(t).UntilAsserted(func(t check.Tester) { 39 | s := m.MaybeStats() 40 | if assert.NotNil(t, s) { 41 | assert.Equal(t, "test-name", s.Name) 42 | assert.Equal(t, int64(3), s.TotalCount) 43 | assert.Equal(t, int64(2), s.IntervalCount) 44 | } 45 | }) 46 | 47 | m.Add(1) 48 | m.Reset() 49 | 50 | wait(t).UntilAsserted(func(t check.Tester) { 51 | s := m.MaybeStats() 52 | if assert.NotNil(t, s) { 53 | assert.Equal(t, "test-name", s.Name) 54 | assert.Equal(t, int64(0), s.TotalCount) 55 | assert.Equal(t, int64(0), s.IntervalCount) 56 | } 57 | }) 58 | } 59 | 60 | func TestMeterMaybeStatsCall(t *testing.T) { 61 | m := NewMeter("test-name", time.Millisecond) 62 | m.Add(1) 63 | wait(t).UntilAsserted(func(t check.Tester) { 64 | var statsPtr *MeterStats 65 | called := m.MaybeStatsCall(func(stats MeterStats) { 66 | statsPtr = &stats 67 | }) 68 | if assert.True(t, called) { 69 | assert.NotNil(t, statsPtr) 70 | assert.Equal(t, "test-name", statsPtr.Name) 71 | assert.Equal(t, int64(1), statsPtr.TotalCount) 72 | assert.Equal(t, int64(1), statsPtr.IntervalCount) 73 | } else { 74 | assert.Nil(t, statsPtr) 75 | } 76 | }) 77 | } 78 | 79 | func TestMeterMaybeStatsLog(t *testing.T) { 80 | m := NewMeter("test-name", time.Millisecond) 81 | m.Add(1) 82 | 83 | mockscribe := scribe.NewMock() 84 | scr := scribe.New(mockscribe.Factories()) 85 | wait(t).UntilAsserted(func(t check.Tester) { 86 | called := m.MaybeStatsLog(scr.I()) 87 | if assert.True(t, called) { 88 | mockscribe.Entries(). 89 | Having(scribe.LogLevel(scribe.Info)). 90 | Having(scribe.MessageContaining("test-name")). 91 | Assert(t, scribe.Count(1)) 92 | } else { 93 | mockscribe.Entries(). 94 | Assert(t, scribe.Count(0)) 95 | } 96 | }) 97 | } 98 | -------------------------------------------------------------------------------- /cmd/pump/pump_main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "database/sql" 5 | "flag" 6 | "fmt" 7 | "log" 8 | "math/rand" 9 | "strconv" 10 | "time" 11 | 12 | "github.com/obsidiandynamics/goharvest" 13 | "github.com/obsidiandynamics/goharvest/metric" 14 | "github.com/obsidiandynamics/goharvest/stasher" 15 | ) 16 | 17 | const recordsPerTxn = 20 18 | 19 | func main() { 20 | var keys, records, interval int 21 | var dataSource, outboxTable, kafkaTopic string 22 | var blank bool 23 | flag.IntVar(&keys, "keys", -1, "Number of unique keys") 24 | flag.IntVar(&records, "records", -1, "Number of records to generate") 25 | flag.IntVar(&interval, "interval", 0, "Write interval (in milliseconds") 26 | flag.StringVar(&dataSource, "ds", "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable", "Data source") 27 | flag.StringVar(&outboxTable, "outbox", "outbox", "Outbox table name") 28 | flag.StringVar(&kafkaTopic, "topic", "pump", "Kafka output topic name") 29 | flag.BoolVar(&blank, "blank", false, "Generate blank records (nil value)") 30 | flag.Parse() 31 | 32 | errorFunc := func(field string) { 33 | flag.PrintDefaults() 34 | panic(fmt.Errorf("required '-%s' has not been set", field)) 35 | } 36 | if keys == -1 { 37 | errorFunc("keys") 38 | } 39 | if records == -1 { 40 | errorFunc("records") 41 | } 42 | 43 | fmt.Printf("Starting stasher; keys: %d, records: %d, interval: %d ms\n", keys, records, interval) 44 | fmt.Printf(" Data source: %s\n", dataSource) 45 | fmt.Printf(" Outbox table name: %s\n", outboxTable) 46 | 47 | db, err := sql.Open("postgres", dataSource) 48 | if err != nil { 49 | panic(err) 50 | } 51 | defer db.Close() 52 | 53 | st := stasher.New(outboxTable) 54 | 55 | meter := metric.NewMeter("pump", 5*time.Second) 56 | 57 | var tx *sql.Tx 58 | var pre stasher.PreStash 59 | for i := 0; i < records; i++ { 60 | if i%recordsPerTxn == 0 { 61 | finaliseTx(tx) 62 | 63 | tx, err = db.Begin() 64 | if err != nil { 65 | panic(err) 66 | } 67 | pre, err = st.Prepare(tx) 68 | if err != nil { 69 | panic(err) 70 | } 71 | } 72 | 73 | rand := rand.Uint64() 74 | var value *string 75 | if !blank { 76 | value = goharvest.String(fmt.Sprintf("value-%x", rand)) 77 | } 78 | 79 | rec := goharvest.OutboxRecord{ 80 | KafkaTopic: kafkaTopic, 81 | KafkaKey: fmt.Sprintf("key-%x", rand%uint64(keys)), 82 | KafkaValue: value, 83 | KafkaHeaders: goharvest.KafkaHeaders{ 84 | goharvest.KafkaHeader{Key: "Seq", Value: strconv.Itoa(i)}, 85 | }, 86 | } 87 | err := pre.Stash(rec) 88 | if err != nil { 89 | panic(err) 90 | } 91 | time.Sleep(time.Duration(interval * int(time.Millisecond))) 92 | meter.Add(1) 93 | meter.MaybeStatsLog(log.Printf) 94 | } 95 | finaliseTx(tx) 96 | } 97 | 98 | func finaliseTx(tx *sql.Tx) { 99 | if tx != nil { 100 | err := tx.Commit() 101 | if err != nil { 102 | panic(err) 103 | } 104 | } 105 | } 106 | -------------------------------------------------------------------------------- /kafka.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "fmt" 5 | "time" 6 | 7 | "gopkg.in/confluentinc/confluent-kafka-go.v1/kafka" 8 | ) 9 | 10 | /* 11 | Interfaces. 12 | */ 13 | 14 | // KafkaConsumer specifies the methods of a minimal consumer. 15 | type KafkaConsumer interface { 16 | Subscribe(topic string, rebalanceCb kafka.RebalanceCb) error 17 | ReadMessage(timeout time.Duration) (*kafka.Message, error) 18 | Close() error 19 | } 20 | 21 | // KafkaConsumerProvider is a factory for creating KafkaConsumer instances. 22 | type KafkaConsumerProvider func(conf *KafkaConfigMap) (KafkaConsumer, error) 23 | 24 | // KafkaProducer specifies the methods of a minimal producer. 25 | type KafkaProducer interface { 26 | Events() chan kafka.Event 27 | Produce(msg *kafka.Message, deliveryChan chan kafka.Event) error 28 | Close() 29 | } 30 | 31 | // KafkaProducerProvider is a factory for creating KafkaProducer instances. 32 | type KafkaProducerProvider func(conf *KafkaConfigMap) (KafkaProducer, error) 33 | 34 | /* 35 | Standard provider implementations. 36 | */ 37 | 38 | // StandardKafkaConsumerProvider returns a factory for creating a conventional KafkaConsumer, backed by the real client API. 39 | func StandardKafkaConsumerProvider() KafkaConsumerProvider { 40 | return func(conf *KafkaConfigMap) (KafkaConsumer, error) { 41 | return kafka.NewConsumer(toKafkaNativeConfig(conf)) 42 | } 43 | } 44 | 45 | // StandardKafkaProducerProvider returns a factory for creating a conventional KafkaProducer, backed by the real client API. 46 | func StandardKafkaProducerProvider() KafkaProducerProvider { 47 | return func(conf *KafkaConfigMap) (KafkaProducer, error) { 48 | return kafka.NewProducer(toKafkaNativeConfig(conf)) 49 | } 50 | } 51 | 52 | /* 53 | Various helpers. 54 | */ 55 | 56 | func toKafkaNativeConfig(conf *KafkaConfigMap) *kafka.ConfigMap { 57 | result := kafka.ConfigMap{} 58 | for k, v := range *conf { 59 | result[k] = v 60 | } 61 | return &result 62 | } 63 | 64 | func copyKafkaConfig(configMap KafkaConfigMap) KafkaConfigMap { 65 | copy := KafkaConfigMap{} 66 | putAllKafkaConfig(configMap, copy) 67 | return copy 68 | } 69 | 70 | func putAllKafkaConfig(source, target KafkaConfigMap) { 71 | for k, v := range source { 72 | target[k] = v 73 | } 74 | } 75 | 76 | func setKafkaConfig(configMap KafkaConfigMap, key string, value interface{}) error { 77 | _, containsKey := configMap[key] 78 | if containsKey { 79 | return fmt.Errorf("cannot override configuration '%s'", key) 80 | } 81 | 82 | configMap[key] = value 83 | return nil 84 | } 85 | 86 | func setKafkaConfigs(configMap, toSet KafkaConfigMap) error { 87 | for k, v := range toSet { 88 | err := setKafkaConfig(configMap, k, v) 89 | if err != nil { 90 | return err 91 | } 92 | } 93 | return nil 94 | } 95 | 96 | func toNativeKafkaHeaders(headers KafkaHeaders) (nativeHeaders []kafka.Header) { 97 | if numHeaders := len(headers); numHeaders > 0 { 98 | nativeHeaders = make([]kafka.Header, numHeaders) 99 | for i, header := range headers { 100 | nativeHeaders[i] = kafka.Header{Key: header.Key, Value: []byte(header.Value)} 101 | } 102 | } 103 | return 104 | } 105 | -------------------------------------------------------------------------------- /kafka_mock_test.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "time" 5 | 6 | "github.com/obsidiandynamics/libstdgo/concurrent" 7 | "gopkg.in/confluentinc/confluent-kafka-go.v1/kafka" 8 | ) 9 | 10 | type consMockFuncs struct { 11 | Subscribe func(m *consMock, topic string, rebalanceCb kafka.RebalanceCb) error 12 | ReadMessage func(m *consMock, timeout time.Duration) (*kafka.Message, error) 13 | Close func(m *consMock) error 14 | } 15 | 16 | type consMockCounts struct { 17 | Subscribe, 18 | ReadMessage, 19 | Close concurrent.AtomicCounter 20 | } 21 | 22 | type consMock struct { 23 | rebalanceCallback kafka.RebalanceCb 24 | rebalanceEvents chan kafka.Event 25 | f consMockFuncs 26 | c consMockCounts 27 | } 28 | 29 | func (m *consMock) Subscribe(topic string, rebalanceCb kafka.RebalanceCb) error { 30 | defer m.c.Subscribe.Inc() 31 | m.rebalanceCallback = rebalanceCb 32 | return m.f.Subscribe(m, topic, rebalanceCb) 33 | } 34 | 35 | func (m *consMock) ReadMessage(timeout time.Duration) (*kafka.Message, error) { 36 | defer m.c.ReadMessage.Inc() 37 | if m.rebalanceCallback != nil { 38 | // The rebalance events should only be delivered in the polling thread, which is why we wait for 39 | // ReadMessage before forwarding the events to the rebalance callback 40 | select { 41 | case e := <-m.rebalanceEvents: 42 | m.rebalanceCallback(nil, e) 43 | default: 44 | } 45 | } 46 | return m.f.ReadMessage(m, timeout) 47 | } 48 | 49 | func (m *consMock) Close() error { 50 | defer m.c.Close.Inc() 51 | return m.f.Close(m) 52 | } 53 | 54 | func (m *consMock) fillDefaults() { 55 | if m.rebalanceEvents == nil { 56 | m.rebalanceEvents = make(chan kafka.Event) 57 | } 58 | if m.f.Subscribe == nil { 59 | m.f.Subscribe = func(m *consMock, topic string, rebalanceCb kafka.RebalanceCb) error { 60 | return nil 61 | } 62 | } 63 | if m.f.ReadMessage == nil { 64 | m.f.ReadMessage = func(m *consMock, timeout time.Duration) (*kafka.Message, error) { 65 | return nil, newTimedOutError() 66 | } 67 | } 68 | if m.f.Close == nil { 69 | m.f.Close = func(m *consMock) error { 70 | return nil 71 | } 72 | } 73 | m.c.Subscribe = concurrent.NewAtomicCounter() 74 | m.c.ReadMessage = concurrent.NewAtomicCounter() 75 | m.c.Close = concurrent.NewAtomicCounter() 76 | } 77 | 78 | func mockKafkaConsumerProvider(m *consMock) func(conf *KafkaConfigMap) (KafkaConsumer, error) { 79 | return func(conf *KafkaConfigMap) (KafkaConsumer, error) { 80 | return m, nil 81 | } 82 | } 83 | 84 | type prodMockFuncs struct { 85 | Events func(m *prodMock) chan kafka.Event 86 | Produce func(m *prodMock, msg *kafka.Message, deliveryChan chan kafka.Event) error 87 | Close func(m *prodMock) 88 | } 89 | 90 | type prodMockCounts struct { 91 | Events, 92 | Produce, 93 | Close concurrent.AtomicCounter 94 | } 95 | 96 | type prodMock struct { 97 | events chan kafka.Event 98 | f prodMockFuncs 99 | c prodMockCounts 100 | } 101 | 102 | func (m *prodMock) Events() chan kafka.Event { 103 | defer m.c.Events.Inc() 104 | return m.f.Events(m) 105 | } 106 | 107 | func (m *prodMock) Produce(msg *kafka.Message, deliveryChan chan kafka.Event) error { 108 | defer m.c.Produce.Inc() 109 | return m.f.Produce(m, msg, deliveryChan) 110 | } 111 | 112 | func (m *prodMock) Close() { 113 | defer m.c.Close.Inc() 114 | m.f.Close(m) 115 | } 116 | 117 | func (m *prodMock) fillDefaults() { 118 | if m.events == nil { 119 | m.events = make(chan kafka.Event) 120 | } 121 | if m.f.Events == nil { 122 | m.f.Events = func(m *prodMock) chan kafka.Event { 123 | return m.events 124 | } 125 | } 126 | if m.f.Produce == nil { 127 | m.f.Produce = func(m *prodMock, msg *kafka.Message, deliveryChan chan kafka.Event) error { 128 | return nil 129 | } 130 | } 131 | if m.f.Close == nil { 132 | m.f.Close = func(m *prodMock) { 133 | close(m.events) 134 | } 135 | } 136 | m.c.Events = concurrent.NewAtomicCounter() 137 | m.c.Produce = concurrent.NewAtomicCounter() 138 | m.c.Close = concurrent.NewAtomicCounter() 139 | } 140 | 141 | func mockKafkaProducerProvider(m *prodMock) func(conf *KafkaConfigMap) (KafkaProducer, error) { 142 | return func(conf *KafkaConfigMap) (KafkaProducer, error) { 143 | return m, nil 144 | } 145 | } 146 | -------------------------------------------------------------------------------- /metric/meter.go: -------------------------------------------------------------------------------- 1 | package metric 2 | 3 | import ( 4 | "fmt" 5 | "time" 6 | 7 | "github.com/obsidiandynamics/libstdgo/scribe" 8 | ) 9 | 10 | // MeterStats is an immutable snapshot of meter statistics. 11 | type MeterStats struct { 12 | Name string 13 | Start time.Time 14 | TotalCount int64 15 | TotalRatePerS float64 16 | IntervalCount int64 17 | IntervalRatePerS float64 18 | } 19 | 20 | // String produces a textual representation of a MeterStats object. 21 | func (s MeterStats) String() string { 22 | return fmt.Sprintf("Meter <%s>: %d since %v, rate: %.3f current, %.3f average\n", 23 | s.Name, s.TotalCount, s.Start.Format(timeFormat), s.IntervalRatePerS, s.TotalRatePerS) 24 | } 25 | 26 | // Meter is a simple structure for tracking the volume of events observed from two points in time: 27 | // 1. When the Meter object was created (or when it was last reset) 28 | // 2. From the last snapshot point. 29 | // 30 | // A meter can be updated by adding more observations. Statistics can be periodically extracted from the 31 | // meter, reflecting the total observed volume as well as the volume in the most recent period. 32 | // 33 | // A meter is not thread-safe. In the absence of locking, it should only be accessed from a single 34 | // goroutine. 35 | type Meter struct { 36 | name string 37 | printInterval time.Duration 38 | start time.Time 39 | totalCount int64 40 | lastIntervalStart time.Time 41 | lastCount int64 42 | lastStats MeterStats 43 | } 44 | 45 | const timeFormat = "2006-01-02T15:04:05" 46 | 47 | // String produces a textual representation of a Meter object. 48 | func (m Meter) String() string { 49 | return fmt.Sprint("Meter[name=", m.name, 50 | ", snapshotInterval=", m.printInterval, 51 | ", start=", m.start.Format(timeFormat), 52 | ", totalCount=", m.totalCount, 53 | ", lastIntervalStart=", m.lastIntervalStart.Format(timeFormat), 54 | ", lastCount=", m.lastCount, 55 | ", lastStats=", m.lastStats, "]") 56 | } 57 | 58 | // NewMeter constructs a new meter object, with a given name and snapshot interval. The actual snapshotting 59 | // of meter statistics is the responsibility of the goroutine that owns the meter. 60 | func NewMeter(name string, snapshotInterval time.Duration) *Meter { 61 | m := Meter{} 62 | m.name = name 63 | m.printInterval = snapshotInterval 64 | m.Reset() 65 | return &m 66 | } 67 | 68 | // Reset the meter to its initial state — clearing all counters and resetting the clocks. 69 | func (m *Meter) Reset() { 70 | m.start = time.Now() 71 | m.totalCount = 0 72 | m.lastIntervalStart = m.start 73 | m.lastCount = 0 74 | } 75 | 76 | // Add a value to the meter, contributing to the overall count and to the current interval. 77 | func (m *Meter) Add(amount int64) { 78 | m.totalCount += amount 79 | } 80 | 81 | // MaybeStats conditionally returns a stats snapshot if the current sampling interval has lapsed. Otherwise, if the 82 | // sampling interval is still valid, a nil is returned. 83 | func (m *Meter) MaybeStats() *MeterStats { 84 | now := time.Now() 85 | elapsedInIntervalMs := now.Sub(m.lastIntervalStart).Milliseconds() 86 | if elapsedInIntervalMs > m.printInterval.Milliseconds() { 87 | intervalCount := m.totalCount - m.lastCount 88 | intervalRatePerS := float64(intervalCount) / float64(elapsedInIntervalMs) * 1000.0 89 | m.lastCount = m.totalCount 90 | m.lastIntervalStart = now 91 | 92 | elapsedTotalMs := now.Sub(m.start).Milliseconds() 93 | totalRatePerS := float64(m.totalCount) / float64(elapsedTotalMs) * 1000.0 94 | 95 | m.lastStats = MeterStats{ 96 | Name: m.name, 97 | Start: m.start, 98 | TotalCount: m.totalCount, 99 | TotalRatePerS: totalRatePerS, 100 | IntervalCount: intervalCount, 101 | IntervalRatePerS: intervalRatePerS, 102 | } 103 | return &m.lastStats 104 | } 105 | return nil 106 | } 107 | 108 | // MeterStatsCallback is invoked by MaybeStatsCall(). 109 | type MeterStatsCallback func(stats MeterStats) 110 | 111 | // MaybeStatsCall conditionally invokes the given MeterStatsCallback if the current sampling interval has lapsed, returning true 112 | // if the callback was invoked. 113 | func (m *Meter) MaybeStatsCall(cb MeterStatsCallback) bool { 114 | s := m.MaybeStats() 115 | if s != nil { 116 | cb(*s) 117 | return true 118 | } 119 | return false 120 | } 121 | 122 | // MaybeStatsLog conditionally logs the snapshot of the recent sampling interval if the latter has lapsed, returning true if an 123 | // entry was logged. 124 | func (m *Meter) MaybeStatsLog(logger scribe.Logger) bool { 125 | return m.MaybeStatsCall(func(stats MeterStats) { 126 | logger("%v", stats) 127 | }) 128 | } 129 | -------------------------------------------------------------------------------- /postgres.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "database/sql" 5 | "fmt" 6 | "sort" 7 | 8 | "github.com/google/uuid" 9 | 10 | // init postgres driver 11 | "github.com/lib/pq" 12 | ) 13 | 14 | type database struct { 15 | db *sql.DB 16 | markStmt *sql.Stmt 17 | purgeStmt *sql.Stmt 18 | resetStmt *sql.Stmt 19 | } 20 | 21 | const markQueryTemplate = ` 22 | -- mark query 23 | UPDATE %s 24 | SET leader_id = $1 25 | WHERE id IN ( 26 | SELECT id FROM %s 27 | WHERE leader_id IS NULL OR leader_id != $1 28 | ORDER BY id 29 | LIMIT $2 30 | ) 31 | RETURNING id, create_time, kafka_topic, kafka_key, kafka_value, kafka_header_keys, kafka_header_values, leader_id 32 | ` 33 | 34 | const purgeQueryTemplate = ` 35 | -- purge query 36 | DELETE FROM %s 37 | WHERE id = $1 38 | ` 39 | 40 | const resetQueryTemplate = ` 41 | -- reset query 42 | UPDATE %s 43 | SET leader_id = NULL 44 | WHERE id = $1 45 | ` 46 | 47 | func closeResource(stmt *sql.Stmt) { 48 | if stmt != nil { 49 | stmt.Close() 50 | } 51 | } 52 | 53 | func closeResources(stmts ...*sql.Stmt) { 54 | for _, resource := range stmts { 55 | closeResource(resource) 56 | } 57 | } 58 | 59 | type databaseProvider func() (*sql.DB, error) 60 | 61 | // StandardPostgresBindingProvider returns a DatabaseBindingProvider that connects to a real Postgres database. 62 | func StandardPostgresBindingProvider() DatabaseBindingProvider { 63 | return NewPostgresBinding 64 | } 65 | 66 | // NewPostgresBinding creates a Postgres binding for the given dataSource and outboxTable args. 67 | func NewPostgresBinding(dataSource string, outboxTable string) (DatabaseBinding, error) { 68 | return newPostgresBinding(func() (*sql.DB, error) { 69 | return sql.Open("postgres", dataSource) 70 | }, outboxTable) 71 | } 72 | 73 | func newPostgresBinding(dbProvider databaseProvider, outboxTable string) (DatabaseBinding, error) { 74 | success := false 75 | var db *sql.DB 76 | var markStmt, purgeStmt, resetStmt *sql.Stmt 77 | defer func() { 78 | if !success { 79 | if db != nil { 80 | db.Close() 81 | } 82 | closeResources(markStmt, purgeStmt, resetStmt) 83 | } 84 | }() 85 | 86 | db, err := dbProvider() 87 | if err != nil { 88 | return nil, err 89 | } 90 | db.SetMaxOpenConns(2) 91 | db.SetMaxIdleConns(2) 92 | 93 | markStmt, err = db.Prepare(fmt.Sprintf(markQueryTemplate, outboxTable, outboxTable)) 94 | if err != nil { 95 | return nil, err 96 | } 97 | 98 | purgeStmt, err = db.Prepare(fmt.Sprintf(purgeQueryTemplate, outboxTable)) 99 | if err != nil { 100 | return nil, err 101 | } 102 | 103 | resetStmt, err = db.Prepare(fmt.Sprintf(resetQueryTemplate, outboxTable)) 104 | if err != nil { 105 | return nil, err 106 | } 107 | 108 | success = true 109 | return &database{ 110 | db: db, 111 | markStmt: markStmt, 112 | purgeStmt: purgeStmt, 113 | resetStmt: resetStmt, 114 | }, nil 115 | } 116 | 117 | func (db *database) Mark(leaderID uuid.UUID, limit int) ([]OutboxRecord, error) { 118 | rows, err := db.markStmt.Query(leaderID, limit) 119 | if err != nil { 120 | return nil, err 121 | } 122 | 123 | defer rows.Close() 124 | records := make([]OutboxRecord, 0, limit) 125 | for rows.Next() { 126 | record := OutboxRecord{} 127 | var keys []string 128 | var values []string 129 | err := rows.Scan( 130 | &record.ID, 131 | &record.CreateTime, 132 | &record.KafkaTopic, 133 | &record.KafkaKey, 134 | &record.KafkaValue, 135 | pq.Array(&keys), 136 | pq.Array(&values), 137 | &record.LeaderID, 138 | ) 139 | if err != nil { 140 | return nil, err 141 | } 142 | numKeys := len(keys) 143 | if len(keys) != len(values) { 144 | return nil, fmt.Errorf("unequal number of header keys (%d) and values (%d)", numKeys, len(values)) 145 | } 146 | 147 | record.KafkaHeaders = make(KafkaHeaders, numKeys) 148 | for i := 0; i < numKeys; i++ { 149 | record.KafkaHeaders[i] = KafkaHeader{keys[i], values[i]} 150 | } 151 | records = append(records, record) 152 | } 153 | 154 | sort.Slice(records, func(i, j int) bool { 155 | return records[i].ID < records[j].ID 156 | }) 157 | 158 | return records, nil 159 | } 160 | 161 | func (db *database) Purge(id int64) (bool, error) { 162 | res, err := db.purgeStmt.Exec(id) 163 | if err != nil { 164 | return false, err 165 | } 166 | affected, _ := res.RowsAffected() 167 | if affected != 1 { 168 | return false, nil 169 | } 170 | return true, err 171 | } 172 | 173 | func (db *database) Reset(id int64) (bool, error) { 174 | res, err := db.resetStmt.Exec(id) 175 | if err != nil { 176 | return false, err 177 | } 178 | affected, _ := res.RowsAffected() 179 | if affected != 1 { 180 | return false, nil 181 | } 182 | return true, err 183 | } 184 | 185 | func (db *database) Dispose() { 186 | db.db.Close() 187 | closeResources(db.markStmt, db.purgeStmt, db.resetStmt) 188 | } 189 | -------------------------------------------------------------------------------- /goharvest_doc_test.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "database/sql" 5 | "log" 6 | "testing" 7 | 8 | "github.com/obsidiandynamics/libstdgo/check" 9 | "github.com/obsidiandynamics/libstdgo/scribe" 10 | scribelogrus "github.com/obsidiandynamics/libstdgo/scribe/logrus" 11 | "github.com/sirupsen/logrus" 12 | ) 13 | 14 | func Example() { 15 | const dataSource = "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable" 16 | 17 | // Optional: Ensure the database table exists before we start harvesting. 18 | func() { 19 | db, err := sql.Open("postgres", dataSource) 20 | if err != nil { 21 | panic(err) 22 | } 23 | defer db.Close() 24 | 25 | _, err = db.Exec(` 26 | CREATE TABLE IF NOT EXISTS outbox ( 27 | id BIGSERIAL PRIMARY KEY, 28 | create_time TIMESTAMP WITH TIME ZONE NOT NULL, 29 | kafka_topic VARCHAR(249) NOT NULL, 30 | kafka_key VARCHAR(100) NOT NULL, -- pick your own key size 31 | kafka_value VARCHAR(10000), -- pick your own value size 32 | kafka_header_keys TEXT[] NOT NULL, 33 | kafka_header_values TEXT[] NOT NULL, 34 | leader_id UUID 35 | ) 36 | `) 37 | if err != nil { 38 | panic(err) 39 | } 40 | }() 41 | 42 | // Configure the harvester. It will use its own database and Kafka connections under the hood. 43 | config := Config{ 44 | BaseKafkaConfig: KafkaConfigMap{ 45 | "bootstrap.servers": "localhost:9092", 46 | }, 47 | DataSource: dataSource, 48 | } 49 | 50 | // Create a new harvester. 51 | harvest, err := New(config) 52 | if err != nil { 53 | panic(err) 54 | } 55 | 56 | // Start it. 57 | err = harvest.Start() 58 | if err != nil { 59 | panic(err) 60 | } 61 | 62 | // Wait indefinitely for it to end. 63 | log.Fatal(harvest.Await()) 64 | } 65 | 66 | func TestExample(t *testing.T) { 67 | check.RunTargetted(t, Example) 68 | } 69 | 70 | func Example_withCustomLogger() { 71 | // Example: Configure GoHarvest with a Logrus binding for Scribe. 72 | 73 | log := logrus.StandardLogger() 74 | log.SetLevel(logrus.DebugLevel) 75 | 76 | // Configure the custom logger using a binding. 77 | config := Config{ 78 | BaseKafkaConfig: KafkaConfigMap{ 79 | "bootstrap.servers": "localhost:9092", 80 | }, 81 | Scribe: scribe.New(scribelogrus.Bind()), 82 | DataSource: "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable", 83 | } 84 | 85 | // Create a new harvester. 86 | harvest, err := New(config) 87 | if err != nil { 88 | panic(err) 89 | } 90 | 91 | // Start it. 92 | err = harvest.Start() 93 | if err != nil { 94 | panic(err) 95 | } 96 | 97 | // Wait indefinitely for it to end. 98 | log.Fatal(harvest.Await()) 99 | } 100 | 101 | func TestExample_withCustomLogger(t *testing.T) { 102 | check.RunTargetted(t, Example_withCustomLogger) 103 | } 104 | 105 | func Example_withSaslSslAndCustomProducerConfig() { 106 | // Example: Using Kafka with sasl_ssl for authentication and encryption. 107 | 108 | config := Config{ 109 | BaseKafkaConfig: KafkaConfigMap{ 110 | "bootstrap.servers": "localhost:9094", 111 | "security.protocol": "sasl_ssl", 112 | "ssl.ca.location": "ca-cert.pem", 113 | "sasl.mechanism": "SCRAM-SHA-512", 114 | "sasl.username": "alice", 115 | "sasl.password": "alice-secret", 116 | }, 117 | ProducerKafkaConfig: KafkaConfigMap{ 118 | "compression.type": "lz4", 119 | }, 120 | DataSource: "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable", 121 | } 122 | 123 | // Create a new harvester. 124 | harvest, err := New(config) 125 | if err != nil { 126 | panic(err) 127 | } 128 | 129 | // Start harvesting in the background. 130 | err = harvest.Start() 131 | if err != nil { 132 | panic(err) 133 | } 134 | 135 | // Wait indefinitely for the harvester to end. 136 | log.Fatal(harvest.Await()) 137 | } 138 | 139 | func TestExample_withSaslSslAndCustomProducerConfig(t *testing.T) { 140 | check.RunTargetted(t, Example_withSaslSslAndCustomProducerConfig) 141 | } 142 | 143 | func Example_withEventHandler() { 144 | // Example: Registering a custom event handler to get notified of leadership changes and metrics. 145 | 146 | log := logrus.StandardLogger() 147 | log.SetLevel(logrus.TraceLevel) 148 | config := Config{ 149 | BaseKafkaConfig: KafkaConfigMap{ 150 | "bootstrap.servers": "localhost:9092", 151 | }, 152 | DataSource: "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable", 153 | Scribe: scribe.New(scribelogrus.Bind()), 154 | } 155 | 156 | // Create a new harvester and register an event hander. 157 | harvest, err := New(config) 158 | if err != nil { 159 | panic(err) 160 | } 161 | 162 | // Register a handler callback, invoked when an event occurs within goharvest. 163 | // The callback is completely optional; it lets the application piggy-back on leader 164 | // status updates, in case it needs to schedule some additional work (other than 165 | // harvesting outbox records) that should only be run on one process at any given time. 166 | harvest.SetEventHandler(func(e Event) { 167 | switch event := e.(type) { 168 | case LeaderAcquired: 169 | // The application may initialise any state necessary to perform work as a leader. 170 | log.Infof("Got event: leader acquired: %v", event.LeaderID()) 171 | case LeaderRefreshed: 172 | // Indicates that a new leader ID was generated, as a result of having to remark 173 | // a record (typically as due to an earlier delivery error). This is purely 174 | // informational; there is nothing an application should do about this, other 175 | // than taking note of the new leader ID if it has come to rely on it. 176 | log.Infof("Got event: leader refreshed: %v", event.LeaderID()) 177 | case LeaderRevoked: 178 | // The application may block the callback until it wraps up any in-flight 179 | // activity. Only upon returning from the callback, will a new leader be elected. 180 | log.Infof("Got event: leader revoked") 181 | case LeaderFenced: 182 | // The application must immediately terminate any ongoing activity, on the assumption 183 | // that another leader may be imminently elected. Unlike the handling of LeaderRevoked, 184 | // blocking in the callback will not prevent a new leader from being elected. 185 | log.Infof("Got event: leader fenced") 186 | case MeterRead: 187 | // Periodic statistics regarding the harvester's throughput. 188 | log.Infof("Got event: meter read: %v", event.Stats()) 189 | } 190 | }) 191 | 192 | // Start harvesting in the background. 193 | err = harvest.Start() 194 | if err != nil { 195 | panic(err) 196 | } 197 | 198 | // Wait indefinitely for it to end. 199 | log.Fatal(harvest.Await()) 200 | } 201 | 202 | func TestExample_withEventHandler(t *testing.T) { 203 | check.RunTargetted(t, Example_withEventHandler) 204 | } 205 | -------------------------------------------------------------------------------- /config_test.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "testing" 5 | "time" 6 | 7 | "github.com/google/go-cmp/cmp" 8 | "github.com/google/go-cmp/cmp/cmpopts" 9 | "github.com/obsidiandynamics/goneli" 10 | "github.com/obsidiandynamics/libstdgo/check" 11 | "github.com/obsidiandynamics/libstdgo/scribe" 12 | "github.com/stretchr/testify/assert" 13 | "github.com/stretchr/testify/require" 14 | "gopkg.in/yaml.v2" 15 | ) 16 | 17 | func TestDefaultKafkaConsumerProvider(t *testing.T) { 18 | c := Config{} 19 | c.SetDefaults() 20 | 21 | cons, err := c.KafkaConsumerProvider(&KafkaConfigMap{}) 22 | assert.Nil(t, cons) 23 | if assert.NotNil(t, err) { 24 | assert.Contains(t, err.Error(), "Required property") 25 | } 26 | } 27 | 28 | func TestDefaultKafkaProducerProvider(t *testing.T) { 29 | c := Config{} 30 | c.SetDefaults() 31 | 32 | prod, err := c.KafkaProducerProvider(&KafkaConfigMap{"foo": "bar"}) 33 | assert.Nil(t, prod) 34 | if assert.NotNil(t, err) { 35 | assert.Contains(t, err.Error(), "No such configuration property") 36 | } 37 | } 38 | 39 | func TestDefaultNeliProvider(t *testing.T) { 40 | c := Config{} 41 | c.SetDefaults() 42 | 43 | consMock := &consMock{} 44 | consMock.fillDefaults() 45 | prodMock := &prodMock{} 46 | prodMock.fillDefaults() 47 | neli, err := c.NeliProvider(goneli.Config{ 48 | KafkaConsumerProvider: convertKafkaConsumerProvider(mockKafkaConsumerProvider(consMock)), 49 | KafkaProducerProvider: convertKafkaProducerProvider(mockKafkaProducerProvider(prodMock)), 50 | }, goneli.NopBarrier()) 51 | assert.NotNil(t, neli) 52 | assert.Nil(t, err) 53 | assert.Nil(t, neli.Close()) 54 | } 55 | 56 | func TestLimitsString(t *testing.T) { 57 | lim := Limits{} 58 | lim.SetDefaults() 59 | assert.Contains(t, lim.String(), "Limits[") 60 | } 61 | 62 | func TestLimitsFromYaml(t *testing.T) { 63 | const y = ` 64 | ioErrorBackoff: 10ms 65 | pollDuration: 20ms 66 | minPollInterval: 30ms 67 | ` 68 | lim := Limits{} 69 | err := yaml.UnmarshalStrict([]byte(y), &lim) 70 | assert.Nil(t, err) 71 | assert.Equal(t, 10*time.Millisecond, *lim.IOErrorBackoff) 72 | assert.Equal(t, 20*time.Millisecond, *lim.PollDuration) 73 | assert.Equal(t, 30*time.Millisecond, *lim.MinPollInterval) 74 | 75 | lim.SetDefaults() 76 | 77 | // Check that the defaults weren't overridden. 78 | def := Limits{} 79 | def.SetDefaults() 80 | assert.Equal(t, *def.MarkBackoff, *lim.MarkBackoff) 81 | } 82 | 83 | func TestGetString(t *testing.T) { 84 | assert.Equal(t, "some-default", getString("some-default", func() (string, error) { return "", check.ErrSimulated })) 85 | assert.Equal(t, "some-string", getString("some-default", func() (string, error) { return "some-string", nil })) 86 | } 87 | 88 | func TestValidateLimits(t *testing.T) { 89 | lim := Limits{} 90 | lim.SetDefaults() 91 | assert.Nil(t, lim.Validate()) 92 | 93 | lim = Limits{ 94 | IOErrorBackoff: Duration(-1), 95 | PollDuration: Duration(time.Millisecond), 96 | } 97 | lim.SetDefaults() 98 | if err := lim.Validate(); assert.NotNil(t, err) { 99 | assert.Equal(t, "IOErrorBackoff: must be no less than 0.", lim.Validate().Error()) 100 | } 101 | 102 | lim = Limits{ 103 | DrainInterval: Duration(0), 104 | } 105 | lim.SetDefaults() 106 | if err := lim.Validate(); assert.NotNil(t, err) { 107 | assert.Equal(t, "DrainInterval: cannot be blank.", lim.Validate().Error()) 108 | } 109 | 110 | lim = Limits{ 111 | DrainInterval: Duration(1 * time.Nanosecond), 112 | } 113 | lim.SetDefaults() 114 | if err := lim.Validate(); assert.NotNil(t, err) { 115 | assert.Equal(t, "DrainInterval: must be no less than 1ms.", lim.Validate().Error()) 116 | } 117 | } 118 | 119 | func TestConfigString(t *testing.T) { 120 | cfg := Config{} 121 | cfg.SetDefaults() 122 | assert.Contains(t, cfg.String(), "Config[") 123 | } 124 | 125 | func TestValidateConfig_valid(t *testing.T) { 126 | cfg := Config{ 127 | BaseKafkaConfig: KafkaConfigMap{}, 128 | ProducerKafkaConfig: KafkaConfigMap{}, 129 | LeaderTopic: "leader-topic", 130 | LeaderGroupID: "leader-group-d", 131 | DataSource: "data-source", 132 | OutboxTable: "outbox-table", 133 | KafkaConsumerProvider: StandardKafkaConsumerProvider(), 134 | KafkaProducerProvider: StandardKafkaProducerProvider(), 135 | DatabaseBindingProvider: StandardPostgresBindingProvider(), 136 | Scribe: scribe.New(scribe.StandardBinding()), 137 | Name: "name", 138 | } 139 | cfg.SetDefaults() 140 | assert.Nil(t, cfg.Validate()) 141 | } 142 | 143 | func TestValidateConfig_invalidLimits(t *testing.T) { 144 | cfg := Config{ 145 | BaseKafkaConfig: KafkaConfigMap{}, 146 | ProducerKafkaConfig: KafkaConfigMap{}, 147 | LeaderTopic: "leader-topic", 148 | LeaderGroupID: "leader-group-id", 149 | DataSource: "data-source", 150 | OutboxTable: "outbox-table", 151 | Limits: Limits{ 152 | SendConcurrency: Int(-1), 153 | }, 154 | KafkaConsumerProvider: StandardKafkaConsumerProvider(), 155 | KafkaProducerProvider: StandardKafkaProducerProvider(), 156 | DatabaseBindingProvider: StandardPostgresBindingProvider(), 157 | Scribe: scribe.New(scribe.StandardBinding()), 158 | Name: "name", 159 | } 160 | cfg.SetDefaults() 161 | assert.NotNil(t, cfg.Validate()) 162 | } 163 | 164 | func TestValidateConfig_default(t *testing.T) { 165 | cfg := Config{} 166 | cfg.SetDefaults() 167 | assert.Nil(t, cfg.Validate()) 168 | } 169 | 170 | func TestDefaultDrainTimeout(t *testing.T) { 171 | cfg := Config{ 172 | Limits: Limits{ 173 | HeartbeatTimeout: Duration(40 * time.Second), 174 | }, 175 | } 176 | cfg.SetDefaults() 177 | assert.Equal(t, 20*time.Second, *cfg.Limits.MaxPollInterval) 178 | assert.Equal(t, 20*time.Second, *cfg.Limits.DrainInterval) 179 | 180 | cfg = Config{ 181 | Limits: Limits{ 182 | HeartbeatTimeout: Duration(40 * time.Second), 183 | QueueTimeout: Duration(15 * time.Second), 184 | }, 185 | } 186 | cfg.SetDefaults() 187 | assert.Equal(t, 20*time.Second, *cfg.Limits.MaxPollInterval) 188 | assert.Equal(t, 15*time.Second, *cfg.Limits.DrainInterval) 189 | } 190 | 191 | func TestUnmarshal_fullyPopulated(t *testing.T) { 192 | const y = ` 193 | baseKafkaConfig: 194 | bootstrap.servers: localhost:9093 195 | producerKafkaConfig: 196 | compression.type: lz4 197 | leaderTopic: leader-topic 198 | leaderGroupID: leader-group-id 199 | dataSource: data-source 200 | outboxTable: outbox-table 201 | limits: 202 | ioErrorBackoff: 10ms 203 | pollDuration: 20ms 204 | minPollInterval: 30ms 205 | maxPollInterval: 35ms 206 | heartbeatTimeout: 15ms 207 | drainInterval: 32ms 208 | queueTimeout: 40ms 209 | markBackoff: 50ms 210 | maxInFlightRecords: 60 211 | sendConcurrency: 70 212 | sendBuffer: 80 213 | minMetricsInterval: 90ms 214 | name: test-name 215 | ` 216 | cfg, err := Unmarshal([]byte(y)) 217 | require.Nil(t, err) 218 | if !assert.Nil(t, cfg.Validate()) { 219 | t.Errorf("Validation error: %s", cfg.Validate().Error()) 220 | } 221 | exp := Config{ 222 | BaseKafkaConfig: KafkaConfigMap{ 223 | "bootstrap.servers": "localhost:9093", 224 | }, 225 | ProducerKafkaConfig: KafkaConfigMap{ 226 | "compression.type": "lz4", 227 | }, 228 | LeaderTopic: "leader-topic", 229 | LeaderGroupID: "leader-group-id", 230 | DataSource: "data-source", 231 | OutboxTable: "outbox-table", 232 | Limits: Limits{ 233 | IOErrorBackoff: Duration(10 * time.Millisecond), 234 | PollDuration: Duration(20 * time.Millisecond), 235 | MinPollInterval: Duration(30 * time.Millisecond), 236 | MaxPollInterval: Duration(35 * time.Millisecond), 237 | HeartbeatTimeout: Duration(15 * time.Millisecond), 238 | DrainInterval: Duration(32 * time.Millisecond), 239 | QueueTimeout: Duration(40 * time.Millisecond), 240 | MarkBackoff: Duration(50 * time.Millisecond), 241 | MaxInFlightRecords: Int(60), 242 | SendConcurrency: Int(70), 243 | SendBuffer: Int(80), 244 | MinMetricsInterval: Duration(90 * time.Millisecond), 245 | }, 246 | Name: "test-name", 247 | } 248 | exp.SetDefaults() 249 | ignoreFields := cmpopts.IgnoreFields( 250 | Config{}, 251 | "KafkaConsumerProvider", "KafkaProducerProvider", "DatabaseBindingProvider", "NeliProvider", "Scribe", 252 | ) 253 | assert.True(t, cmp.Equal(exp, cfg, ignoreFields), "Diff: %v", cmp.Diff(exp, cfg, ignoreFields)) 254 | } 255 | 256 | func TestUnmarshal_empty(t *testing.T) { 257 | const y = `` 258 | cfg, err := Unmarshal([]byte(y)) 259 | assert.Nil(t, err) 260 | if !assert.Nil(t, cfg.Validate()) { 261 | t.Errorf("Validation error: %s", cfg.Validate().Error()) 262 | } 263 | exp := Config{} 264 | exp.SetDefaults() 265 | ignoreFields := cmpopts.IgnoreFields( 266 | Config{}, 267 | "KafkaConsumerProvider", "KafkaProducerProvider", "DatabaseBindingProvider", "NeliProvider", "Scribe", "Name", 268 | ) 269 | assert.True(t, cmp.Equal(exp, cfg, ignoreFields), "Diff: %v", cmp.Diff(exp, cfg, ignoreFields)) 270 | } 271 | -------------------------------------------------------------------------------- /config.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "fmt" 5 | "os" 6 | "time" 7 | 8 | validation "github.com/go-ozzo/ozzo-validation" 9 | "github.com/obsidiandynamics/goneli" 10 | "github.com/obsidiandynamics/libstdgo/scribe" 11 | "gopkg.in/yaml.v2" 12 | ) 13 | 14 | // Duration is a convenience for deriving a pointer from a given Duration argument. 15 | func Duration(d time.Duration) *time.Duration { 16 | return &d 17 | } 18 | 19 | // Int is a convenience for deriving a pointer from a given int argument. 20 | func Int(i int) *int { 21 | return &i 22 | } 23 | 24 | // Limits configuration. 25 | type Limits struct { 26 | IOErrorBackoff *time.Duration `yaml:"ioErrorBackoff"` 27 | PollDuration *time.Duration `yaml:"pollDuration"` 28 | MinPollInterval *time.Duration `yaml:"minPollInterval"` 29 | MaxPollInterval *time.Duration `yaml:"maxPollInterval"` 30 | HeartbeatTimeout *time.Duration `yaml:"heartbeatTimeout"` 31 | DrainInterval *time.Duration `yaml:"drainInterval"` 32 | QueueTimeout *time.Duration `yaml:"queueTimeout"` 33 | MarkBackoff *time.Duration `yaml:"markBackoff"` 34 | MaxInFlightRecords *int `yaml:"maxInFlightRecords"` 35 | SendConcurrency *int `yaml:"sendConcurrency"` 36 | SendBuffer *int `yaml:"sendBuffer"` 37 | MarkQueryRecords *int `yaml:"markQueryRecords"` 38 | MinMetricsInterval *time.Duration `yaml:"minMetricsInterval"` 39 | } 40 | 41 | func defaultInt(i **int, def int) { 42 | if *i == nil { 43 | *i = &def 44 | } 45 | } 46 | 47 | func defaultDuration(d **time.Duration, def time.Duration) { 48 | if *d == nil { 49 | *d = &def 50 | } 51 | } 52 | 53 | // SetDefaults assigns the defaults for optional values. 54 | func (l *Limits) SetDefaults() { 55 | defaultDuration(&l.IOErrorBackoff, 500*time.Millisecond) 56 | defaultDuration(&l.HeartbeatTimeout, goneli.DefaultHeartbeatTimeout) 57 | defaultDuration(&l.MaxPollInterval, *l.HeartbeatTimeout/2) 58 | defaultDuration(&l.QueueTimeout, 30*time.Second) 59 | defaultDuration(&l.DrainInterval, minDuration(*l.MaxPollInterval, *l.QueueTimeout)) 60 | defaultDuration(&l.MarkBackoff, 10*time.Millisecond) 61 | defaultInt(&l.MaxInFlightRecords, 1000) 62 | defaultInt(&l.SendConcurrency, 8) 63 | defaultInt(&l.SendBuffer, 10) 64 | defaultInt(&l.MarkQueryRecords, 100) 65 | defaultDuration(&l.MinMetricsInterval, 5*time.Second) 66 | } 67 | 68 | func minDuration(d0, d1 time.Duration) time.Duration { 69 | if d0 < d1 { 70 | return d0 71 | } 72 | return d1 73 | } 74 | 75 | // Validate the Limits configuration, returning an error if invalid 76 | func (l Limits) Validate() error { 77 | minimumMaxPollInterval := 1 * time.Millisecond 78 | if l.MinPollInterval != nil { 79 | minimumMaxPollInterval = *l.MinPollInterval 80 | } 81 | return validation.ValidateStruct(&l, 82 | validation.Field(&l.IOErrorBackoff, validation.Min(0)), 83 | validation.Field(&l.DrainInterval, validation.Required, validation.Min(1*time.Millisecond)), 84 | validation.Field(&l.MaxPollInterval, validation.Required, validation.Min(minimumMaxPollInterval)), 85 | validation.Field(&l.QueueTimeout, validation.Required, validation.Min(1*time.Millisecond)), 86 | validation.Field(&l.MarkBackoff, validation.Min(0)), 87 | validation.Field(&l.MaxInFlightRecords, validation.Required, validation.Min(1)), 88 | validation.Field(&l.SendConcurrency, validation.Required, validation.Min(1)), 89 | validation.Field(&l.SendBuffer, validation.Min(0)), 90 | validation.Field(&l.MarkQueryRecords, validation.Required, validation.Min(1)), 91 | validation.Field(&l.MinMetricsInterval, validation.Min(0)), 92 | ) 93 | } 94 | 95 | // String obtains a textural representation of Limits. 96 | func (l Limits) String() string { 97 | return fmt.Sprint( 98 | "Limits[IOErrorBackoff=", l.IOErrorBackoff, 99 | ", PollDuration=", l.PollDuration, 100 | ", MinPollInterval=", l.MinPollInterval, 101 | ", MaxPollInterval=", l.MaxPollInterval, 102 | ", HeartbeatTimeout=", l.HeartbeatTimeout, 103 | ", DrainInterval=", l.DrainInterval, 104 | ", QueueTimeout=", l.QueueTimeout, 105 | ", MarkBackoff=", l.MarkBackoff, 106 | ", MaxInFlightRecords=", l.MaxInFlightRecords, 107 | ", SendConcurrency=", l.SendConcurrency, 108 | ", SendBuffer=", l.SendBuffer, 109 | ", MarkQueryRecords=", l.MarkQueryRecords, 110 | ", MinMetricsInterval=", l.MinMetricsInterval, "]", 111 | ) 112 | } 113 | 114 | // KafkaConfigMap represents the Kafka key-value configuration. 115 | type KafkaConfigMap map[string]interface{} 116 | 117 | // Config encapsulates configuration for Harvest. 118 | type Config struct { 119 | BaseKafkaConfig KafkaConfigMap `yaml:"baseKafkaConfig"` 120 | ProducerKafkaConfig KafkaConfigMap `yaml:"producerKafkaConfig"` 121 | LeaderTopic string `yaml:"leaderTopic"` 122 | LeaderGroupID string `yaml:"leaderGroupID"` 123 | DataSource string `yaml:"dataSource"` 124 | OutboxTable string `yaml:"outboxTable"` 125 | Limits Limits `yaml:"limits"` 126 | KafkaConsumerProvider KafkaConsumerProvider 127 | KafkaProducerProvider KafkaProducerProvider 128 | DatabaseBindingProvider DatabaseBindingProvider 129 | NeliProvider NeliProvider 130 | Scribe scribe.Scribe 131 | Name string `yaml:"name"` 132 | } 133 | 134 | // Validate the Config, returning an error if invalid. 135 | func (c Config) Validate() error { 136 | return validation.ValidateStruct(&c, 137 | validation.Field(&c.BaseKafkaConfig, validation.NotNil), 138 | validation.Field(&c.ProducerKafkaConfig, validation.NotNil), 139 | validation.Field(&c.DataSource, validation.Required), 140 | validation.Field(&c.OutboxTable, validation.Required), 141 | validation.Field(&c.Limits), 142 | validation.Field(&c.KafkaConsumerProvider, validation.NotNil), 143 | validation.Field(&c.KafkaProducerProvider, validation.NotNil), 144 | validation.Field(&c.DatabaseBindingProvider, validation.NotNil), 145 | validation.Field(&c.NeliProvider, validation.NotNil), 146 | validation.Field(&c.Scribe, validation.NotNil), 147 | validation.Field(&c.Name, validation.Required), 148 | ) 149 | } 150 | 151 | // Obtains a textual representation of the configuration. 152 | func (c Config) String() string { 153 | return fmt.Sprint( 154 | "Config[BaseKafkaConfig=", c.BaseKafkaConfig, 155 | ", ProducerKafkaConfig=", c.ProducerKafkaConfig, 156 | ", LeaderTopic=", c.LeaderTopic, 157 | ", LeaderGroupID=", c.LeaderGroupID, 158 | ", DataSource=", c.DataSource, 159 | ", OutboxTable=", c.OutboxTable, 160 | ", Limits=", c.Limits, 161 | ", KafkaConsumerProvider=", c.KafkaConsumerProvider, 162 | ", KafkaProducerProvider=", c.KafkaProducerProvider, 163 | ", DatabaseBindingProvider=", c.DatabaseBindingProvider, 164 | ", NeliProvider=", c.NeliProvider, 165 | ", Scribe=", c.Scribe, 166 | ", Name=", c.Name, "]") 167 | } 168 | 169 | // SetDefaults assigns the default values to optional fields. 170 | func (c *Config) SetDefaults() { 171 | if c.BaseKafkaConfig == nil { 172 | c.BaseKafkaConfig = KafkaConfigMap{} 173 | } 174 | if _, ok := c.BaseKafkaConfig["bootstrap.servers"]; !ok { 175 | c.BaseKafkaConfig["bootstrap.servers"] = "localhost:9092" 176 | } 177 | if c.ProducerKafkaConfig == nil { 178 | c.ProducerKafkaConfig = KafkaConfigMap{} 179 | } 180 | if c.DataSource == "" { 181 | c.DataSource = "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable" 182 | } 183 | if c.OutboxTable == "" { 184 | c.OutboxTable = "outbox" 185 | } 186 | c.Limits.SetDefaults() 187 | if c.KafkaConsumerProvider == nil { 188 | c.KafkaConsumerProvider = StandardKafkaConsumerProvider() 189 | } 190 | if c.KafkaProducerProvider == nil { 191 | c.KafkaProducerProvider = StandardKafkaProducerProvider() 192 | } 193 | if c.DatabaseBindingProvider == nil { 194 | c.DatabaseBindingProvider = StandardPostgresBindingProvider() 195 | } 196 | if c.NeliProvider == nil { 197 | c.NeliProvider = StandardNeliProvider() 198 | } 199 | if c.Scribe == nil { 200 | c.Scribe = scribe.New(scribe.StandardBinding()) 201 | } 202 | if c.Name == "" { 203 | c.Name = fmt.Sprintf("%s_%d_%d", goneli.Sanitise(getString("localhost", os.Hostname)), os.Getpid(), time.Now().Unix()) 204 | } 205 | } 206 | 207 | // Unmarshal a configuration from a byte slice, returning the configuration struct with pre-initialised defaults, 208 | // or an error if unmarshalling failed. The configuration is not validated prior to returning, in case further 209 | // amendments are required by the caller. The caller should call Validate() independently. 210 | func Unmarshal(in []byte) (Config, error) { 211 | cfg := Config{} 212 | err := yaml.UnmarshalStrict(in, &cfg) 213 | if err == nil { 214 | cfg.SetDefaults() 215 | } 216 | return cfg, err 217 | } 218 | 219 | type stringGetter func() (string, error) 220 | 221 | func getString(def string, stringGetter stringGetter) string { 222 | str, err := stringGetter() 223 | if err != nil { 224 | return def 225 | } 226 | return str 227 | } 228 | -------------------------------------------------------------------------------- /postgres_test.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "database/sql" 5 | "database/sql/driver" 6 | "testing" 7 | "time" 8 | 9 | "github.com/DATA-DOG/go-sqlmock" 10 | "github.com/google/uuid" 11 | "github.com/lib/pq" 12 | "github.com/obsidiandynamics/libstdgo/check" 13 | "github.com/stretchr/testify/assert" 14 | "github.com/stretchr/testify/require" 15 | ) 16 | 17 | const outboxTable = "outbox" 18 | const markPrepare = "-- mark query" 19 | const purgePrepare = "-- purge query" 20 | const resetPrepare = "-- reset query" 21 | 22 | func pgFixtures() (databaseProvider, sqlmock.Sqlmock) { 23 | db, mock, err := sqlmock.New() 24 | if err != nil { 25 | panic(err) 26 | } 27 | dbProvider := func() (*sql.DB, error) { 28 | return db, nil 29 | } 30 | return dbProvider, mock 31 | } 32 | 33 | func TestErrorInDBProvider(t *testing.T) { 34 | dbProvider := func() (*sql.DB, error) { 35 | return nil, check.ErrSimulated 36 | } 37 | b, err := newPostgresBinding(dbProvider, outboxTable) 38 | assert.Nil(t, b) 39 | assert.Equal(t, check.ErrSimulated, err) 40 | } 41 | 42 | func TestErrorInPrepareMarkQuery(t *testing.T) { 43 | dbProvider, mock := pgFixtures() 44 | mock.ExpectPrepare(markPrepare).WillReturnError(check.ErrSimulated) 45 | 46 | mock.ExpectClose() 47 | b, err := newPostgresBinding(dbProvider, outboxTable) 48 | assert.Nil(t, b) 49 | assert.Equal(t, check.ErrSimulated, err) 50 | assert.Nil(t, mock.ExpectationsWereMet()) 51 | } 52 | 53 | func TestErrorInPreparePurgeQuery(t *testing.T) { 54 | dbProvider, mock := pgFixtures() 55 | mark := mock.ExpectPrepare(markPrepare) 56 | mock.ExpectPrepare(purgePrepare).WillReturnError(check.ErrSimulated) 57 | 58 | mark.WillBeClosed() 59 | mock.ExpectClose() 60 | b, err := newPostgresBinding(dbProvider, outboxTable) 61 | assert.Nil(t, b) 62 | assert.Equal(t, check.ErrSimulated, err) 63 | assert.Nil(t, mock.ExpectationsWereMet()) 64 | } 65 | 66 | func TestErrorInPrepareResetQuery(t *testing.T) { 67 | dbProvider, mock := pgFixtures() 68 | mark := mock.ExpectPrepare(markPrepare) 69 | purge := mock.ExpectPrepare(purgePrepare) 70 | mock.ExpectPrepare(resetPrepare).WillReturnError(check.ErrSimulated) 71 | 72 | mark.WillBeClosed() 73 | purge.WillBeClosed() 74 | mock.ExpectClose() 75 | b, err := newPostgresBinding(dbProvider, outboxTable) 76 | assert.Nil(t, b) 77 | assert.Equal(t, check.ErrSimulated, err) 78 | assert.Nil(t, mock.ExpectationsWereMet()) 79 | } 80 | 81 | const testMarkQueryLimit = 100 82 | 83 | func TestExecuteMark_queryError(t *testing.T) { 84 | dbProvider, mock := pgFixtures() 85 | mark := mock.ExpectPrepare(markPrepare) 86 | purge := mock.ExpectPrepare(purgePrepare) 87 | reset := mock.ExpectPrepare(resetPrepare) 88 | 89 | b, err := newPostgresBinding(dbProvider, outboxTable) 90 | assert.NotNil(t, b) 91 | assert.Nil(t, err) 92 | 93 | leaderID, _ := uuid.NewRandom() 94 | mark.ExpectQuery().WithArgs(leaderID, testMarkQueryLimit).WillReturnError(check.ErrSimulated) 95 | 96 | records, err := b.Mark(leaderID, testMarkQueryLimit) 97 | assert.Nil(t, records) 98 | assert.Equal(t, check.ErrSimulated, err) 99 | 100 | mock.ExpectClose() 101 | mark.WillBeClosed() 102 | purge.WillBeClosed() 103 | reset.WillBeClosed() 104 | b.Dispose() 105 | assert.Nil(t, mock.ExpectationsWereMet()) 106 | } 107 | 108 | // Tests error when one of the columns is of the wrong data type. 109 | func TestExecuteMarkQuery_scanError(t *testing.T) { 110 | dbProvider, mock := pgFixtures() 111 | mark := mock.ExpectPrepare(markPrepare) 112 | mock.ExpectPrepare(purgePrepare) 113 | mock.ExpectPrepare(resetPrepare) 114 | 115 | b, err := newPostgresBinding(dbProvider, outboxTable) 116 | assert.NotNil(t, b) 117 | assert.Nil(t, err) 118 | 119 | leaderID, _ := uuid.NewRandom() 120 | rows := sqlmock.NewRows([]string{ 121 | "id", 122 | "create_time", 123 | "kafka_topic", 124 | "kafka_key", 125 | "kafka_value", 126 | "kafka_header_keys", 127 | "kafka_header_values", 128 | "leader_id", 129 | }) 130 | rows.AddRow("non-int", "", "", "", "", pq.Array([]string{"some-key"}), pq.Array([]string{"some-value"}), leaderID) 131 | mark.ExpectQuery().WithArgs(leaderID, testMarkQueryLimit).WillReturnRows(rows) 132 | 133 | records, err := b.Mark(leaderID, testMarkQueryLimit) 134 | assert.Nil(t, records) 135 | if assert.NotNil(t, err) { 136 | assert.Contains(t, err.Error(), "Scan error on column") 137 | } 138 | } 139 | 140 | func TestExecuteMark_success(t *testing.T) { 141 | dbProvider, mock := pgFixtures() 142 | mark := mock.ExpectPrepare(markPrepare) 143 | mock.ExpectPrepare(purgePrepare) 144 | mock.ExpectPrepare(resetPrepare) 145 | 146 | b, err := newPostgresBinding(dbProvider, outboxTable) 147 | assert.NotNil(t, b) 148 | assert.Nil(t, err) 149 | 150 | leaderID, _ := uuid.NewRandom() 151 | exp := []OutboxRecord{ 152 | { 153 | ID: 77, 154 | CreateTime: time.Now(), 155 | KafkaTopic: "kafka_topic", 156 | KafkaKey: "kafka_key", 157 | KafkaValue: String("kafka_value"), 158 | KafkaHeaders: KafkaHeaders{ 159 | KafkaHeader{Key: "some-key", Value: "some-value"}, 160 | }, 161 | LeaderID: nil, 162 | }, 163 | { 164 | ID: 78, 165 | CreateTime: time.Now(), 166 | KafkaTopic: "kafka_topic", 167 | KafkaKey: "kafka_key", 168 | KafkaValue: String("kafka_value"), 169 | KafkaHeaders: KafkaHeaders{}, 170 | LeaderID: nil, 171 | }, 172 | } 173 | reverse := func(recs []OutboxRecord) []OutboxRecord { 174 | reversed := make([]OutboxRecord, len(recs)) 175 | for i, j := len(recs)-1, 0; i >= 0; i, j = i-1, j+1 { 176 | reversed[i] = recs[j] 177 | } 178 | return reversed 179 | } 180 | 181 | rows := sqlmock.NewRows([]string{ 182 | "id", 183 | "create_time", 184 | "kafka_topic", 185 | "kafka_key", 186 | "kafka_value", 187 | "kafka_header_keys", 188 | "kafka_header_values", 189 | "leader_id", 190 | }) 191 | // Reverse the order before returning to test the sorter inside the marker implementation. 192 | for _, expRec := range reverse(exp) { 193 | headerKeys, headerValues := flattenHeaders(expRec.KafkaHeaders) 194 | rows.AddRow( 195 | expRec.ID, 196 | expRec.CreateTime, 197 | expRec.KafkaTopic, 198 | expRec.KafkaKey, 199 | expRec.KafkaValue, 200 | pq.Array(headerKeys), 201 | pq.Array(headerValues), 202 | expRec.LeaderID, 203 | ) 204 | } 205 | mark.ExpectQuery().WithArgs(leaderID, testMarkQueryLimit).WillReturnRows(rows) 206 | 207 | records, err := b.Mark(leaderID, testMarkQueryLimit) 208 | assert.Nil(t, err) 209 | assert.ElementsMatch(t, []interface{}{exp[0], exp[1]}, records) 210 | assert.Nil(t, mock.ExpectationsWereMet()) 211 | } 212 | 213 | func TestExecuteMark_headerLengthMismatch(t *testing.T) { 214 | dbProvider, mock := pgFixtures() 215 | mark := mock.ExpectPrepare(markPrepare) 216 | mock.ExpectPrepare(purgePrepare) 217 | mock.ExpectPrepare(resetPrepare) 218 | 219 | b, err := newPostgresBinding(dbProvider, outboxTable) 220 | assert.NotNil(t, b) 221 | assert.Nil(t, err) 222 | 223 | leaderID, _ := uuid.NewRandom() 224 | 225 | rows := sqlmock.NewRows([]string{ 226 | "id", 227 | "create_time", 228 | "kafka_topic", 229 | "kafka_key", 230 | "kafka_value", 231 | "kafka_header_keys", 232 | "kafka_header_values", 233 | "leader_id", 234 | }) 235 | rows.AddRow( 236 | 1, 237 | time.Now(), 238 | "some-topic", 239 | "some-key", 240 | "some-value", 241 | pq.Array([]string{"k0"}), 242 | pq.Array([]string{"v0", "v1"}), 243 | leaderID, 244 | ) 245 | mark.ExpectQuery().WithArgs(leaderID, testMarkQueryLimit).WillReturnRows(rows) 246 | 247 | records, err := b.Mark(leaderID, testMarkQueryLimit) 248 | assert.Nil(t, records) 249 | require.NotNil(t, err) 250 | assert.Equal(t, "unequal number of header keys (1) and values (2)", err.Error()) 251 | } 252 | 253 | func flattenHeaders(headers KafkaHeaders) (headerKeys, headerValues []string) { 254 | if numHeaders := len(headers); numHeaders > 0 { 255 | headerKeys = make([]string, numHeaders) 256 | headerValues = make([]string, numHeaders) 257 | for i, header := range headers { 258 | headerKeys[i], headerValues[i] = header.Key, header.Value 259 | } 260 | } else { 261 | headerKeys, headerValues = []string{}, []string{} 262 | } 263 | return 264 | } 265 | 266 | func TestExecutePurge_error(t *testing.T) { 267 | dbProvider, mock := pgFixtures() 268 | mock.ExpectPrepare(markPrepare) 269 | purge := mock.ExpectPrepare(purgePrepare) 270 | mock.ExpectPrepare(resetPrepare) 271 | 272 | b, err := newPostgresBinding(dbProvider, outboxTable) 273 | assert.NotNil(t, b) 274 | assert.Nil(t, err) 275 | 276 | const id = 77 277 | purge.ExpectExec().WithArgs(id).WillReturnError(check.ErrSimulated) 278 | 279 | done, err := b.Purge(id) 280 | assert.False(t, done) 281 | assert.Equal(t, check.ErrSimulated, err) 282 | assert.Nil(t, mock.ExpectationsWereMet()) 283 | } 284 | 285 | func TestExecutePurge_success(t *testing.T) { 286 | dbProvider, mock := pgFixtures() 287 | mock.ExpectPrepare(markPrepare) 288 | purge := mock.ExpectPrepare(purgePrepare) 289 | mock.ExpectPrepare(resetPrepare) 290 | 291 | b, err := newPostgresBinding(dbProvider, outboxTable) 292 | assert.NotNil(t, b) 293 | assert.Nil(t, err) 294 | 295 | const id = 77 296 | purge.ExpectExec().WithArgs(id).WillReturnResult(sqlmock.NewResult(-1, 1)) 297 | 298 | done, err := b.Purge(id) 299 | assert.True(t, done) 300 | assert.Nil(t, err) 301 | assert.Nil(t, mock.ExpectationsWereMet()) 302 | } 303 | 304 | func TestExecutePurge_notDone(t *testing.T) { 305 | dbProvider, mock := pgFixtures() 306 | mock.ExpectPrepare(markPrepare) 307 | purge := mock.ExpectPrepare(purgePrepare) 308 | mock.ExpectPrepare(resetPrepare) 309 | 310 | b, err := newPostgresBinding(dbProvider, outboxTable) 311 | assert.NotNil(t, b) 312 | assert.Nil(t, err) 313 | 314 | const id = 77 315 | purge.ExpectExec().WithArgs(id).WillReturnResult(driver.ResultNoRows) 316 | 317 | done, err := b.Purge(id) 318 | assert.False(t, done) 319 | assert.Nil(t, err) 320 | assert.Nil(t, mock.ExpectationsWereMet()) 321 | } 322 | 323 | func TestExecuteReset_error(t *testing.T) { 324 | dbProvider, mock := pgFixtures() 325 | mock.ExpectPrepare(markPrepare) 326 | mock.ExpectPrepare(purgePrepare) 327 | reset := mock.ExpectPrepare(resetPrepare) 328 | 329 | b, err := newPostgresBinding(dbProvider, outboxTable) 330 | assert.NotNil(t, b) 331 | assert.Nil(t, err) 332 | 333 | const id = 77 334 | reset.ExpectExec().WithArgs(id).WillReturnError(check.ErrSimulated) 335 | 336 | done, err := b.Reset(id) 337 | assert.False(t, done) 338 | assert.Equal(t, check.ErrSimulated, err) 339 | assert.Nil(t, mock.ExpectationsWereMet()) 340 | } 341 | 342 | func TestExecuteReset_success(t *testing.T) { 343 | dbProvider, mock := pgFixtures() 344 | mock.ExpectPrepare(markPrepare) 345 | mock.ExpectPrepare(purgePrepare) 346 | reset := mock.ExpectPrepare(resetPrepare) 347 | 348 | b, err := newPostgresBinding(dbProvider, outboxTable) 349 | assert.NotNil(t, b) 350 | assert.Nil(t, err) 351 | 352 | const id = 77 353 | reset.ExpectExec().WithArgs(id).WillReturnResult(sqlmock.NewResult(-1, 1)) 354 | 355 | done, err := b.Reset(id) 356 | assert.True(t, done) 357 | assert.Nil(t, err) 358 | assert.Nil(t, mock.ExpectationsWereMet()) 359 | } 360 | 361 | func TestExecuteReset_notDone(t *testing.T) { 362 | dbProvider, mock := pgFixtures() 363 | mock.ExpectPrepare(markPrepare) 364 | mock.ExpectPrepare(purgePrepare) 365 | reset := mock.ExpectPrepare(resetPrepare) 366 | 367 | b, err := newPostgresBinding(dbProvider, outboxTable) 368 | assert.NotNil(t, b) 369 | assert.Nil(t, err) 370 | 371 | const id = 77 372 | reset.ExpectExec().WithArgs(id).WillReturnResult(driver.ResultNoRows) 373 | 374 | done, err := b.Reset(id) 375 | assert.False(t, done) 376 | assert.Nil(t, err) 377 | assert.Nil(t, mock.ExpectationsWereMet()) 378 | } 379 | 380 | func TestRealPostgresBinding(t *testing.T) { 381 | b, err := NewPostgresBinding("***corrupt connection info string***", outboxTable) 382 | assert.Nil(t, b) 383 | assert.NotNil(t, err) 384 | } 385 | -------------------------------------------------------------------------------- /go.sum: -------------------------------------------------------------------------------- 1 | github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= 2 | github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= 3 | github.com/DATA-DOG/go-sqlmock v1.4.1 h1:ThlnYciV1iM/V0OSF/dtkqWb6xo5qITT1TJBG1MRDJM= 4 | github.com/DATA-DOG/go-sqlmock v1.4.1/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= 5 | github.com/asaskevich/govalidator v0.0.0-20200108200545-475eaeb16496 h1:zV3ejI06GQ59hwDQAvmK1qxOQGB3WuVTRoY0okPTAv0= 6 | github.com/asaskevich/govalidator v0.0.0-20200108200545-475eaeb16496/go.mod h1:oGkLhpf+kjZl6xBf758TQhh5XrAeiJv/7FRz/2spLIg= 7 | github.com/cihub/seelog v0.0.0-20170130134532-f561c5e57575 h1:kHaBemcxl8o/pQ5VM1c8PVE1PubbNx3mjUr09OqWGCs= 8 | github.com/cihub/seelog v0.0.0-20170130134532-f561c5e57575/go.mod h1:9d6lWj8KzO/fd/NrVaLscBKmPigpZpn5YawRPw+e3Yo= 9 | github.com/confluentinc/confluent-kafka-go v1.5.2 h1:l+qt+a0Okmq0Bdr1P55IX4fiwFJyg0lZQmfHkAFkv7E= 10 | github.com/confluentinc/confluent-kafka-go v1.5.2/go.mod h1:u2zNLny2xq+5rWeTQjFHbDzzNuba4P1vo31r9r4uAdg= 11 | github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= 12 | github.com/davecgh/go-spew v1.1.0 h1:ZDRjVQ15GmhC3fiQ8ni8+OwkZQO4DARzQgrnXU1Liz8= 13 | github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= 14 | github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= 15 | github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= 16 | github.com/go-ozzo/ozzo-validation v3.6.0+incompatible h1:msy24VGS42fKO9K1vLz82/GeYW1cILu7Nuuj1N3BBkE= 17 | github.com/go-ozzo/ozzo-validation v3.6.0+incompatible/go.mod h1:gsEKFIVnabGBt6mXmxK0MoFy+cZoTJY6mu5Ll3LVLBU= 18 | github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk= 19 | github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= 20 | github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= 21 | github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= 22 | github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= 23 | github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= 24 | github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= 25 | github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY= 26 | github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= 27 | github.com/inconshreveable/log15 v0.0.0-20200109203555-b30bc20e4fd1 h1:KUDFlmBg2buRWNzIcwLlKvfcnujcHQRQ1As1LoaCLAM= 28 | github.com/inconshreveable/log15 v0.0.0-20200109203555-b30bc20e4fd1/go.mod h1:cOaXtrgN4ScfRrD9Bre7U1thNq5RtJ8ZoP4iXVGRj6o= 29 | github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= 30 | github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk= 31 | github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= 32 | github.com/konsorten/go-windows-terminal-sequences v1.0.2 h1:DB17ag19krx9CFsz4o3enTrPXyIXCl+2iCXH/aMAp9s= 33 | github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= 34 | github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= 35 | github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= 36 | github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= 37 | github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= 38 | github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= 39 | github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= 40 | github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= 41 | github.com/lib/pq v1.5.1 h1:Jn6HYxiYrtQ92CopqJLvfPCJUrrruw1+1cn0jM9dKrI= 42 | github.com/lib/pq v1.5.1/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= 43 | github.com/mattn/go-colorable v0.1.6 h1:6Su7aK7lXmJ/U79bYtBjLNaha4Fs1Rg9plHpcH+vvnE= 44 | github.com/mattn/go-colorable v0.1.6/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= 45 | github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY= 46 | github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= 47 | github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= 48 | github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= 49 | github.com/obsidiandynamics/goneli v0.4.3 h1:lf3x/qSgEX9S6+Ak5GPcc3TBUQBhPJeiWvGrCykZcbM= 50 | github.com/obsidiandynamics/goneli v0.4.3/go.mod h1:1i3mTL/PaaDKu6f+hlndeRUCbV8uiDxu+203vBpn6oE= 51 | github.com/obsidiandynamics/libstdgo v0.4.1 h1:ZUnz+72xQSMgAjEqxp7i7NOBZlu6AcAE6ppmvVKxK3M= 52 | github.com/obsidiandynamics/libstdgo v0.4.1/go.mod h1:0gKiFsJhfrlCqbWFNhDDUJgj6XbXWZyrl0JS/C+jU5g= 53 | github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= 54 | github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 55 | github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 56 | github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= 57 | github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= 58 | github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= 59 | github.com/sirupsen/logrus v1.5.0 h1:1N5EYkVAPEywqZRJd7cwnRtCb6xJx7NH3T3WUTF980Q= 60 | github.com/sirupsen/logrus v1.5.0/go.mod h1:+F7Ogzej0PZc/94MaYx/nvG9jOFMD2osvC3s+Squfpo= 61 | github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= 62 | github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= 63 | github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= 64 | github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= 65 | github.com/stretchr/testify v1.5.1 h1:nOGnQDM7FYENwehXlg/kFVnos3rEvtKTjRvOWSzb6H4= 66 | github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= 67 | github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= 68 | go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= 69 | go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= 70 | go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= 71 | go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= 72 | go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= 73 | go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= 74 | go.uber.org/zap v1.14.1 h1:nYDKopTbvAPq/NrUVZwT15y2lpROBiLLyoRTbXOYWOo= 75 | go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= 76 | golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= 77 | golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= 78 | golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= 79 | golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= 80 | golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= 81 | golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= 82 | golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= 83 | golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= 84 | golang.org/x/mod v0.2.0 h1:KU7oHjnv3XNWfa5COkzUifxZmxp1TyI7ImMXqFxLwvQ= 85 | golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= 86 | golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= 87 | golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= 88 | golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= 89 | golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= 90 | golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 91 | golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= 92 | golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= 93 | golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 94 | golang.org/x/sys v0.0.0-20190422165155-953cdadca894 h1:Cz4ceDQGXuKRnVBDTS23GTn/pU5OE2C0WrNTOYK1Uuc= 95 | golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 96 | golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 97 | golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae h1:/WDfKMnPU+m5M4xB+6x4kaepxRw6jWvR5iDRdvjHgy8= 98 | golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 99 | golang.org/x/sys v0.0.0-20200413165638-669c56c373c4 h1:opSr2sbRXk5X5/givKrrKj9HXxFpW2sdCiP8MJSKLQY= 100 | golang.org/x/sys v0.0.0-20200413165638-669c56c373c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 101 | golang.org/x/sys v0.0.0-20200420163511-1957bb5e6d1f h1:gWF768j/LaZugp8dyS4UwsslYCYz9XgFxvlgsn0n9H8= 102 | golang.org/x/sys v0.0.0-20200420163511-1957bb5e6d1f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 103 | golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= 104 | golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= 105 | golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= 106 | golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= 107 | golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5 h1:hKsoRgsbwY1NafxrwTs+k64bikrLBkAgPir1TNCj3Zs= 108 | golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= 109 | golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= 110 | golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= 111 | golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= 112 | golang.org/x/tools v0.0.0-20200417140056-c07e33ef3290/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= 113 | golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 114 | golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 115 | golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= 116 | golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 117 | gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= 118 | gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= 119 | gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f h1:BLraFXnmrev5lT+xlilqcH8XK9/i0At2xKjWk4p6zsU= 120 | gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= 121 | gopkg.in/confluentinc/confluent-kafka-go.v1 v1.4.0 h1:70Hht0HKadDe6GpSgstEtYrDMtHo3ZqK+3KeHepusaw= 122 | gopkg.in/confluentinc/confluent-kafka-go.v1 v1.4.0/go.mod h1:ZdI3yfYmdNSLQPNCpO1y00EHyWaHG5EnQEyL/ntAegY= 123 | gopkg.in/confluentinc/confluent-kafka-go.v1 v1.5.2 h1:g0WBLy6fobNUU8W/e9zx6I0Yl79Ya+BDW1NwzAlTiiQ= 124 | gopkg.in/confluentinc/confluent-kafka-go.v1 v1.5.2/go.mod h1:ZdI3yfYmdNSLQPNCpO1y00EHyWaHG5EnQEyL/ntAegY= 125 | gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= 126 | gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= 127 | gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= 128 | gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10= 129 | gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= 130 | honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= 131 | honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= 132 | honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= 133 | -------------------------------------------------------------------------------- /int/harvest_int_test.go: -------------------------------------------------------------------------------- 1 | package int 2 | 3 | import ( 4 | "context" 5 | "database/sql" 6 | "fmt" 7 | "os" 8 | "os/signal" 9 | "strconv" 10 | "strings" 11 | "syscall" 12 | "testing" 13 | "time" 14 | 15 | "github.com/google/uuid" 16 | . "github.com/obsidiandynamics/goharvest" 17 | "github.com/obsidiandynamics/goharvest/stasher" 18 | "github.com/obsidiandynamics/libstdgo/check" 19 | "github.com/obsidiandynamics/libstdgo/concurrent" 20 | "github.com/obsidiandynamics/libstdgo/diags" 21 | "github.com/obsidiandynamics/libstdgo/fault" 22 | "github.com/obsidiandynamics/libstdgo/scribe" 23 | "github.com/obsidiandynamics/libstdgo/scribe/overlog" 24 | "github.com/stretchr/testify/assert" 25 | "github.com/stretchr/testify/require" 26 | "gopkg.in/confluentinc/confluent-kafka-go.v1/kafka" 27 | ) 28 | 29 | type externals struct { 30 | cons *kafka.Consumer 31 | admin *kafka.AdminClient 32 | db *sql.DB 33 | } 34 | 35 | const ( 36 | kafkaNamespace = "goharvest_test" 37 | topic = kafkaNamespace + ".topic" 38 | partitions = 10 39 | dbSchema = "goharvest_test" 40 | outboxTable = dbSchema + ".outbox" 41 | leaderTopic = kafkaNamespace + ".neli" 42 | leaderGroupID = kafkaNamespace + ".group" 43 | receiverGroupID = kafkaNamespace + ".receiver_group" 44 | bootstrapServers = "localhost:9092" 45 | dataSource = "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable" 46 | generateInterval = 5 * time.Millisecond 47 | generateRecordsPerTxn = 20 48 | generateMinRecords = 100 49 | generateUniqueKeys = 10 50 | receiverPollDuration = 500 * time.Millisecond 51 | receiverNoMessagesWarningTime = 10 * time.Second 52 | waitTimeout = 90 * time.Second 53 | ) 54 | 55 | var logger = overlog.New(overlog.StandardFormat()) 56 | var scr = scribe.New(overlog.Bind(logger)) 57 | 58 | func openExternals() externals { 59 | cons, err := kafka.NewConsumer(&kafka.ConfigMap{ 60 | "bootstrap.servers": bootstrapServers, 61 | "group.id": receiverGroupID, 62 | "enable.auto.commit": true, 63 | "auto.offset.reset": "earliest", 64 | "socket.timeout.ms": 10000, 65 | // "debug": "all", 66 | }) 67 | if err != nil { 68 | panic(err) 69 | } 70 | 71 | admin, err := kafka.NewAdminClientFromConsumer(cons) 72 | if err != nil { 73 | panic(err) 74 | } 75 | for { 76 | result, err := admin.CreateTopics(context.Background(), []kafka.TopicSpecification{ 77 | { 78 | Topic: topic, 79 | NumPartitions: partitions, 80 | ReplicationFactor: 1, 81 | }, 82 | }) 83 | if err != nil { 84 | if isFatalError(err) { 85 | panic(err) 86 | } else { 87 | // Allow for timeouts and other non-fatal errors. 88 | scr.W()("Non-fatal error creating topic: %v", err) 89 | } 90 | } else { 91 | if result[0].Error.Code() == kafka.ErrTopicAlreadyExists { 92 | scr.I()("Topic %s already exists", topic) 93 | } else if result[0].Error.Code() != kafka.ErrNoError { 94 | panic(result[0].Error) 95 | } 96 | break 97 | } 98 | } 99 | 100 | db, err := sql.Open("postgres", dataSource) 101 | if err != nil { 102 | panic(err) 103 | } 104 | 105 | const ddlTemplate = ` 106 | CREATE SCHEMA IF NOT EXISTS %s; 107 | DROP TABLE IF EXISTS %s; 108 | CREATE TABLE %s ( 109 | id BIGSERIAL PRIMARY KEY, 110 | create_time TIMESTAMP WITH TIME ZONE NOT NULL, 111 | kafka_topic VARCHAR(249) NOT NULL, 112 | kafka_key VARCHAR(5) NOT NULL, 113 | kafka_value VARCHAR(50), 114 | kafka_header_keys TEXT[] NOT NULL, 115 | kafka_header_values TEXT[] NOT NULL, 116 | leader_id UUID 117 | ) 118 | ` 119 | _, err = db.Exec(fmt.Sprintf(ddlTemplate, dbSchema, outboxTable, outboxTable)) 120 | if err != nil { 121 | panic(err) 122 | } 123 | 124 | return externals{cons, admin, db} 125 | } 126 | 127 | func (x *externals) close() { 128 | x.cons.Close() 129 | x.db.Close() 130 | x.admin.Close() 131 | } 132 | 133 | func wait(t check.Tester) check.Timesert { 134 | return check.Wait(t, waitTimeout) 135 | } 136 | 137 | func TestOneNode_withFailures(t *testing.T) { 138 | test(t, 1, 5*time.Second, ProducerFaultSpecs{ 139 | OnProduce: fault.Spec{Cnt: fault.Random(0.02), Err: check.ErrSimulated}, 140 | OnDelivery: fault.Spec{Cnt: fault.Random(0.02), Err: check.ErrSimulated}, 141 | }) 142 | } 143 | 144 | func TestFourNodes_withFailures(t *testing.T) { 145 | test(t, 4, 5*time.Second, ProducerFaultSpecs{ 146 | OnProduce: fault.Spec{Cnt: fault.Random(0.02), Err: check.ErrSimulated}, 147 | OnDelivery: fault.Spec{Cnt: fault.Random(0.02), Err: check.ErrSimulated}, 148 | }) 149 | } 150 | 151 | func TestEightNodes_withoutFailures(t *testing.T) { 152 | test(t, 8, 2*time.Second, ProducerFaultSpecs{}) 153 | } 154 | 155 | func test(t *testing.T, numHarvests int, spawnInterval time.Duration, producerFaultSpecs ProducerFaultSpecs) { 156 | check.RequireLabel(t, "int") 157 | installSigQuitHandler() 158 | 159 | testID, _ := uuid.NewRandom() 160 | x := openExternals() 161 | defer x.close() 162 | 163 | scr.I()("Starting generator") 164 | generator := startGenerator(t, testID, x.db, generateInterval, generateUniqueKeys) 165 | defer func() { <-generator.stop() }() 166 | 167 | scr.I()("Starting receiver") 168 | receiver := startReceiver(t, testID, x.cons) 169 | defer func() { <-receiver.stop() }() 170 | 171 | harvests := make([]Harvest, numHarvests) 172 | defer func() { 173 | for _, h := range harvests { 174 | if h != nil { 175 | h.Stop() 176 | } 177 | } 178 | }() 179 | // Start harvests at a set interval. 180 | for i := 0; i < numHarvests; i++ { 181 | config := Config{ 182 | KafkaProducerProvider: FaultyKafkaProducerProvider(StandardKafkaProducerProvider(), producerFaultSpecs), 183 | Name: fmt.Sprintf("harvest-#%d", i+1), 184 | Scribe: scribe.New(overlog.Bind(logger)), 185 | BaseKafkaConfig: KafkaConfigMap{ 186 | "bootstrap.servers": bootstrapServers, 187 | "socket.timeout.ms": 10000, 188 | }, 189 | ProducerKafkaConfig: KafkaConfigMap{ 190 | "delivery.timeout.ms": 10000, 191 | // "debug": "broker,topic,metadata", 192 | }, 193 | LeaderTopic: leaderTopic, 194 | OutboxTable: outboxTable, 195 | LeaderGroupID: leaderGroupID, 196 | DataSource: dataSource, 197 | Limits: Limits{ 198 | MinPollInterval: Duration(100 * time.Millisecond), 199 | MarkBackoff: Duration(1 * time.Millisecond), 200 | IOErrorBackoff: Duration(1 * time.Millisecond), 201 | }, 202 | } 203 | config.Scribe.SetEnabled(scribe.Trace) 204 | 205 | scr.I()("Starting harvest %d/%d", i+1, numHarvests) 206 | h, err := New(config) 207 | require.Nil(t, err) 208 | harvests[i] = h 209 | require.Nil(t, h.Start()) 210 | 211 | scr.I()("Sleeping") 212 | sleepWithDeadline(spawnInterval) 213 | } 214 | 215 | // Stop harvests in the order they were started, except for the last one. The last harvest will be stopped 216 | // only after we've asserted the receipt of all messages. 217 | for i := 0; i < numHarvests-1; i++ { 218 | scr.I()("Stopping harvest %d/%d", i+1, numHarvests) 219 | harvests[i].Stop() 220 | scr.I()("In-flight records: %d", harvests[i].InFlightRecords()) 221 | sleepWithDeadline(spawnInterval) 222 | } 223 | 224 | // Wait until the generator produces some records. Once we've produced enough records, stop the 225 | // generator so that we can assert receipt. 226 | generator.recs.Fill(generateMinRecords, concurrent.Indefinitely) 227 | scr.I()("Stopping generator") 228 | <-generator.stop() 229 | generated := generator.recs.GetInt() 230 | scr.I()("Generated %d records", generated) 231 | 232 | // Wait until we received all records. Keep sliding in bite-sized chunks through successive assertions so that, as 233 | // long as we keep on receiving records, the assertion does not fail. This deals with slow harvesters (when we are 234 | // simulating lots of faults). 235 | const waitBatchSize = 100 236 | for r := waitBatchSize; r < generated; r += waitBatchSize { 237 | advanced := wait(t).UntilAsserted(func(t check.Tester) { 238 | assert.GreaterOrEqual(t, receiver.recs.GetInt(), r) 239 | }) 240 | if !advanced { 241 | scr.E()("Stack traces:\n%s", diags.DumpAllStacks()) 242 | } 243 | require.True(t, advanced) 244 | scr.I()("Received %d messages", r) 245 | } 246 | wait(t).UntilAsserted(func(t check.Tester) { 247 | assert.GreaterOrEqual(t, receiver.recs.GetInt(), generated) 248 | }) 249 | assert.Equal(t, generated, receiver.recs.GetInt()) 250 | scr.I()("Stopping receiver") 251 | <-receiver.stop() 252 | 253 | // Stop the last harvest as we've already received all messages and there's nothing more to publish. 254 | scr.I()("Stopping harvest %d/%d", numHarvests, numHarvests) 255 | harvests[numHarvests-1].Stop() 256 | 257 | // Await harvests. 258 | for i, h := range harvests { 259 | scr.I()("Awaiting harvest %d/%d", i+1, numHarvests) 260 | assert.Nil(t, h.Await()) 261 | } 262 | scr.I()("Done") 263 | } 264 | 265 | func sleepWithDeadline(duration time.Duration) { 266 | beforeSleep := time.Now() 267 | time.Sleep(duration) 268 | if elapsed := time.Now().Sub(beforeSleep); elapsed > 2*duration { 269 | scr.W()("Sleep deadline exceeded; expected %v but slept for %v", duration, elapsed) 270 | } 271 | } 272 | 273 | type generator struct { 274 | cancel context.CancelFunc 275 | recs concurrent.AtomicCounter 276 | stopped chan int 277 | } 278 | 279 | func (g generator) stop() chan int { 280 | g.cancel() 281 | return g.stopped 282 | } 283 | 284 | func startGenerator(t *testing.T, testID uuid.UUID, db *sql.DB, interval time.Duration, keys int) generator { 285 | st := stasher.New(outboxTable) 286 | ctx, cancel := concurrent.Forever(context.Background()) 287 | recs := concurrent.NewAtomicCounter() 288 | stopped := make(chan int, 1) 289 | 290 | go func() { 291 | defer scr.T()("Generator exiting") 292 | defer close(stopped) 293 | ticker := time.NewTicker(interval) 294 | defer ticker.Stop() 295 | 296 | var tx *sql.Tx 297 | defer func() { 298 | err := finaliseTx(t, tx) 299 | if err != nil { 300 | scr.E()("Could not finalise transaction: %v", err) 301 | t.Errorf("Could not finalise transaction: %v", err) 302 | } 303 | }() 304 | 305 | var pre stasher.PreStash 306 | seq := 0 307 | for { 308 | if seq%generateRecordsPerTxn == 0 { 309 | err := finaliseTx(t, tx) 310 | if err != nil { 311 | scr.E()("Could not finalise transaction: %v", err) 312 | t.Errorf("Could not finalise transaction: %v", err) 313 | return 314 | } 315 | 316 | newTx, err := db.Begin() 317 | tx = newTx 318 | if err != nil { 319 | scr.E()("Could not begin transaction: %v", err) 320 | t.Errorf("Could not begin transaction: %v", err) 321 | return 322 | } 323 | pre, err = st.Prepare(tx) 324 | if err != nil { 325 | scr.E()("Could not prepare: %v", err) 326 | t.Errorf("Could not prepare: %v", err) 327 | return 328 | } 329 | } 330 | 331 | testIDStr := testID.String() 332 | rec := OutboxRecord{ 333 | KafkaTopic: topic, 334 | KafkaKey: strconv.Itoa(seq % keys), 335 | KafkaValue: String(testIDStr + "_" + strconv.Itoa(seq)), 336 | KafkaHeaders: KafkaHeaders{ 337 | KafkaHeader{Key: "testId", Value: testIDStr}, 338 | }, 339 | } 340 | err := pre.Stash(rec) 341 | if err != nil { 342 | scr.E()("Could not stash: %v", err) 343 | t.Errorf("Could not stash: %v", err) 344 | return 345 | } 346 | 347 | seq = int(recs.Inc()) 348 | select { 349 | case <-ctx.Done(): 350 | return 351 | case <-ticker.C: 352 | } 353 | } 354 | }() 355 | 356 | return generator{cancel, recs, stopped} 357 | } 358 | 359 | func finaliseTx(t *testing.T, tx *sql.Tx) error { 360 | if tx != nil { 361 | return tx.Commit() 362 | } 363 | return nil 364 | } 365 | 366 | type receiver struct { 367 | cancel context.CancelFunc 368 | received map[string]int 369 | recs concurrent.AtomicCounter 370 | stopped chan int 371 | } 372 | 373 | func (r receiver) stop() chan int { 374 | r.cancel() 375 | return r.stopped 376 | } 377 | 378 | func startReceiver(t *testing.T, testID uuid.UUID, cons *kafka.Consumer) receiver { 379 | received := make(map[string]int) 380 | ctx, cancel := concurrent.Forever(context.Background()) 381 | recs := concurrent.NewAtomicCounter() 382 | stopped := make(chan int, 1) 383 | 384 | go func() { 385 | defer scr.T()("Receiver exiting") 386 | defer close(stopped) 387 | 388 | successiveTimeouts := 0 389 | resetTimeouts := func() { 390 | if successiveTimeouts > 0 { 391 | successiveTimeouts = 0 392 | } 393 | } 394 | 395 | err := cons.Subscribe(topic, func(_ *kafka.Consumer, event kafka.Event) error { 396 | switch e := event.(type) { 397 | case kafka.AssignedPartitions: 398 | resetTimeouts() 399 | scr.I()("Receiver: assigned partitions %v", e.Partitions) 400 | case kafka.RevokedPartitions: 401 | resetTimeouts() 402 | scr.I()("Receiver: revoked partitions %v", e.Partitions) 403 | } 404 | return nil 405 | }) 406 | if err != nil { 407 | scr.E()("Could not subscribe: %v", err) 408 | t.Errorf("Could not subscribe: %v", err) 409 | return 410 | } 411 | 412 | lastMessageReceivedTime := time.Now() 413 | messageAbsencePrinted := false 414 | expectedTestID := testID.String() 415 | const partitions = 64 416 | lastReceivedOffsets := make([]kafka.Offset, partitions) 417 | for i := 0; i < partitions; i++ { 418 | lastReceivedOffsets[i] = kafka.Offset(-1) 419 | } 420 | 421 | for { 422 | msg, err := cons.ReadMessage(receiverPollDuration) 423 | if err != nil { 424 | if isFatalError(err) { 425 | scr.E()("Fatal error during poll: %v", err) 426 | t.Errorf("Fatal error during poll: %v", err) 427 | return 428 | } else if !isTimedOutError(err) { 429 | scr.W()("Error during poll: %v", err) 430 | } else { 431 | successiveTimeouts++ 432 | logger.Raw(".") 433 | } 434 | } 435 | 436 | if msg != nil { 437 | if msg.TopicPartition.Offset <= lastReceivedOffsets[msg.TopicPartition.Partition] { 438 | scr.D()("Skipping duplicate delivery at offset %d", msg.TopicPartition.Offset) 439 | continue 440 | } 441 | lastReceivedOffsets[msg.TopicPartition.Partition] = msg.TopicPartition.Offset 442 | lastMessageReceivedTime = time.Now() 443 | messageAbsencePrinted = false 444 | 445 | resetTimeouts() 446 | 447 | valueFrags := strings.Split(string(msg.Value), "_") 448 | if len(valueFrags) != 2 { 449 | scr.E()("invalid value '%s'", string(msg.Value)) 450 | t.Errorf("invalid value '%s'", string(msg.Value)) 451 | return 452 | } 453 | receivedTestID, value := valueFrags[0], valueFrags[1] 454 | if receivedTestID != expectedTestID { 455 | scr.I()("Skipping %s (test ID %s)", string(msg.Value), expectedTestID) 456 | continue 457 | } 458 | key := string(msg.Key) 459 | 460 | receivedSeq, err := strconv.Atoi(value) 461 | if err != nil { 462 | scr.E()("Could not convert message value to sequence: '%s'", value) 463 | t.Errorf("Could not convert message value to sequence: '%s'", value) 464 | return 465 | } 466 | 467 | if assert.Equal(t, 1, len(msg.Headers)) { 468 | assert.Equal(t, expectedTestID, string(msg.Headers[0].Value)) 469 | } 470 | 471 | if existingSeq, ok := received[key]; ok { 472 | if assert.GreaterOrEqual(t, receivedSeq, existingSeq) { 473 | if receivedSeq > existingSeq { 474 | received[key] = receivedSeq 475 | recs.Inc() 476 | } else { 477 | scr.I()("Received duplicate %d for key %s (this is okay)", existingSeq, key) 478 | } 479 | } else { 480 | scr.E()("Received records out of order, %d is behind %d", receivedSeq, existingSeq) 481 | t.Errorf("Received records out of order, %d is behind %d", receivedSeq, existingSeq) 482 | } 483 | } else { 484 | keyInt, err := strconv.Atoi(key) 485 | if err != nil { 486 | scr.E()("Could not convert message key '%s'", key) 487 | t.Errorf("Could not convert message key '%s'", key) 488 | return 489 | } 490 | if assert.Equal(t, keyInt, receivedSeq) { 491 | recs.Inc() 492 | received[key] = receivedSeq 493 | } 494 | } 495 | } else { 496 | elapsed := time.Now().Sub(lastMessageReceivedTime) 497 | if elapsed > receiverNoMessagesWarningTime && !messageAbsencePrinted { 498 | scr.W()("No messages received since %v", lastMessageReceivedTime) 499 | messageAbsencePrinted = true 500 | } 501 | } 502 | 503 | select { 504 | case <-ctx.Done(): 505 | return 506 | default: 507 | } 508 | } 509 | }() 510 | 511 | return receiver{cancel, received, recs, stopped} 512 | } 513 | 514 | func isTimedOutError(err error) bool { 515 | kafkaError, ok := err.(kafka.Error) 516 | return ok && kafkaError.Code() == kafka.ErrTimedOut 517 | } 518 | 519 | func isFatalError(err error) bool { 520 | kafkaError, ok := err.(kafka.Error) 521 | return ok && kafkaError.IsFatal() 522 | } 523 | 524 | var sigQuitHandlerInstalled = concurrent.NewAtomicCounter() 525 | 526 | func installSigQuitHandler() { 527 | if sigQuitHandlerInstalled.CompareAndSwap(0, 1) { 528 | sig := make(chan os.Signal, 1) 529 | go func() { 530 | signal.Notify(sig, syscall.SIGQUIT) 531 | select { 532 | case <-sig: 533 | scr.I()("Stack\n%s", diags.DumpAllStacks()) 534 | } 535 | }() 536 | } 537 | } 538 | -------------------------------------------------------------------------------- /harvest.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "fmt" 5 | "runtime/debug" 6 | "sync" 7 | "sync/atomic" 8 | "time" 9 | 10 | "github.com/google/uuid" 11 | "github.com/obsidiandynamics/goharvest/metric" 12 | "github.com/obsidiandynamics/goneli" 13 | "github.com/obsidiandynamics/libstdgo/concurrent" 14 | "github.com/obsidiandynamics/libstdgo/diags" 15 | "github.com/obsidiandynamics/libstdgo/scribe" 16 | "gopkg.in/confluentinc/confluent-kafka-go.v1/kafka" 17 | _ "gopkg.in/confluentinc/confluent-kafka-go.v1/kafka/librdkafka" 18 | ) 19 | 20 | var noLeader uuid.UUID 21 | 22 | // State of the Harvest instance. 23 | type State int 24 | 25 | const ( 26 | // Created — initialised (configured) but not started. 27 | Created State = iota 28 | 29 | // Running — currently running. 30 | Running 31 | 32 | // Stopping — in the process of being stopped. I.e. Stop() has been invoked, but workers are still running. 33 | Stopping 34 | 35 | // Stopped — has been completely disposed of. 36 | Stopped 37 | ) 38 | 39 | type tracedPanic struct { 40 | cause interface{} 41 | stack string 42 | } 43 | 44 | func (e tracedPanic) Error() string { 45 | return fmt.Sprintf("%v\n%s", e.cause, e.stack) 46 | } 47 | 48 | // Harvest performs background harvesting of a transactional outbox table. 49 | type Harvest interface { 50 | Start() error 51 | Stop() 52 | Await() error 53 | State() State 54 | IsLeader() bool 55 | LeaderID() *uuid.UUID 56 | InFlightRecords() int 57 | InFlightRecordKeys() []string 58 | SetEventHandler(eventHandler EventHandler) 59 | } 60 | 61 | const watcherTimeout = 60 * time.Second 62 | 63 | type harvest struct { 64 | config Config 65 | producerConfigs KafkaConfigMap 66 | scribe scribe.Scribe 67 | state concurrent.AtomicReference 68 | shouldBeRunningFlag concurrent.AtomicCounter 69 | neli goneli.Neli 70 | leaderID atomic.Value 71 | db DatabaseBinding 72 | queuedRecords concurrent.AtomicCounter 73 | inFlightRecords concurrent.AtomicCounter 74 | inFlightKeys concurrent.Scoreboard 75 | throughput *metric.Meter 76 | throughputLock sync.Mutex 77 | panicCause atomic.Value 78 | eventHandler EventHandler 79 | forceRemarkFlag concurrent.AtomicCounter 80 | sendBattery battery 81 | } 82 | 83 | // New creates a new Harvest instance from the supplied config. 84 | func New(config Config) (Harvest, error) { 85 | config.SetDefaults() 86 | if err := config.Validate(); err != nil { 87 | return nil, err 88 | } 89 | h := &harvest{ 90 | config: config, 91 | scribe: config.Scribe, 92 | state: concurrent.NewAtomicReference(Created), 93 | shouldBeRunningFlag: concurrent.NewAtomicCounter(1), 94 | queuedRecords: concurrent.NewAtomicCounter(), 95 | inFlightRecords: concurrent.NewAtomicCounter(), 96 | inFlightKeys: concurrent.NewScoreboard(*config.Limits.SendConcurrency), 97 | forceRemarkFlag: concurrent.NewAtomicCounter(), 98 | eventHandler: func(e Event) {}, 99 | } 100 | h.leaderID.Store(noLeader) 101 | 102 | h.producerConfigs = copyKafkaConfig(h.config.BaseKafkaConfig) 103 | putAllKafkaConfig(h.config.ProducerKafkaConfig, h.producerConfigs) 104 | err := setKafkaConfigs(h.producerConfigs, KafkaConfigMap{ 105 | "enable.idempotence": true, 106 | }) 107 | if err != nil { 108 | return nil, err 109 | } 110 | 111 | return h, nil 112 | } 113 | 114 | // State obtains the present state of this Harvest instance. 115 | func (h *harvest) State() State { 116 | return h.state.Get().(State) 117 | } 118 | 119 | func (h *harvest) logger() scribe.StdLogAPI { 120 | return h.scribe.Capture(h.scene()) 121 | } 122 | 123 | func (h *harvest) scene() scribe.Scene { 124 | return scribe.Scene{Fields: scribe.Fields{ 125 | "name": h.config.Name, 126 | "lib": "goharvest", 127 | }} 128 | } 129 | func (h *harvest) cleanupFailedStart() { 130 | if h.State() != Created { 131 | return 132 | } 133 | 134 | if h.db != nil { 135 | h.db.Dispose() 136 | } 137 | } 138 | 139 | // Start the harvester. 140 | func (h *harvest) Start() error { 141 | ensureState(h.State() == Created, "Cannot start at this time") 142 | defer h.cleanupFailedStart() 143 | 144 | db, err := h.config.DatabaseBindingProvider(h.config.DataSource, h.config.OutboxTable) 145 | if err != nil { 146 | return err 147 | } 148 | h.db = db 149 | 150 | neliConfig := goneli.Config{ 151 | KafkaConfig: configToNeli(h.config.BaseKafkaConfig), 152 | LeaderTopic: h.config.LeaderTopic, 153 | LeaderGroupID: h.config.LeaderGroupID, 154 | KafkaConsumerProvider: convertKafkaConsumerProvider(h.config.KafkaConsumerProvider), 155 | KafkaProducerProvider: convertKafkaProducerProvider(h.config.KafkaProducerProvider), 156 | Scribe: h.config.Scribe, 157 | Name: h.config.Name, 158 | PollDuration: h.config.Limits.PollDuration, 159 | MinPollInterval: h.config.Limits.MinPollInterval, 160 | HeartbeatTimeout: h.config.Limits.HeartbeatTimeout, 161 | } 162 | h.logger().T()("Creating NELI with config %v", neliConfig) 163 | n, err := h.config.NeliProvider(neliConfig, func(e goneli.Event) { 164 | switch e.(type) { 165 | case goneli.LeaderAcquired: 166 | h.onAcquired() 167 | case goneli.LeaderRevoked: 168 | h.onRevoked() 169 | case goneli.LeaderFenced: 170 | h.onFenced() 171 | } 172 | }) 173 | if err != nil { 174 | return err 175 | } 176 | h.neli = n 177 | 178 | h.throughput = metric.NewMeter("throughput", *h.config.Limits.MinMetricsInterval) 179 | 180 | h.state.Set(Running) 181 | go backgroundPoller(h) 182 | return nil 183 | } 184 | 185 | // IsLeader returns true if the current Harvest is the leader among competing instances. 186 | func (h *harvest) IsLeader() bool { 187 | return h.LeaderID() != nil 188 | } 189 | 190 | // LeaderID returns the leader UUID of the current instance, if it is a leader at the time of this call. 191 | // Otherwise, a nil is returned. 192 | func (h *harvest) LeaderID() *uuid.UUID { 193 | if stored := h.leaderID.Load().(uuid.UUID); stored != noLeader { 194 | return &stored 195 | } 196 | return nil 197 | } 198 | 199 | // InFlightRecords returns the number of in-flight records; i.e. records that have been published on Kafka for which an 200 | // acknowledgement is still pending. 201 | func (h *harvest) InFlightRecords() int { 202 | return h.inFlightRecords.GetInt() 203 | } 204 | 205 | // InFlightRecordKeys returns the keys of records that are still in-flight. For any given key, there will be at most one 206 | // record pending acknowledgement. 207 | func (h *harvest) InFlightRecordKeys() []string { 208 | view := h.inFlightKeys.View() 209 | keys := make([]string, len(view)) 210 | 211 | i := 0 212 | for k := range view { 213 | keys[i] = k 214 | i++ 215 | } 216 | return keys 217 | } 218 | 219 | // SetEventHandler assigns an optional event handler callback to be notified of changes in leader state as well as other 220 | // events of interest. 221 | // 222 | // This method must be invoked prior to Start(). 223 | func (h *harvest) SetEventHandler(eventHandler EventHandler) { 224 | ensureState(h.State() == Created, "Cannot set event handler at this time") 225 | h.eventHandler = eventHandler 226 | } 227 | 228 | func (h *harvest) shouldBeRunning() bool { 229 | return h.shouldBeRunningFlag.Get() == 1 230 | } 231 | 232 | func (h *harvest) reportPanic(goroutineName string) { 233 | if r := recover(); r != nil { 234 | h.logger().E()("Caught panic in %s: %v", goroutineName, r) 235 | h.panicCause.Store(tracedPanic{r, string(debug.Stack())}) 236 | h.logger().E()(string(debug.Stack())) 237 | h.Stop() 238 | } 239 | } 240 | 241 | func ensureState(expected bool, format string, args ...interface{}) { 242 | if !expected { 243 | panic(fmt.Errorf("state assertion failed: "+format, args...)) 244 | } 245 | } 246 | 247 | func backgroundPoller(h *harvest) { 248 | h.logger().I()("Starting background poller") 249 | defer h.logger().I()("Stopped") 250 | defer h.state.Set(Stopped) 251 | defer h.reportPanic("background poller") 252 | defer h.db.Dispose() 253 | defer h.neli.Close() 254 | defer h.shutdownSendBattery() 255 | defer h.state.Set(Stopping) 256 | defer h.logger().I()("Stopping") 257 | 258 | for h.shouldBeRunning() { 259 | isLeader, err := h.neli.Pulse(1 * time.Millisecond) 260 | if err != nil { 261 | panic(err) 262 | } 263 | 264 | if isLeader { 265 | if h.forceRemarkFlag.Get() == 1 { 266 | h.logger().D()("Remark requested") 267 | h.shutdownSendBattery() 268 | h.refreshLeader() 269 | } 270 | if h.sendBattery == nil { 271 | inFlightRecordsValue := h.inFlightRecords.Get() 272 | ensureState(inFlightRecordsValue == 0, "inFlightRecords=%d", inFlightRecordsValue) 273 | inFlightKeysView := h.inFlightKeys.View() 274 | ensureState(len(inFlightKeysView) == 0, "inFlightKeys=%d", inFlightKeysView) 275 | h.spawnSendBattery() 276 | } 277 | onLeaderPoll(h) 278 | } 279 | } 280 | } 281 | 282 | func (h *harvest) spawnSendBattery() { 283 | ensureState(h.sendBattery == nil, "send battery not nil before spawn") 284 | h.logger().D()("Spawning send battery") 285 | h.sendBattery = newConcurrentBattery(*h.config.Limits.SendConcurrency, *h.config.Limits.SendBuffer, func(records chan OutboxRecord) { 286 | defer h.reportPanic("send cell") 287 | 288 | h.logger().T()("Creating Kafka producer with config %v", h.producerConfigs) 289 | prod, err := h.config.KafkaProducerProvider(&h.producerConfigs) 290 | if err != nil { 291 | panic(err) 292 | } 293 | 294 | deliveryHandlerDone := make(chan int) 295 | go backgroundDeliveryHandler(h, prod, deliveryHandlerDone) 296 | 297 | defer func() { 298 | <-deliveryHandlerDone 299 | }() 300 | defer func() { 301 | go func() { 302 | // A bug in confluent-kafka-go (#463) occasionally causes an indefinite syscall hang in Close(), after it closes 303 | // the Events channel. So we delegate this to a separate goroutine — better an orphaned goroutine than a 304 | // frozen harvester. (The rest of the battery will still unwind normally.) 305 | closeWatcher := h.watch("close producer") 306 | prod.Close() 307 | closeWatcher.End() 308 | }() 309 | }() 310 | 311 | var lastID *int64 312 | for rec := range records { 313 | ensureState(lastID == nil || rec.ID >= *lastID, "discontinuity for key %s: ID %s, lastID: %v", rec.KafkaKey, rec.ID, lastID) 314 | lastID = &rec.ID 315 | 316 | m := &kafka.Message{ 317 | TopicPartition: kafka.TopicPartition{Topic: &rec.KafkaTopic, Partition: kafka.PartitionAny}, 318 | Key: []byte(rec.KafkaKey), 319 | Value: stringPointerToByteArray(rec.KafkaValue), 320 | Opaque: rec, 321 | Headers: toNativeKafkaHeaders(rec.KafkaHeaders), 322 | } 323 | 324 | h.inFlightRecords.Drain(int64(*h.config.Limits.MaxInFlightRecords-1), concurrent.Indefinitely) 325 | 326 | startTime := time.Now() 327 | for { 328 | if h.deadlineExceeded("poll", h.neli.Deadline().Elapsed(), *h.config.Limits.MaxPollInterval) { 329 | break 330 | } 331 | if h.deadlineExceeded("message queueing", time.Now().Sub(startTime), *h.config.Limits.QueueTimeout) { 332 | break 333 | } 334 | if remaining := h.inFlightKeys.Drain(rec.KafkaKey, 0, *h.config.Limits.DrainInterval); remaining <= 0 { 335 | ensureState(remaining == 0, "drain failed: %d remaining in-flight records for key %s", remaining, rec.KafkaKey) 336 | break 337 | } 338 | h.logger().D()("Drain stalled for record %d (key %s)", rec.ID, rec.KafkaKey) 339 | } 340 | 341 | if h.forceRemarkFlag.Get() == 1 { 342 | h.queuedRecords.Dec() 343 | continue 344 | } 345 | 346 | h.inFlightRecords.Inc() 347 | h.queuedRecords.Dec() 348 | h.inFlightKeys.Inc(rec.KafkaKey) 349 | 350 | err := prod.Produce(m, nil) 351 | if err != nil { 352 | h.logger().W()("Error publishing record %v: %v", rec, err) 353 | h.inFlightKeys.Dec(rec.KafkaKey) 354 | h.inFlightRecords.Dec() 355 | h.forceRemarkFlag.Set(1) 356 | } 357 | } 358 | }) 359 | } 360 | 361 | func stringPointerToByteArray(str *string) []byte { 362 | if str != nil { 363 | return []byte(*str) 364 | } 365 | return nil 366 | } 367 | 368 | func (h *harvest) shutdownSendBattery() { 369 | if h.sendBattery != nil { 370 | shutdownWatcher := h.watch("shutdown send battery") 371 | h.logger().D()("Shutting down send battery") 372 | 373 | // Expedite shutdown by raising the remark flag, forcing any queued records to be skipped. 374 | h.forceRemarkFlag.Set(1) 375 | 376 | // Take the battery down, waiting for all goroutines to complete. 377 | h.sendBattery.shutdown() 378 | h.sendBattery = nil 379 | 380 | // Reset flags and counters for next time. 381 | h.forceRemarkFlag.Set(0) 382 | h.inFlightRecords.Set(0) 383 | h.inFlightKeys.Clear() 384 | h.logger().D()("Send battery terminated") 385 | shutdownWatcher.End() 386 | } 387 | } 388 | 389 | func onLeaderPoll(h *harvest) { 390 | markBegin := time.Now() 391 | records, err := h.db.Mark(*h.LeaderID(), *h.config.Limits.MarkQueryRecords) 392 | 393 | if err != nil { 394 | h.logger().W()("Error executing mark query: %v", err) 395 | // When an error occurs during marking, we cannot just backoff and retry, as the error could have 396 | // occurred on the return leg (i.e. DB operation succeeded on the server, but timed out on the client). 397 | h.forceRemarkFlag.Set(1) 398 | time.Sleep(*h.config.Limits.IOErrorBackoff) 399 | return 400 | } 401 | 402 | if len(records) > 0 { 403 | sendBegin := time.Now() 404 | h.logger().T()("Leader poll: marked %d in the range %d-%d, took %v", 405 | len(records), records[0].ID, records[len(records)-1].ID, sendBegin.Sub(markBegin)) 406 | 407 | enqueueWatcher := h.watch("enqueue marked records") 408 | for _, rec := range records { 409 | h.queuedRecords.Inc() 410 | h.sendBattery.enqueue(rec) 411 | } 412 | enqueueWatcher.End() 413 | h.logger().T()("Send took %v", time.Now().Sub(sendBegin)) 414 | } else { 415 | time.Sleep(*h.config.Limits.MarkBackoff) 416 | } 417 | } 418 | 419 | func (h *harvest) watch(operation string) *diags.Watcher { 420 | return diags.Watch(operation, watcherTimeout, diags.Print(h.logger().W())) 421 | } 422 | 423 | func (h *harvest) refreshLeader() { 424 | newLeaderID, _ := uuid.NewRandom() 425 | h.leaderID.Store(newLeaderID) 426 | h.logger().W()("Refreshed leader ID: %v", newLeaderID) 427 | h.eventHandler(LeaderRefreshed{newLeaderID}) 428 | } 429 | 430 | func (h *harvest) deadlineExceeded(deadline string, elapsed time.Duration, threshold time.Duration) bool { 431 | if excess := elapsed - threshold; excess > 0 { 432 | if h.forceRemarkFlag.CompareAndSwap(0, 1) { 433 | h.logger().W()("Exceeded %s deadline by %v", deadline, excess) 434 | } 435 | return true 436 | } 437 | return false 438 | } 439 | 440 | func backgroundDeliveryHandler(h *harvest, prod KafkaProducer, done chan int) { 441 | h.logger().I()("Starting background delivery handler") 442 | defer h.reportPanic("background delivery handler") 443 | defer close(done) 444 | 445 | for e := range prod.Events() { 446 | switch ev := e.(type) { 447 | case *kafka.Message: 448 | rec := ev.Opaque.(OutboxRecord) 449 | if ev.TopicPartition.Error != nil { 450 | onFailedDelivery(h, rec, ev.TopicPartition.Error) 451 | } else { 452 | onSuccessfulDelivery(h, rec) 453 | h.updateStats() 454 | } 455 | default: 456 | h.logger().I()("Observed event: %v (%T)", e, e) 457 | } 458 | } 459 | } 460 | 461 | func (h *harvest) updateStats() { 462 | h.throughputLock.Lock() 463 | defer h.throughputLock.Unlock() 464 | h.throughput.MaybeStatsCall(func(stats metric.MeterStats) { 465 | h.logger().D()("%v", stats) 466 | h.eventHandler(MeterRead{stats}) 467 | }) 468 | h.throughput.Add(1) 469 | } 470 | 471 | func onSuccessfulDelivery(h *harvest, rec OutboxRecord) { 472 | for { 473 | done, err := h.db.Purge(rec.ID) 474 | if err == nil { 475 | if !done { 476 | h.logger().W()("Did not purge record %v", rec) 477 | } 478 | break 479 | } 480 | h.logger().W()("Error executing purge query for record %v: %v", rec, err) 481 | time.Sleep(*h.config.Limits.IOErrorBackoff) 482 | } 483 | h.inFlightKeys.Dec(rec.KafkaKey) 484 | h.inFlightRecords.Dec() 485 | } 486 | 487 | func onFailedDelivery(h *harvest, rec OutboxRecord, err error) { 488 | h.logger().W()("Delivery failed for %v, err: %v", rec, err) 489 | for { 490 | done, err := h.db.Reset(rec.ID) 491 | if err == nil { 492 | if !done { 493 | h.logger().W()("Did not reset record %v", rec) 494 | } else { 495 | h.forceRemarkFlag.Set(1) 496 | } 497 | break 498 | } 499 | h.logger().W()("Error executing reset query for record %v: %v", rec, err) 500 | time.Sleep(*h.config.Limits.IOErrorBackoff) 501 | } 502 | h.inFlightKeys.Dec(rec.KafkaKey) 503 | h.inFlightRecords.Dec() 504 | } 505 | 506 | func (h *harvest) onAcquired() { 507 | newLeaderID, _ := uuid.NewRandom() 508 | h.leaderID.Store(newLeaderID) 509 | h.logger().I()("Elected as leader, ID: %v", newLeaderID) 510 | h.eventHandler(LeaderAcquired{newLeaderID}) 511 | } 512 | 513 | func (h *harvest) onRevoked() { 514 | h.logger().I()("Lost leader status") 515 | h.cleanupLeaderState() 516 | h.eventHandler(LeaderRevoked{}) 517 | } 518 | 519 | func (h *harvest) onFenced() { 520 | h.logger().W()("Leader fenced") 521 | h.cleanupLeaderState() 522 | h.eventHandler(LeaderFenced{}) 523 | } 524 | 525 | func (h *harvest) cleanupLeaderState() { 526 | h.shutdownSendBattery() 527 | h.leaderID.Store(noLeader) 528 | } 529 | 530 | // Stop the harvester, returning immediately. 531 | // 532 | // This method does not wait until the underlying Goroutines have been terminated 533 | // and all resources have been disposed off properly. This is accomplished by calling Await() 534 | func (h *harvest) Stop() { 535 | h.shouldBeRunningFlag.Set(0) 536 | } 537 | 538 | // Await the termination of this Harvest instance. 539 | // 540 | // This method blocks indefinitely, returning only when this instance has completed an orderly shutdown. I.e. 541 | // when all Goroutines have returned and all resources have been disposed of. 542 | func (h *harvest) Await() error { 543 | h.state.Await(concurrent.RefEqual(Stopped), concurrent.Indefinitely) 544 | panicCause := h.panicCause.Load() 545 | if panicCause != nil { 546 | return panicCause.(tracedPanic) 547 | } 548 | return nil 549 | } 550 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | logo  2 | === 3 | ![Go version](https://img.shields.io/github/go-mod/go-version/obsidiandynamics/goharvest) 4 | [![Build](https://travis-ci.org/obsidiandynamics/goharvest.svg?branch=master) ](https://travis-ci.org/obsidiandynamics/goharvest#) 5 | ![Release](https://img.shields.io/github/v/release/obsidiandynamics/goharvest?color=ff69b4) 6 | [![Codecov](https://codecov.io/gh/obsidiandynamics/goharvest/branch/master/graph/badge.svg)](https://codecov.io/gh/obsidiandynamics/goharvest) 7 | [![Go Report Card](https://goreportcard.com/badge/github.com/obsidiandynamics/goharvest)](https://goreportcard.com/report/github.com/obsidiandynamics/goharvest) 8 | [![Total alerts](https://img.shields.io/lgtm/alerts/g/obsidiandynamics/goharvest.svg?logo=lgtm&logoWidth=18)](https://lgtm.com/projects/g/obsidiandynamics/goharvest/alerts/) 9 | [![GoDoc Reference](https://img.shields.io/badge/docs-GoDoc-blue.svg)](https://pkg.go.dev/github.com/obsidiandynamics/goharvest?tab=doc) 10 | 11 | `goharvest` is a Go implementation of the [Transactional Outbox](https://microservices.io/patterns/data/transactional-outbox.html) pattern for Postgres and Kafka. 12 | 13 | Transactional Outbox 14 | 15 | While `goharvest` is a complex beast, the end result is dead simple: to publish Kafka messages reliably and atomically, simply write a record to a dedicated **outbox table** in a transaction, alongside any other database changes. (Outbox schema provided below.) `goharvest` scrapes the outbox table in the background and publishes records to a Kafka topic of the application's choosing, using the key, value and headers specified in the outbox record. `goharvest` currently works with Postgres. It maintains causal order of messages and does not require CDC to be enabled on the database, making for a zero-hassle setup. It handles thousands of records/second on commodity hardware. 16 | 17 | # Getting started 18 | ## 1. Create an outbox table for your application 19 | ```sql 20 | CREATE TABLE IF NOT EXISTS outbox ( 21 | id BIGSERIAL PRIMARY KEY, 22 | create_time TIMESTAMP WITH TIME ZONE NOT NULL, 23 | kafka_topic VARCHAR(249) NOT NULL, 24 | kafka_key VARCHAR(100) NOT NULL, -- pick your own maximum key size 25 | kafka_value VARCHAR(10000), -- pick your own maximum value size 26 | kafka_header_keys TEXT[] NOT NULL, 27 | kafka_header_values TEXT[] NOT NULL, 28 | leader_id UUID 29 | ) 30 | ``` 31 | 32 | ## 2. Run `goharvest` 33 | ### Standalone mode 34 | This runs `goharvest` within a separate process called `reaper`, which will work alongside **any** application that writes to a standard outbox. (Not just applications written in Go.) 35 | 36 | #### Install `reaper` 37 | ```sh 38 | go get -u github.com/obsidiandynamics/goharvest/cmd/reaper 39 | ``` 40 | 41 | #### Create `reaper.yaml` configuration 42 | ```yaml 43 | harvest: 44 | baseKafkaConfig: 45 | bootstrap.servers: localhost:9092 46 | producerKafkaConfig: 47 | compression.type: lz4 48 | delivery.timeout.ms: 10000 49 | leaderTopic: my-app-name 50 | leaderGroupID: my-app-name 51 | dataSource: host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable 52 | outboxTable: outbox 53 | limits: 54 | minPollInterval: 1s 55 | heartbeatTimeout: 5s 56 | maxInFlightRecords: 1000 57 | minMetricsInterval: 5s 58 | sendConcurrency: 4 59 | sendBuffer: 10 60 | logging: 61 | level: Debug 62 | ``` 63 | 64 | #### Start `reaper` 65 | ```sh 66 | reaper -f reaper.yaml 67 | ``` 68 | 69 | ### Embedded mode 70 | `goharvest` can be run in the same process as your application. 71 | 72 | #### Add the dependency 73 | ```sh 74 | go get -u github.com/obsidiandynamics/goharvest 75 | ``` 76 | 77 | #### Create and start a `Harvest` instance 78 | ```go 79 | import "github.com/obsidiandynamics/goharvest" 80 | ``` 81 | 82 | ```go 83 | // Configure the harvester. It will use its own database and Kafka connections under the hood. 84 | config := Config{ 85 | BaseKafkaConfig: KafkaConfigMap{ 86 | "bootstrap.servers": "localhost:9092", 87 | }, 88 | DataSource: "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable", 89 | } 90 | 91 | // Create a new harvester. 92 | harvest, err := New(config) 93 | if err != nil { 94 | panic(err) 95 | } 96 | 97 | // Start harvesting in the background. 98 | err = harvest.Start() 99 | if err != nil { 100 | panic(err) 101 | } 102 | 103 | // Wait indefinitely for the harvester to end. 104 | log.Fatal(harvest.Await()) 105 | ``` 106 | 107 | ### Using a custom logger 108 | `goharvest` uses `log.Printf` for output by default. Logger configuration is courtesy of the Scribe façade, from [libstdgo](https://github.com/obsidiandynamics/libstdgo). The example below uses a Logrus binding for Scribe. 109 | 110 | ```go 111 | import ( 112 | "github.com/obsidiandynamics/goharvest" 113 | scribelogrus "github.com/obsidiandynamics/libstdgo/scribe/logrus" 114 | "github.com/sirupsen/logrus" 115 | ) 116 | ``` 117 | 118 | ```sh 119 | log := logrus.StandardLogger() 120 | log.SetLevel(logrus.DebugLevel) 121 | 122 | // Configure the custom logger using a binding. 123 | config := Config{ 124 | BaseKafkaConfig: KafkaConfigMap{ 125 | "bootstrap.servers": "localhost:9092", 126 | }, 127 | Scribe: scribe.New(scribelogrus.Bind()), 128 | DataSource: "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable", 129 | } 130 | ``` 131 | 132 | ### Listening for leader status updates 133 | Just like `goharvest` uses [NELI](https://github.com/obsidiandynamics/goneli) to piggy-back on Kafka's leader election, you can piggy-back on `goharvest` to get leader status updates: 134 | 135 | ```go 136 | log := logrus.StandardLogger() 137 | log.SetLevel(logrus.TraceLevel) 138 | config := Config{ 139 | BaseKafkaConfig: KafkaConfigMap{ 140 | "bootstrap.servers": "localhost:9092", 141 | }, 142 | DataSource: "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable", 143 | Scribe: scribe.New(scribelogrus.Bind()), 144 | } 145 | 146 | // Create a new harvester and register an event hander. 147 | harvest, err := New(config) 148 | 149 | // Register a handler callback, invoked when an event occurs within goharvest. 150 | // The callback is completely optional; it lets the application piggy-back on leader 151 | // status updates, in case it needs to schedule some additional work (other than 152 | // harvesting outbox records) that should only be run on one process at any given time. 153 | harvest.SetEventHandler(func(e Event) { 154 | switch event := e.(type) { 155 | case LeaderAcquired: 156 | // The application may initialise any state necessary to perform work as a leader. 157 | log.Infof("Got event: leader acquired: %v", event.LeaderID()) 158 | case LeaderRefreshed: 159 | // Indicates that a new leader ID was generated, as a result of having to remark 160 | // a record (typically as due to an earlier delivery error). This is purely 161 | // informational; there is nothing an application should do about this, other 162 | // than taking note of the new leader ID if it has come to rely on it. 163 | log.Infof("Got event: leader refreshed: %v", event.LeaderID()) 164 | case LeaderRevoked: 165 | // The application may block the callback until it wraps up any in-flight 166 | // activity. Only upon returning from the callback, will a new leader be elected. 167 | log.Infof("Got event: leader revoked") 168 | case LeaderFenced: 169 | // The application must immediately terminate any ongoing activity, on the assumption 170 | // that another leader may be imminently elected. Unlike the handling of LeaderRevoked, 171 | // blocking in the callback will not prevent a new leader from being elected. 172 | log.Infof("Got event: leader fenced") 173 | case MeterRead: 174 | // Periodic statistics regarding the harvester's throughput. 175 | log.Infof("Got event: meter read: %v", event.Stats()) 176 | } 177 | }) 178 | 179 | // Start harvesting in the background. 180 | err = harvest.Start() 181 | ``` 182 | 183 | ### Which mode should I use 184 | Running `goharvest` in standalone mode using `reaper` is the recommended approach for most use cases, as it fully insulates the harvester from the rest of the application. Ideally, you should deploy `reaper` as a sidecar daemon, to run alongside your application. All the reaper needs is access to the outbox table and the Kafka cluster. 185 | 186 | Embedded `goharvest` is useful if you require additional insights into its operation, which is accomplished by registering an `EventHandler` callback, as shown in the example above. This callback is invoked whenever the underlying leader status changes, which may be useful if you need to schedule additional workloads that should only be run on one process at any given time. 187 | 188 | ## 3. Write outbox records 189 | ### Directly, using SQL 190 | You can write database records from any app, by simply issuing the following `INSERT` statement: 191 | 192 | ```sql 193 | INSERT INTO ${outbox_table} ( 194 | create_time, 195 | kafka_topic, 196 | kafka_key, 197 | kafka_value, 198 | kafka_header_keys, 199 | kafka_header_values 200 | ) 201 | VALUES (NOW(), $1, $2, $3, $4, $5) 202 | ``` 203 | 204 | Replace `${outbox_table}` and bind the query variables as appropriate: 205 | 206 | * `kafka_topic` column specifies an arbitrary topic name, which may differ among records. 207 | * `kafka_key` is a mandatory `string` key. Each record must be published with a specified key, which will affect its placement among the topic's partitions. 208 | * `kafka_value` is an optional `string` value. If unspecified, the record will be published with a `nil` value, allowing it to be used as a compaction tombstone. 209 | * `kafka_header_keys` and `kafka_header_values` are arrays that specify the keys and values of record headers. When used each element in `kafka_header_keys` corresponds to an element in `kafka_header_values` at the same index. If not using headers, set both arrays to empty. 210 | 211 | > **Note**: **Writing outbox records should be performed in the same transaction as other related database updates.** Otherwise, messaging will not be atomic — the updates may be stably persisted while the message might be lost, and *vice versa*. 212 | 213 | ### Using `stasher` 214 | The `goharvest` library comes with a `stasher` helper package for writing records to an outbox. 215 | 216 | #### One-off messages 217 | When one database update corresponds to one message, the easiest approach is to call `Stasher.Stash()`: 218 | 219 | ```go 220 | import "github.com/obsidiandynamics/goharvest" 221 | ``` 222 | 223 | ```go 224 | db, err := sql.Open("postgres", "host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable") 225 | if err != nil { 226 | panic(err) 227 | } 228 | defer db.Close() 229 | 230 | st := New("outbox") 231 | 232 | // Begin a transaction. 233 | tx, _ := db.Begin() 234 | defer tx.Rollback() 235 | 236 | // Update other database entities in transaction scope. 237 | 238 | // Stash an outbox record for subsequent harvesting. 239 | err = st.Stash(tx, goharvest.OutboxRecord{ 240 | KafkaTopic: "my-app.topic", 241 | KafkaKey: "hello", 242 | KafkaValue: goharvest.String("world"), 243 | KafkaHeaders: goharvest.KafkaHeaders{ 244 | {Key: "applicationId", Value: "my-app"}, 245 | }, 246 | }) 247 | if err != nil { 248 | panic(err) 249 | } 250 | 251 | // Commit the transaction. 252 | tx.Commit() 253 | ``` 254 | 255 | #### Multiple messages 256 | Sending multiple messages within a single transaction may be done more efficiently using prepared statements: 257 | 258 | ```go 259 | // Begin a transaction. 260 | tx, _ := db.Begin() 261 | defer tx.Rollback() 262 | 263 | // Update other database entities in transaction scope. 264 | // ... 265 | 266 | // Formulates a prepared statement that may be reused within the scope of the transaction. 267 | prestash, _ := st.Prepare(tx) 268 | 269 | // Publish a bunch of messages using the same prepared statement. 270 | for i := 0; i < 10; i++ { 271 | // Stash an outbox record for subsequent harvesting. 272 | err = prestash.Stash(goharvest.OutboxRecord{ 273 | KafkaTopic: "my-app.topic", 274 | KafkaKey: "hello", 275 | KafkaValue: goharvest.String("world"), 276 | KafkaHeaders: goharvest.KafkaHeaders{ 277 | {Key: "applicationId", Value: "my-app"}, 278 | }, 279 | }) 280 | if err != nil { 281 | panic(err) 282 | } 283 | } 284 | 285 | // Commit the transaction. 286 | tx.Commit() 287 | ``` 288 | 289 | # Configuration 290 | There are handful of parameters that for configuring `goharvest`, assigned via the `Config` struct: 291 | 292 | 293 | 294 | 295 | 296 | 297 | 298 | 299 | 300 | 301 | 302 | 303 | 304 | 305 | 306 | 307 | 308 | 309 | 310 | 311 | 312 | 313 | 314 | 315 | 316 | 317 | 318 | 319 | 320 | 321 | 322 | 323 | 324 | 325 | 326 | 327 | 328 | 329 | 330 | 331 | 332 | 333 | 334 | 335 | 336 | 337 | 338 | 339 | 340 | 341 | 342 | 343 | 344 | 345 | 346 | 347 | 348 | 349 | 350 | 351 | 352 | 353 | 354 | 355 | 356 | 357 | 358 | 359 | 360 | 361 | 362 | 363 | 364 | 365 | 366 | 367 | 368 | 369 | 370 | 371 | 372 | 373 | 374 | 375 | 376 | 377 | 378 | 379 | 380 | 381 | 382 | 383 | 384 | 385 | 386 | 387 | 388 | 389 | 390 | 391 | 392 |
ParameterDefault valueDescription
BaseKafkaConfigMap containing bootstrap.servers=localhost:9092.Configuration shared by the underlying Kafka producer and consumer clients, including those used for leader election.
ProducerKafkaConfigEmpty map.Additional configuration on top of BaseKafkaConfig that is specific to the producer clients created by goharvest for publishing harvested messages. This configuration does not apply to the underlying NELI leader election protocol.
LeaderGroupIDAssumes the filename of the application binary.Used by the underlying leader election protocol as a unique identifier shared by all instances in a group of competing processes. The LeaderGroupID is used as Kafka group.id property under the hood, when subscribing to the leader election topic.
LeaderTopicAssumes the value of LeaderGroupID, suffixed with the string .neli.Used by NELI as the name of the Kafka topic for orchestrating leader election. Competing processes subscribe to the same topic under an identical consumer group ID, using Kafka's exclusive partition assignment as a mechanism for arbitrating leader status.
DataSourceLocal Postgres data source host=localhost port=5432 user=postgres password= dbname=postgres sslmode=disable.The database driver-specific data source string.
OutboxTableoutboxThe name of the outbox table, optionally including the schema name.
ScribeScribe configured with bindings for log.Printf(); effectively the result of running scribe.New(scribe.StandardBinding()).The logging façade used by the library, preconfigured with your logger of choice. See Scribe GoDocs.
NameA string in the form {hostname}_{pid}_{time}, where {hostname} is the result of invoking os.Hostname(), {pid} is the process ID, and {time} is the UNIX epoch time, in seconds.The symbolic name of this instance. This field is informational only, accompanying all log messages.
Limits.MinPollInterval100 msThe lower bound on the poll interval, preventing the over-polling of Kafka on successive Pulse() invocations. Assuming Pulse() is called repeatedly by the application, NELI may poll Kafka at a longer interval than MinPollInterval. (Regular polling is necessary to prove client's liveness and maintain internal partition assignment, but polling excessively is counterproductive.)
Limits.HeartbeatTimeout5 sThe period that a leader will maintain its status, not having received a heartbeat message on the leader topic. After the timeout elapses, the leader will assume a network partition and will voluntarily yield its status, signalling a LeaderFenced event to the application.
Limits.QueueTimeout30 sThe maximum period of time a record may be queued after having been marked, before timing out and triggering a remark.
Limits.MarkBackoff10 msThe backoff delay introduced by the mark thread when a query returns no results, indicating the absence of backlogged records. A mark backoff prevents aggressive querying of the database in the absence of a steady flow of outbox records.
Limits.IOErrorBackoff500 msThe backoff delay introduced when any of the mark, purge or reset queries encounter a database error.
Limits.MaxInFlightRecords1000An upper bound on the number of marked records that may be in flight at any given time. I.e. the number of records that have been enqueued with a producer client, for which acknowledgements have yet to be received.
Limits.SendConcurrency8The number of concurrent shards used for queuing causally unrelated records. Each shard is equipped with a dedicated producer client, allowing for its records to be sent independently of other shards.
Limits.SendBuffer10The maximum number of marked records that may be buffered for subsequent sending, for any given shard. When the buffer is full, the marker will halt — waiting for records to be sent and for their acknowledgements to flow through.
Limits.MarkQueryRecords100An upper bound on the number of records that may be marked in any given query. Limiting this number avoids long-running database queries.
Limits.MinMetricsInterval5 sThe minimum interval at which throughput metrics are emitted. Metrics are emitted conservatively and may be observed less frequently; in fact, throughput metrics are only emitted upon a successful message acknowledgement, which will not occur during periods of inactivity.
393 | 394 | # Docs 395 | [Design](https://github.com/obsidiandynamics/goharvest/wiki/Design) 396 | 397 | [Comparison of messaging patterns](https://github.com/obsidiandynamics/goharvest/wiki/Comparison-of-messaging-patterns) 398 | 399 | [Comparison of harvesting methods](https://github.com/obsidiandynamics/goharvest/wiki/Comparison-of-harvesting-methods) 400 | 401 | [FAQ](https://github.com/obsidiandynamics/goharvest/wiki/FAQ) 402 | 403 | 404 | -------------------------------------------------------------------------------- /harvest_test.go: -------------------------------------------------------------------------------- 1 | package goharvest 2 | 3 | import ( 4 | "fmt" 5 | "math" 6 | "strconv" 7 | "sync" 8 | "testing" 9 | "time" 10 | 11 | "github.com/google/uuid" 12 | "github.com/obsidiandynamics/goneli" 13 | "github.com/obsidiandynamics/libstdgo/check" 14 | "github.com/obsidiandynamics/libstdgo/concurrent" 15 | "github.com/obsidiandynamics/libstdgo/scribe" 16 | "github.com/stretchr/testify/assert" 17 | "github.com/stretchr/testify/require" 18 | "gopkg.in/confluentinc/confluent-kafka-go.v1/kafka" 19 | ) 20 | 21 | func wait(t check.Tester) check.Timesert { 22 | return check.Wait(t, 10*time.Second) 23 | } 24 | 25 | // Aggressive limits used for (fast) testing and without send concurrency to simplify assertions. 26 | func testLimits() Limits { 27 | return Limits{ 28 | IOErrorBackoff: Duration(1 * time.Millisecond), 29 | PollDuration: Duration(1 * time.Millisecond), 30 | MinPollInterval: Duration(1 * time.Millisecond), 31 | MaxPollInterval: Duration(60 * time.Second), 32 | HeartbeatTimeout: Duration(60 * time.Second), 33 | DrainInterval: Duration(60 * time.Second), 34 | QueueTimeout: Duration(60 * time.Second), 35 | MarkBackoff: Duration(1 * time.Millisecond), 36 | MaxInFlightRecords: Int(math.MaxInt64), 37 | SendConcurrency: Int(1), 38 | SendBuffer: Int(0), 39 | } 40 | } 41 | 42 | type fixtures struct { 43 | producerMockSetup producerMockSetup 44 | } 45 | 46 | func (f *fixtures) setDefaults() { 47 | if f.producerMockSetup == nil { 48 | f.producerMockSetup = func(prodMock *prodMock) {} 49 | } 50 | } 51 | 52 | type producerMockSetup func(prodMock *prodMock) 53 | 54 | func (f fixtures) create() (scribe.MockScribe, *dbMock, *goneli.MockNeli, Config) { 55 | f.setDefaults() 56 | m := scribe.NewMock() 57 | 58 | db := &dbMock{} 59 | db.fillDefaults() 60 | 61 | var neli goneli.MockNeli 62 | 63 | config := Config{ 64 | Limits: testLimits(), 65 | Scribe: scribe.New(m.Factories()), 66 | DatabaseBindingProvider: mockDatabaseBindingProvider(db), 67 | NeliProvider: func(config goneli.Config, barrier goneli.Barrier) (goneli.Neli, error) { 68 | n, err := goneli.NewMock(goneli.MockConfig{ 69 | MinPollInterval: config.MinPollInterval, 70 | }, barrier) 71 | if err != nil { 72 | panic(err) 73 | } 74 | neli = n 75 | return n, nil 76 | }, 77 | KafkaProducerProvider: func(conf *KafkaConfigMap) (KafkaProducer, error) { 78 | prod := &prodMock{} 79 | prod.fillDefaults() 80 | f.producerMockSetup(prod) 81 | return prod, nil 82 | }, 83 | } 84 | config.Scribe.SetEnabled(scribe.All) 85 | 86 | return m, db, &neli, config 87 | } 88 | 89 | type testEventHandler struct { 90 | mutex sync.Mutex 91 | events []Event 92 | } 93 | 94 | func (c *testEventHandler) handler() EventHandler { 95 | return func(e Event) { 96 | c.mutex.Lock() 97 | defer c.mutex.Unlock() 98 | c.events = append(c.events, e) 99 | } 100 | } 101 | 102 | func (c *testEventHandler) list() []Event { 103 | c.mutex.Lock() 104 | defer c.mutex.Unlock() 105 | eventsCopy := make([]Event, len(c.events)) 106 | copy(eventsCopy, c.events) 107 | return eventsCopy 108 | } 109 | 110 | func (c *testEventHandler) length() int { 111 | c.mutex.Lock() 112 | defer c.mutex.Unlock() 113 | return len(c.events) 114 | } 115 | 116 | func TestCorrectInitialisation(t *testing.T) { 117 | _, db, neli, config := fixtures{}.create() 118 | 119 | var givenDataSource string 120 | var givenOutboxTable string 121 | 122 | config.DatabaseBindingProvider = func(dataSource string, outboxTable string) (DatabaseBinding, error) { 123 | givenDataSource = dataSource 124 | givenOutboxTable = outboxTable 125 | return db, nil 126 | } 127 | config.DataSource = "test data source" 128 | config.OutboxTable = "test table name" 129 | config.LeaderGroupID = "test leader group ID" 130 | config.BaseKafkaConfig = KafkaConfigMap{ 131 | "bootstrap.servers": "localhost:9092", 132 | } 133 | 134 | h, err := New(config) 135 | require.Nil(t, err) 136 | assert.Equal(t, Created, h.State()) 137 | assertNoError(t, h.Start) 138 | assert.Equal(t, Running, h.State()) 139 | 140 | assert.Equal(t, config.DataSource, givenDataSource) 141 | assert.Equal(t, config.OutboxTable, givenOutboxTable) 142 | 143 | h.Stop() 144 | assert.Nil(t, h.Await()) 145 | assert.Equal(t, Stopped, h.State()) 146 | 147 | assert.Equal(t, 1, db.c.Dispose.GetInt()) 148 | assert.Equal(t, goneli.Closed, (*neli).State()) 149 | } 150 | 151 | func TestConfigError(t *testing.T) { 152 | h, err := New(Config{ 153 | Limits: Limits{ 154 | IOErrorBackoff: Duration(-1), 155 | }, 156 | }) 157 | assert.Nil(t, h) 158 | assert.NotNil(t, err) 159 | } 160 | 161 | func TestErrorDuringDBInitialisation(t *testing.T) { 162 | _, _, _, config := fixtures{}.create() 163 | 164 | config.DatabaseBindingProvider = func(dataSource string, outboxTable string) (DatabaseBinding, error) { 165 | return nil, check.ErrSimulated 166 | } 167 | h, err := New(config) 168 | require.Nil(t, err) 169 | 170 | assertErrorContaining(t, h.Start, "simulated") 171 | assert.Equal(t, Created, h.State()) 172 | } 173 | 174 | func TestErrorDuringNeliInitialisation(t *testing.T) { 175 | _, db, _, config := fixtures{}.create() 176 | 177 | config.NeliProvider = func(config goneli.Config, barrier goneli.Barrier) (goneli.Neli, error) { 178 | return nil, check.ErrSimulated 179 | } 180 | h, err := New(config) 181 | require.Nil(t, err) 182 | 183 | assertErrorContaining(t, h.Start, "simulated") 184 | assert.Equal(t, Created, h.State()) 185 | assert.Equal(t, 1, db.c.Dispose.GetInt()) 186 | } 187 | 188 | func TestErrorDuringProducerConfiguration(t *testing.T) { 189 | _, _, _, config := fixtures{}.create() 190 | 191 | config.ProducerKafkaConfig = KafkaConfigMap{ 192 | "enable.idempotence": false, 193 | } 194 | h, err := New(config) 195 | require.NotNil(t, err) 196 | assert.Contains(t, err.Error(), "cannot override configuration 'enable.idempotence'") 197 | assert.Nil(t, h) 198 | } 199 | 200 | func TestErrorDuringProducerInitialisation(t *testing.T) { 201 | m, db, neli, config := fixtures{}.create() 202 | 203 | config.KafkaProducerProvider = func(conf *KafkaConfigMap) (KafkaProducer, error) { 204 | return nil, check.ErrSimulated 205 | } 206 | h, err := New(config) 207 | require.Nil(t, err) 208 | 209 | eh := &testEventHandler{} 210 | h.SetEventHandler(eh.handler()) 211 | assertNoError(t, h.Start) 212 | 213 | // Induce leadership and wait until leader. 214 | (*neli).AcquireLeader() 215 | wait(t).Until(h.IsLeader) 216 | wait(t).UntilAsserted(func(t check.Tester) { 217 | assert.Equal(t, 1, eh.length()) 218 | }) 219 | 220 | wait(t).UntilAsserted(m.ContainsEntries(). 221 | Having(scribe.LogLevel(scribe.Error)). 222 | Having(scribe.MessageEqual("Caught panic in send cell: simulated")). 223 | Passes(scribe.Count(1))) 224 | 225 | // Having detected a panic, it should self-destruct 226 | assertErrorContaining(t, h.Await, "simulated") 227 | 228 | assert.Equal(t, 1, db.c.Dispose.GetInt()) 229 | assert.Equal(t, (*neli).State(), goneli.Closed) 230 | } 231 | 232 | func TestUncaughtPanic_backgroundPoller(t *testing.T) { 233 | m, _, neli, config := fixtures{}.create() 234 | 235 | h, err := New(config) 236 | require.Nil(t, err) 237 | eh := &testEventHandler{} 238 | h.SetEventHandler(eh.handler()) 239 | assertNoError(t, h.Start) 240 | 241 | (*neli).PulseError(check.ErrSimulated) 242 | 243 | // Having detected a panic, it should self-destruct 244 | assertErrorContaining(t, h.Await, "simulated") 245 | assert.Equal(t, 0, eh.length()) 246 | 247 | t.Log(m.Entries().List()) 248 | m.Entries(). 249 | Having(scribe.LogLevel(scribe.Info)). 250 | Having(scribe.MessageEqual("Starting background poller")). 251 | Assert(t, scribe.Count(1)) 252 | 253 | m.Entries(). 254 | Having(scribe.LogLevel(scribe.Error)). 255 | Having(scribe.MessageEqual("Caught panic in background poller: simulated")). 256 | Assert(t, scribe.Count(1)) 257 | } 258 | 259 | func TestUncaughtPanic_backgroundDeliveryHandler(t *testing.T) { 260 | prodRef := concurrent.NewAtomicReference() 261 | m, db, neli, config := fixtures{producerMockSetup: func(prodMock *prodMock) { 262 | prodRef.Set(prodMock) 263 | }}.create() 264 | 265 | db.f.Reset = func(m *dbMock, id int64) (bool, error) { 266 | panic(check.ErrSimulated) 267 | } 268 | 269 | h, err := New(config) 270 | require.Nil(t, err) 271 | assertNoError(t, h.Start) 272 | 273 | // Induce leadership and await 274 | (*neli).AcquireLeader() 275 | wait(t).Until(h.IsLeader) 276 | 277 | // Feed a delivery event to cause a DB reset query 278 | wait(t).UntilAsserted(isNotNil(prodRef.Get)) 279 | prodRef.Get().(*prodMock).events <- message(OutboxRecord{ID: 777}, check.ErrSimulated) 280 | 281 | // Having detected a panic, it should self-destruct 282 | assertErrorContaining(t, h.Await, "simulated") 283 | 284 | t.Log(m.Entries().List()) 285 | m.Entries(). 286 | Having(scribe.LogLevel(scribe.Info)). 287 | Having(scribe.MessageEqual("Starting background delivery handler")). 288 | Assert(t, scribe.Count(1)) 289 | 290 | m.Entries(). 291 | Having(scribe.LogLevel(scribe.Error)). 292 | Having(scribe.MessageEqual("Caught panic in background delivery handler: simulated")). 293 | Assert(t, scribe.Count(1)) 294 | } 295 | 296 | func TestBasicLeaderElectionAndRevocation(t *testing.T) { 297 | m, _, neli, config := fixtures{}.create() 298 | 299 | h, err := New(config) 300 | require.Nil(t, err) 301 | eh := &testEventHandler{} 302 | h.SetEventHandler(eh.handler()) 303 | assertNoError(t, h.Start) 304 | 305 | // Starts off in a non-leader state 306 | assert.Equal(t, false, h.IsLeader()) 307 | assert.Nil(t, h.LeaderID()) 308 | 309 | // Assign leadership via the rebalance listener and wait for the assignment to take effect 310 | (*neli).AcquireLeader() 311 | wait(t).UntilAsserted(isTrue(h.IsLeader)) 312 | wait(t).UntilAsserted(m.ContainsEntries(). 313 | Having(scribe.LogLevel(scribe.Info)). 314 | Having(scribe.MessageEqual(fmt.Sprintf("Elected as leader, ID: %s", h.LeaderID()))). 315 | Passes(scribe.Count(1))) 316 | m.Reset() 317 | wait(t).UntilAsserted(func(t check.Tester) { 318 | if assert.Equal(t, 1, eh.length()) { 319 | e := eh.list()[0].(LeaderAcquired) 320 | assert.Equal(t, e.LeaderID(), *(h.LeaderID())) 321 | } 322 | }) 323 | 324 | // Revoke leadership via the rebalance listener and await its effect 325 | (*neli).RevokeLeader() 326 | wait(t).UntilAsserted(isFalse(h.IsLeader)) 327 | wait(t).UntilAsserted(m.ContainsEntries(). 328 | Having(scribe.LogLevel(scribe.Info)). 329 | Having(scribe.MessageEqual("Lost leader status")). 330 | Passes(scribe.Count(1))) 331 | wait(t).UntilAsserted(m.ContainsEntries(). 332 | Having(scribe.LogLevel(scribe.Debug)). 333 | Having(scribe.MessageEqual("Shutting down send battery")). 334 | Passes(scribe.Count(1))) 335 | wait(t).UntilAsserted(m.ContainsEntries(). 336 | Having(scribe.LogLevel(scribe.Debug)). 337 | Having(scribe.MessageEqual("Send battery terminated")). 338 | Passes(scribe.Count(1))) 339 | m.Reset() 340 | wait(t).UntilAsserted(func(t check.Tester) { 341 | if assert.Equal(t, 2, eh.length()) { 342 | _ = eh.list()[1].(LeaderRevoked) 343 | } 344 | }) 345 | 346 | // Reassign leadership via the rebalance listener and wait for the assignment to take effect 347 | (*neli).AcquireLeader() 348 | wait(t).UntilAsserted(isTrue(h.IsLeader)) 349 | wait(t).UntilAsserted(m.ContainsEntries(). 350 | Having(scribe.LogLevel(scribe.Info)). 351 | Having(scribe.MessageEqual(fmt.Sprintf("Elected as leader, ID: %s", h.LeaderID()))). 352 | Passes(scribe.Count(1))) 353 | m.Reset() 354 | wait(t).UntilAsserted(func(t check.Tester) { 355 | if assert.Equal(t, 3, eh.length()) { 356 | e := eh.list()[2].(LeaderAcquired) 357 | assert.Equal(t, e.LeaderID(), *(h.LeaderID())) 358 | } 359 | }) 360 | 361 | // Fence the leader 362 | (*neli).FenceLeader() 363 | wait(t).UntilAsserted(isFalse(h.IsLeader)) 364 | wait(t).UntilAsserted(m.ContainsEntries(). 365 | Having(scribe.LogLevel(scribe.Warn)). 366 | Having(scribe.MessageEqual("Leader fenced")). 367 | Passes(scribe.Count(1))) 368 | wait(t).UntilAsserted(m.ContainsEntries(). 369 | Having(scribe.LogLevel(scribe.Debug)). 370 | Having(scribe.MessageEqual("Shutting down send battery")). 371 | Passes(scribe.Count(1))) 372 | wait(t).UntilAsserted(m.ContainsEntries(). 373 | Having(scribe.LogLevel(scribe.Debug)). 374 | Having(scribe.MessageEqual("Send battery terminated")). 375 | Passes(scribe.Count(1))) 376 | m.Reset() 377 | wait(t).UntilAsserted(func(t check.Tester) { 378 | if assert.Equal(t, 4, eh.length()) { 379 | _ = eh.list()[3].(LeaderFenced) 380 | } 381 | }) 382 | 383 | h.Stop() 384 | assert.Nil(t, h.Await()) 385 | } 386 | 387 | func TestMetrics(t *testing.T) { 388 | prodRef := concurrent.NewAtomicReference() 389 | m, _, neli, config := fixtures{producerMockSetup: func(prodMock *prodMock) { 390 | prodRef.Set(prodMock) 391 | }}.create() 392 | config.Limits.MinMetricsInterval = Duration(1 * time.Millisecond) 393 | 394 | h, err := New(config) 395 | require.Nil(t, err) 396 | eh := &testEventHandler{} 397 | h.SetEventHandler(eh.handler()) 398 | assertNoError(t, h.Start) 399 | 400 | // Induce leadership and wait for the leadership event 401 | (*neli).AcquireLeader() 402 | wait(t).UntilAsserted(isNotNil(prodRef.Get)) 403 | wait(t).UntilAsserted(func(t check.Tester) { 404 | assert.Equal(t, 1, eh.length()) 405 | }) 406 | 407 | wait(t).UntilAsserted(func(t check.Tester) { 408 | backlogRecords := generateRecords(1, 0) 409 | deliverAll(backlogRecords, nil, prodRef.Get().(*prodMock).events) 410 | if assert.GreaterOrEqual(t, eh.length(), 2) { 411 | e := eh.list()[1].(MeterRead) 412 | if stats := e.Stats(); assert.NotNil(t, stats) { 413 | assert.Equal(t, stats.Name, "throughput") 414 | } 415 | } 416 | }) 417 | wait(t).UntilAsserted(m.ContainsEntries(). 418 | Having(scribe.LogLevel(scribe.Debug)). 419 | Having(scribe.MessageContaining("throughput")). 420 | Passes(scribe.CountAtLeast(1))) 421 | 422 | h.Stop() 423 | assert.Nil(t, h.Await()) 424 | } 425 | 426 | func TestHandleNonMessageEvent(t *testing.T) { 427 | prodRef := concurrent.NewAtomicReference() 428 | m, _, neli, config := fixtures{producerMockSetup: func(prodMock *prodMock) { 429 | prodRef.Set(prodMock) 430 | }}.create() 431 | config.Limits.MinMetricsInterval = Duration(1 * time.Millisecond) 432 | 433 | h, err := New(config) 434 | require.Nil(t, err) 435 | eh := &testEventHandler{} 436 | h.SetEventHandler(eh.handler()) 437 | assertNoError(t, h.Start) 438 | 439 | // Induce leadership and wait for the leadership event 440 | (*neli).AcquireLeader() 441 | wait(t).UntilAsserted(isNotNil(prodRef.Get)) 442 | prod := prodRef.Get().(*prodMock) 443 | wait(t).UntilAsserted(func(t check.Tester) { 444 | assert.Equal(t, 1, eh.length()) 445 | }) 446 | 447 | prod.events <- kafka.NewError(kafka.ErrAllBrokersDown, "brokers down", false) 448 | 449 | wait(t).UntilAsserted(m.ContainsEntries(). 450 | Having(scribe.LogLevel(scribe.Info)). 451 | Having(scribe.MessageContaining("Observed event: brokers down")). 452 | Passes(scribe.CountAtLeast(1))) 453 | 454 | h.Stop() 455 | assert.Nil(t, h.Await()) 456 | } 457 | 458 | func TestThrottleKeys(t *testing.T) { 459 | prod := concurrent.NewAtomicReference() 460 | lastPublished := concurrent.NewAtomicReference() 461 | m, db, neli, config := fixtures{producerMockSetup: func(pm *prodMock) { 462 | pm.f.Produce = func(m *prodMock, msg *kafka.Message, deliveryChan chan kafka.Event) error { 463 | lastPublished.Set(msg) 464 | return nil 465 | } 466 | prod.Set(pm) 467 | }}.create() 468 | 469 | h, err := New(config) 470 | require.Nil(t, err) 471 | eh := &testEventHandler{} 472 | h.SetEventHandler(eh.handler()) 473 | assertNoError(t, h.Start) 474 | 475 | // Starts off with no backlog. 476 | assert.Equal(t, 0, h.InFlightRecords()) 477 | 478 | // Induce leadership and wait until a producer has been spawned. 479 | (*neli).AcquireLeader() 480 | wait(t).UntilAsserted(isNotNil(prod.Get)) 481 | 482 | const backlog = 10 483 | backlogRecords := generateCyclicKeyedRecords(1, backlog, 0) 484 | db.markedRecords <- backlogRecords 485 | 486 | // Even though we pushed several records through, they all had a common key, so only one should 487 | // should be published. 488 | wait(t).UntilAsserted(intEqual(1, h.InFlightRecords)) 489 | assert.True(t, h.IsLeader()) // should definitely be leader by now 490 | wait(t).UntilAsserted(intEqual(1, prod.Get().(*prodMock).c.Produce.GetInt)) 491 | msg := lastPublished.Get().(*kafka.Message) 492 | assert.Equal(t, msg.Value, []byte(*backlogRecords[0].KafkaValue)) 493 | assert.ElementsMatch(t, h.InFlightRecordKeys(), []string{backlogRecords[0].KafkaKey}) 494 | 495 | // Drain the in-flight record... another one should then be published. 496 | deliverAll(backlogRecords[0:1], nil, prod.Get().(*prodMock).events) 497 | wait(t).UntilAsserted(func(t check.Tester) { 498 | msg := lastPublished.Get() 499 | if assert.NotNil(t, msg) { 500 | assert.Equal(t, msg.(*kafka.Message).Value, []byte(*backlogRecords[1].KafkaValue)) 501 | } 502 | }) 503 | 504 | // Drain the backlog by feeding in delivery confirmations one at a time. 505 | for i := 1; i < backlog; i++ { 506 | wait(t).UntilAsserted(intEqual(1, h.InFlightRecords)) 507 | wait(t).UntilAsserted(func(t check.Tester) { 508 | msg := lastPublished.Get() 509 | if assert.NotNil(t, msg) { 510 | assert.Equal(t, []byte(*backlogRecords[i].KafkaValue), msg.(*kafka.Message).Value) 511 | } 512 | }) 513 | deliverAll(backlogRecords[i:i+1], nil, prod.Get().(*prodMock).events) 514 | } 515 | 516 | // Revoke leadership... 517 | (*neli).RevokeLeader() 518 | 519 | // Wait for the backlog to drain... leadership status will be cleared when done. 520 | wait(t).Until(check.Not(h.IsLeader)) 521 | 522 | wait(t).UntilAsserted(m.ContainsEntries(). 523 | Having(scribe.LogLevel(scribe.Debug)). 524 | Having(scribe.MessageEqual("Shutting down send battery")). 525 | Passes(scribe.Count(1))) 526 | wait(t).UntilAsserted(m.ContainsEntries(). 527 | Having(scribe.LogLevel(scribe.Debug)). 528 | Having(scribe.MessageEqual("Send battery terminated")). 529 | Passes(scribe.Count(1))) 530 | wait(t).UntilAsserted(m.ContainsEntries(). 531 | Having(scribe.LogLevel(scribe.Info)). 532 | Having(scribe.MessageContaining("Lost leader status")). 533 | Passes(scribe.Count(1))) 534 | 535 | assert.Equal(t, backlog, db.c.Purge.GetInt()) 536 | assert.Equal(t, backlog, prod.Get().(*prodMock).c.Produce.GetInt()) 537 | assert.Equal(t, 0, h.InFlightRecords()) 538 | 539 | h.Stop() 540 | assert.Nil(t, h.Await()) 541 | } 542 | 543 | func TestPollDeadlineExceeded(t *testing.T) { 544 | m, db, neli, config := fixtures{}.create() 545 | 546 | config.Limits.DrainInterval = Duration(time.Millisecond) 547 | config.Limits.MaxPollInterval = Duration(time.Millisecond) 548 | h, err := New(config) 549 | require.Nil(t, err) 550 | eh := &testEventHandler{} 551 | h.SetEventHandler(eh.handler()) 552 | assertNoError(t, h.Start) 553 | 554 | // Starts off with no backlog. 555 | assert.Equal(t, 0, h.InFlightRecords()) 556 | 557 | // Induce leadership and wait until a producer has been spawned. 558 | (*neli).AcquireLeader() 559 | 560 | db.markedRecords <- generateCyclicKeyedRecords(1, 2, 0) 561 | 562 | wait(t).UntilAsserted(m.ContainsEntries(). 563 | Having(scribe.LogLevel(scribe.Warn)). 564 | Having(scribe.MessageContaining("Exceeded poll deadline")). 565 | Passes(scribe.Count(1))) 566 | wait(t).UntilAsserted(m.ContainsEntries(). 567 | Having(scribe.LogLevel(scribe.Debug)). 568 | Having(scribe.MessageEqual("Shutting down send battery")). 569 | Passes(scribe.Count(1))) 570 | wait(t).UntilAsserted(m.ContainsEntries(). 571 | Having(scribe.LogLevel(scribe.Debug)). 572 | Having(scribe.MessageEqual("Send battery terminated")). 573 | Passes(scribe.Count(1))) 574 | 575 | h.Stop() 576 | assert.Nil(t, h.Await()) 577 | } 578 | 579 | func TestQueueLimitExceeded(t *testing.T) { 580 | m, db, neli, config := fixtures{}.create() 581 | 582 | config.Limits.DrainInterval = Duration(time.Millisecond) 583 | config.Limits.QueueTimeout = Duration(time.Millisecond) 584 | h, err := New(config) 585 | require.Nil(t, err) 586 | eh := &testEventHandler{} 587 | h.SetEventHandler(eh.handler()) 588 | assertNoError(t, h.Start) 589 | 590 | // Starts off with no backlog. 591 | assert.Equal(t, 0, h.InFlightRecords()) 592 | 593 | // Induce leadership and wait until a producer has been spawned. 594 | (*neli).AcquireLeader() 595 | 596 | db.markedRecords <- generateCyclicKeyedRecords(1, 2, 0) 597 | 598 | wait(t).UntilAsserted(m.ContainsEntries(). 599 | Having(scribe.LogLevel(scribe.Warn)). 600 | Having(scribe.MessageContaining("Exceeded message queueing deadline")). 601 | Passes(scribe.Count(1))) 602 | wait(t).UntilAsserted(m.ContainsEntries(). 603 | Having(scribe.LogLevel(scribe.Debug)). 604 | Having(scribe.MessageEqual("Shutting down send battery")). 605 | Passes(scribe.Count(1))) 606 | wait(t).UntilAsserted(m.ContainsEntries(). 607 | Having(scribe.LogLevel(scribe.Debug)). 608 | Having(scribe.MessageEqual("Send battery terminated")). 609 | Passes(scribe.Count(1))) 610 | 611 | h.Stop() 612 | assert.Nil(t, h.Await()) 613 | } 614 | 615 | func TestDrainInFlightRecords_failedDelivery(t *testing.T) { 616 | prod := concurrent.NewAtomicReference() 617 | lastPublished := concurrent.NewAtomicReference() 618 | m, db, neli, config := fixtures{producerMockSetup: func(pm *prodMock) { 619 | pm.f.Produce = func(m *prodMock, msg *kafka.Message, deliveryChan chan kafka.Event) error { 620 | lastPublished.Set(msg) 621 | return nil 622 | } 623 | prod.Set(pm) 624 | }}.create() 625 | 626 | h, err := New(config) 627 | require.Nil(t, err) 628 | assertNoError(t, h.Start) 629 | 630 | // Starts off with no backlog 631 | assert.Equal(t, 0, h.InFlightRecords()) 632 | 633 | // Induce leadership 634 | (*neli).AcquireLeader() 635 | wait(t).UntilAsserted(isNotNil(prod.Get)) 636 | 637 | // Generate a backlog 638 | const backlog = 10 639 | backlogRecords := generateRecords(backlog, 0) 640 | db.markedRecords <- backlogRecords 641 | 642 | // Wait for the backlog to register. 643 | wait(t).UntilAsserted(intEqual(backlog, h.InFlightRecords)) 644 | wait(t).UntilAsserted(intEqual(backlog, prod.Get().(*prodMock).c.Produce.GetInt)) 645 | assert.True(t, h.IsLeader()) // should be leader by now 646 | 647 | // Revoke leadership... this will start the backlog drain. 648 | (*neli).RevokeLeader() 649 | 650 | wait(t).Until(check.Not(h.IsLeader)) 651 | 652 | wait(t).UntilAsserted(m.ContainsEntries(). 653 | Having(scribe.LogLevel(scribe.Debug)). 654 | Having(scribe.MessageEqual("Shutting down send battery")). 655 | Passes(scribe.Count(1))) 656 | wait(t).UntilAsserted(m.ContainsEntries(). 657 | Having(scribe.LogLevel(scribe.Debug)). 658 | Having(scribe.MessageEqual("Send battery terminated")). 659 | Passes(scribe.Count(1))) 660 | wait(t).UntilAsserted(m.ContainsEntries(). 661 | Having(scribe.LogLevel(scribe.Info)). 662 | Having(scribe.MessageContaining("Lost leader status")). 663 | Passes(scribe.Count(1))) 664 | assert.Equal(t, h.InFlightRecords(), 0) 665 | 666 | h.Stop() 667 | assert.Nil(t, h.Await()) 668 | } 669 | 670 | func TestErrorInMarkQuery(t *testing.T) { 671 | m, db, neli, config := fixtures{}.create() 672 | 673 | db.f.Mark = func(m *dbMock, leaderID uuid.UUID, limit int) ([]OutboxRecord, error) { 674 | return nil, check.ErrSimulated 675 | } 676 | 677 | h, err := New(config) 678 | require.Nil(t, err) 679 | assertNoError(t, h.Start) 680 | 681 | // Induce leadership 682 | (*neli).AcquireLeader() 683 | 684 | // Wait for the error to be logged 685 | wait(t).UntilAsserted(m.ContainsEntries(). 686 | Having(scribe.LogLevel(scribe.Warn)). 687 | Having(scribe.MessageContaining("Error executing mark query")). 688 | Passes(scribe.CountAtLeast(1))) 689 | wait(t).UntilAsserted(m.ContainsEntries(). 690 | Having(scribe.LogLevel(scribe.Debug)). 691 | Having(scribe.MessageContaining("Remark requested")). 692 | Passes(scribe.CountAtLeast(1))) 693 | assert.Equal(t, Running, h.State()) 694 | 695 | h.Stop() 696 | assert.Nil(t, h.Await()) 697 | } 698 | 699 | func TestErrorInProduce(t *testing.T) { 700 | prodRef := concurrent.NewAtomicReference() 701 | produceError := concurrent.NewAtomicCounter(1) // 1=true, 0=false 702 | m, db, neli, config := fixtures{producerMockSetup: func(pm *prodMock) { 703 | pm.f.Produce = func(m *prodMock, msg *kafka.Message, deliveryChan chan kafka.Event) error { 704 | if produceError.Get() == 1 { 705 | return kafka.NewError(kafka.ErrFail, "simulated", false) 706 | } 707 | return nil 708 | } 709 | prodRef.Set(pm) 710 | }}.create() 711 | 712 | h, err := New(config) 713 | require.Nil(t, err) 714 | eh := &testEventHandler{} 715 | h.SetEventHandler(eh.handler()) 716 | assertNoError(t, h.Start) 717 | 718 | // Induce leadership 719 | (*neli).AcquireLeader() 720 | wait(t).UntilAsserted(isNotNil(prodRef.Get)) 721 | prod := prodRef.Get().(*prodMock) 722 | prodRef.Set(nil) 723 | 724 | // Mark one record 725 | records := generateRecords(1, 0) 726 | db.markedRecords <- records 727 | 728 | // Wait for the error to be logged 729 | wait(t).UntilAsserted(m.ContainsEntries(). 730 | Having(scribe.LogLevel(scribe.Warn)). 731 | Having(scribe.MessageContaining("Error publishing record")). 732 | Passes(scribe.CountAtLeast(1))) 733 | wait(t).UntilAsserted(m.ContainsEntries(). 734 | Having(scribe.LogLevel(scribe.Warn)). 735 | Having(scribe.MessageContaining("Refreshed leader ID")). 736 | Passes(scribe.CountAtLeast(1))) 737 | m.Reset() 738 | assert.Equal(t, Running, h.State()) 739 | wait(t).UntilAsserted(isNotNil(prodRef.Get)) 740 | prod = prodRef.Get().(*prodMock) 741 | 742 | // Resume normal production... error should clear but the record count should not go up, as 743 | // there can only be one in-flight record for a given key 744 | produceError.Set(0) 745 | db.markedRecords <- records 746 | wait(t).UntilAsserted(intEqual(1, h.InFlightRecords)) 747 | wait(t).UntilAsserted(func(t check.Tester) { 748 | assert.ElementsMatch(t, h.InFlightRecordKeys(), []string{records[0].KafkaKey}) 749 | }) 750 | 751 | if assert.GreaterOrEqual(t, eh.length(), 2) { 752 | _ = eh.list()[0].(LeaderAcquired) 753 | _ = eh.list()[1].(LeaderRefreshed) 754 | } 755 | 756 | // Feed successful delivery report for the first record 757 | prod.events <- message(records[0], nil) 758 | 759 | h.Stop() 760 | assert.Nil(t, h.Await()) 761 | } 762 | 763 | // Tests remarking by feeding through two records for the same key, forcing them to come through in sequence. 764 | // The first is published, but fails upon delivery, which raises the forceRemark flag. 765 | // As the second on is processed, the forceRemark flag raised by the first should be spotted, and a leader 766 | // refresh should occur. 767 | func TestReset(t *testing.T) { 768 | prodRef := concurrent.NewAtomicReference() 769 | lastPublished := concurrent.NewAtomicReference() 770 | m, db, neli, config := fixtures{producerMockSetup: func(pm *prodMock) { 771 | pm.f.Produce = func(m *prodMock, msg *kafka.Message, deliveryChan chan kafka.Event) error { 772 | lastPublished.Set(msg) 773 | return nil 774 | } 775 | prodRef.Set(pm) 776 | }}.create() 777 | 778 | h, err := New(config) 779 | require.Nil(t, err) 780 | eh := &testEventHandler{} 781 | h.SetEventHandler(eh.handler()) 782 | assertNoError(t, h.Start) 783 | 784 | // Induce leadership 785 | (*neli).AcquireLeader() 786 | wait(t).UntilAsserted(isNotNil(prodRef.Get)) 787 | prod := prodRef.Get().(*prodMock) 788 | 789 | // Mark two records for the same key 790 | records := generateCyclicKeyedRecords(1, 2, 0) 791 | db.markedRecords <- records 792 | 793 | // Wait for the backlog to register 794 | wait(t).UntilAsserted(intEqual(1, h.InFlightRecords)) 795 | wait(t).UntilAsserted(func(t check.Tester) { 796 | if msg := lastPublished.Get(); assert.NotNil(t, msg) { 797 | assert.Equal(t, *records[0].KafkaValue, string(msg.(*kafka.Message).Value)) 798 | } 799 | }) 800 | 801 | // Feed an error 802 | prod.events <- message(records[0], check.ErrSimulated) 803 | 804 | // Wait for the error to be logged 805 | wait(t).UntilAsserted(m.ContainsEntries(). 806 | Having(scribe.LogLevel(scribe.Warn)). 807 | Having(scribe.MessageContaining("Delivery failed")). 808 | Passes(scribe.CountAtLeast(1))) 809 | 810 | wait(t).UntilAsserted(m.ContainsEntries(). 811 | Having(scribe.LogLevel(scribe.Warn)). 812 | Having(scribe.MessageContaining("Refreshed leader ID")). 813 | Passes(scribe.CountAtLeast(1))) 814 | m.Reset() 815 | assert.Equal(t, Running, h.State()) 816 | wait(t).UntilAsserted(isNotNil(prodRef.Get)) 817 | 818 | h.Stop() 819 | assert.Nil(t, h.Await()) 820 | } 821 | 822 | func TestErrorInPurgeAndResetQueries(t *testing.T) { 823 | prodRef := concurrent.NewAtomicReference() 824 | m, db, neli, config := fixtures{producerMockSetup: func(pm *prodMock) { 825 | prodRef.Set(pm) 826 | }}.create() 827 | 828 | records := generateRecords(2, 0) 829 | purgeError := concurrent.NewAtomicCounter(1) // 1=true, 0=false 830 | resetError := concurrent.NewAtomicCounter(1) // 1=true, 0=false 831 | db.f.Mark = func(m *dbMock, leaderID uuid.UUID, limit int) ([]OutboxRecord, error) { 832 | if db.c.Mark.Get() == 0 { 833 | return records, nil 834 | } 835 | return []OutboxRecord{}, nil 836 | } 837 | db.f.Purge = func(m *dbMock, id int64) (bool, error) { 838 | if purgeError.Get() == 1 { 839 | return false, check.ErrSimulated 840 | } 841 | return true, nil 842 | } 843 | db.f.Reset = func(m *dbMock, id int64) (bool, error) { 844 | if resetError.Get() == 1 { 845 | return false, check.ErrSimulated 846 | } 847 | return true, nil 848 | } 849 | 850 | h, err := New(config) 851 | require.Nil(t, err) 852 | assertNoError(t, h.Start) 853 | 854 | // Induce leadership and await its registration 855 | (*neli).AcquireLeader() 856 | wait(t).UntilAsserted(isNotNil(prodRef.Get)) 857 | prod := prodRef.Get().(*prodMock) 858 | 859 | wait(t).UntilAsserted(isTrue(h.IsLeader)) 860 | wait(t).UntilAsserted(intEqual(2, h.InFlightRecords)) 861 | 862 | // Feed successful delivery report for the first record 863 | prod.events <- message(records[0], nil) 864 | 865 | // Wait for the error to be logged 866 | wait(t).UntilAsserted(m.ContainsEntries(). 867 | Having(scribe.LogLevel(scribe.Warn)). 868 | Having(scribe.MessageContaining("Error executing purge query for record")). 869 | Passes(scribe.CountAtLeast(1))) 870 | m.Reset() 871 | assert.Equal(t, Running, h.State()) 872 | assert.Equal(t, 2, h.InFlightRecords()) 873 | 874 | // Resume normal production... error should clear 875 | purgeError.Set(0) 876 | wait(t).UntilAsserted(intEqual(1, h.InFlightRecords)) 877 | 878 | // Feed failed delivery report for the first record 879 | prodRef.Get().(*prodMock).events <- message(records[1], kafka.NewError(kafka.ErrFail, "simulated", false)) 880 | 881 | // Wait for the error to be logged 882 | wait(t).UntilAsserted(m.ContainsEntries(). 883 | Having(scribe.LogLevel(scribe.Warn)). 884 | Having(scribe.MessageContaining("Error executing reset query for record")). 885 | Passes(scribe.CountAtLeast(1))) 886 | m.Reset() 887 | assert.Equal(t, Running, h.State()) 888 | assert.Equal(t, 1, h.InFlightRecords()) 889 | 890 | // Resume normal production... error should clear 891 | resetError.Set(0) 892 | wait(t).UntilAsserted(intEqual(0, h.InFlightRecords)) 893 | 894 | h.Stop() 895 | assert.Nil(t, h.Await()) 896 | } 897 | 898 | func TestIncompletePurgeAndResetQueries(t *testing.T) { 899 | prodRef := concurrent.NewAtomicReference() 900 | m, db, neli, config := fixtures{producerMockSetup: func(pm *prodMock) { 901 | prodRef.Set(pm) 902 | }}.create() 903 | 904 | records := generateRecords(2, 0) 905 | db.f.Mark = func(m *dbMock, leaderID uuid.UUID, limit int) ([]OutboxRecord, error) { 906 | if db.c.Mark.Get() == 0 { 907 | return records, nil 908 | } 909 | return []OutboxRecord{}, nil 910 | } 911 | db.f.Purge = func(m *dbMock, id int64) (bool, error) { 912 | return false, nil 913 | } 914 | db.f.Reset = func(m *dbMock, id int64) (bool, error) { 915 | return false, nil 916 | } 917 | 918 | h, err := New(config) 919 | require.Nil(t, err) 920 | assertNoError(t, h.Start) 921 | 922 | // Induce leadership and await its registration 923 | (*neli).AcquireLeader() 924 | wait(t).UntilAsserted(isTrue(h.IsLeader)) 925 | wait(t).UntilAsserted(intEqual(2, h.InFlightRecords)) 926 | wait(t).UntilAsserted(isNotNil(prodRef.Get)) 927 | prod := prodRef.Get().(*prodMock) 928 | 929 | // Feed successful delivery report for the first record 930 | prod.events <- message(records[0], nil) 931 | 932 | // Wait for the warning to be logged 933 | wait(t).UntilAsserted(m.ContainsEntries(). 934 | Having(scribe.LogLevel(scribe.Warn)). 935 | Having(scribe.MessageContaining("Did not purge record")). 936 | Passes(scribe.CountAtLeast(1))) 937 | m.Reset() 938 | assert.Equal(t, Running, h.State()) 939 | wait(t).UntilAsserted(intEqual(1, h.InFlightRecords)) 940 | 941 | // Feed failed delivery report for the first record 942 | prod.events <- message(records[1], kafka.NewError(kafka.ErrFail, "simulated", false)) 943 | 944 | // Wait for the warning to be logged 945 | wait(t).UntilAsserted(m.ContainsEntries(). 946 | Having(scribe.LogLevel(scribe.Warn)). 947 | Having(scribe.MessageContaining("Did not reset record")). 948 | Passes(scribe.CountAtLeast(1))) 949 | m.Reset() 950 | assert.Equal(t, Running, h.State()) 951 | wait(t).UntilAsserted(intEqual(0, h.InFlightRecords)) 952 | 953 | h.Stop() 954 | assert.Nil(t, h.Await()) 955 | } 956 | 957 | func TestEnsureState(t *testing.T) { 958 | check.ThatPanicsAsExpected(t, check.ErrorContaining("must not be false"), func() { 959 | ensureState(false, "must not be false") 960 | }) 961 | 962 | ensureState(true, "must not be false") 963 | } 964 | 965 | func intEqual(expected int, intSupplier func() int) func(t check.Tester) { 966 | return func(t check.Tester) { 967 | assert.Equal(t, expected, intSupplier()) 968 | } 969 | } 970 | 971 | func lengthEqual(expected int, sliceSupplier func() []string) func(t check.Tester) { 972 | return func(t check.Tester) { 973 | assert.Len(t, sliceSupplier(), expected) 974 | } 975 | } 976 | 977 | func atLeast(min int, f func() int) check.Assertion { 978 | return func(t check.Tester) { 979 | assert.GreaterOrEqual(t, f(), min) 980 | } 981 | } 982 | 983 | func isTrue(f func() bool) check.Assertion { 984 | return func(t check.Tester) { 985 | assert.True(t, f()) 986 | } 987 | } 988 | 989 | func isFalse(f func() bool) check.Assertion { 990 | return func(t check.Tester) { 991 | assert.False(t, f()) 992 | } 993 | } 994 | 995 | func isNotNil(f func() interface{}) check.Assertion { 996 | return func(t check.Tester) { 997 | assert.NotNil(t, f()) 998 | } 999 | } 1000 | 1001 | func assertErrorContaining(t *testing.T, f func() error, substr string) { 1002 | err := f() 1003 | if assert.NotNil(t, err) { 1004 | assert.Contains(t, err.Error(), substr) 1005 | } 1006 | } 1007 | 1008 | func assertNoError(t *testing.T, f func() error) { 1009 | err := f() 1010 | require.Nil(t, err) 1011 | } 1012 | 1013 | func newTimedOutError() kafka.Error { 1014 | return kafka.NewError(kafka.ErrTimedOut, "Timed out", false) 1015 | } 1016 | 1017 | func generatePartitions(indexes ...int32) []kafka.TopicPartition { 1018 | parts := make([]kafka.TopicPartition, len(indexes)) 1019 | for i, index := range indexes { 1020 | parts[i] = kafka.TopicPartition{Partition: index} 1021 | } 1022 | return parts 1023 | } 1024 | 1025 | func generateRecords(numRecords int, startID int) []OutboxRecord { 1026 | records := make([]OutboxRecord, numRecords) 1027 | now := time.Now() 1028 | for i := 0; i < numRecords; i++ { 1029 | records[i] = OutboxRecord{ 1030 | ID: int64(startID + i), 1031 | CreateTime: now, 1032 | KafkaTopic: "test_topic", 1033 | KafkaKey: fmt.Sprintf("key-%x", i), 1034 | KafkaValue: String(fmt.Sprintf("value-%x", i)), 1035 | KafkaHeaders: KafkaHeaders{ 1036 | KafkaHeader{Key: "ID", Value: strconv.FormatInt(int64(startID+i), 10)}, 1037 | }, 1038 | } 1039 | } 1040 | return records 1041 | } 1042 | 1043 | func generateCyclicKeyedRecords(numKeys int, numRecords int, startID int) []OutboxRecord { 1044 | records := make([]OutboxRecord, numRecords) 1045 | now := time.Now() 1046 | for i := 0; i < numRecords; i++ { 1047 | records[i] = OutboxRecord{ 1048 | ID: int64(startID + i), 1049 | CreateTime: now, 1050 | KafkaTopic: "test_topic", 1051 | KafkaKey: fmt.Sprintf("key-%x", i%numKeys), 1052 | KafkaValue: String(fmt.Sprintf("value-%x", i)), 1053 | KafkaHeaders: KafkaHeaders{ 1054 | KafkaHeader{Key: "ID", Value: strconv.FormatInt(int64(startID+i), 10)}, 1055 | }, 1056 | } 1057 | } 1058 | return records 1059 | } 1060 | 1061 | func message(record OutboxRecord, err error) *kafka.Message { 1062 | return &kafka.Message{ 1063 | TopicPartition: kafka.TopicPartition{Topic: &record.KafkaTopic, Error: err}, 1064 | Key: []byte(record.KafkaKey), 1065 | Value: stringPointerToByteArray(record.KafkaValue), 1066 | Timestamp: record.CreateTime, 1067 | TimestampType: kafka.TimestampCreateTime, 1068 | Opaque: record, 1069 | } 1070 | } 1071 | 1072 | func deliverAll(records []OutboxRecord, err error, events chan kafka.Event) { 1073 | for _, record := range records { 1074 | events <- message(record, err) 1075 | } 1076 | } 1077 | --------------------------------------------------------------------------------