github.com/rohankumardubey/aresdb@v0.0.2-0.20190517170215-e54e3ca06b9c/subscriber/common/sink/kafka.go (about) 1 // Copyright (c) 2017-2018 Uber Technologies, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package sink 16 17 import ( 18 "fmt" 19 "github.com/Shopify/sarama" 20 "github.com/uber-go/tally" 21 "github.com/uber/aresdb/client" 22 "github.com/uber/aresdb/gateway" 23 memCom "github.com/uber/aresdb/memstore/common" 24 "github.com/uber/aresdb/subscriber/common/rules" 25 "github.com/uber/aresdb/subscriber/config" 26 "github.com/uber/aresdb/utils" 27 "go.uber.org/zap" 28 "strings" 29 "time" 30 ) 31 32 type KafkaPublisher struct { 33 sarama.SyncProducer 34 client.UpsertBatchBuilder 35 36 ServiceConfig config.ServiceConfig 37 JobConfig *rules.JobConfig 38 Scope tally.Scope 39 ClusterName string 40 } 41 42 func NewKafkaPublisher(serviceConfig config.ServiceConfig, jobConfig *rules.JobConfig, cluster string, 43 sinkCfg config.SinkConfig, aresControllerClient gateway.ControllerClient) (Sink, error) { 44 if sinkCfg.GetSinkMode() != config.Sink_Kafka { 45 return nil, fmt.Errorf("Failed to NewKafkaPublisher, wrong sinkMode=%d", sinkCfg.GetSinkMode()) 46 } 47 48 addresses := strings.Split(sinkCfg.KafkaProducerConfig.Brokers, ",") 49 serviceConfig.Logger.Info("Kafka borkers address", zap.Any("brokers", addresses)) 50 51 cfg := sarama.NewConfig() 52 cfg.Producer.RequiredAcks = sarama.WaitForAll // Wait for all in-sync replicas to ack the message 53 if sinkCfg.KafkaProducerConfig.RetryMax > 0 { 54 cfg.Producer.Retry.Max = sinkCfg.KafkaProducerConfig.RetryMax 55 } 56 if sinkCfg.KafkaProducerConfig.TimeoutInSec > 0 { 57 cfg.Producer.Timeout = time.Second * time.Duration(sinkCfg.KafkaProducerConfig.TimeoutInSec) 58 } 59 cfg.Producer.Return.Successes = true 60 61 p, err := sarama.NewSyncProducer(addresses, cfg) 62 if err != nil { 63 return nil, utils.StackError(err, "Unable to initialize Kafka producer") 64 } 65 66 // replace httpSchemaFetcher with gateway client 67 // httpSchemaFetcher := NewHttpSchemaFetcher(httpClient, cfg.Address, metricScope) 68 cachedSchemaHandler := client.NewCachedSchemaHandler( 69 serviceConfig.Logger.Sugar(), 70 serviceConfig.Scope.Tagged(map[string]string{ 71 "job": jobConfig.Name, 72 "aresCluster": cluster, 73 }), aresControllerClient) 74 75 // schema refresh is based on job assignment refresh, so disable at here 76 err = cachedSchemaHandler.Start(0) 77 if err != nil { 78 return nil, err 79 } 80 81 kp := KafkaPublisher{ 82 SyncProducer: p, 83 UpsertBatchBuilder: client.NewUpsertBatchBuilderImpl( 84 serviceConfig.Logger.Sugar(), 85 serviceConfig.Scope.Tagged(map[string]string{ 86 "job": jobConfig.Name, 87 "aresCluster": cluster, 88 }), 89 cachedSchemaHandler), 90 ServiceConfig: serviceConfig, 91 JobConfig: jobConfig, 92 Scope: serviceConfig.Scope.Tagged(map[string]string{ 93 "job": jobConfig.Name, 94 "aresCluster": cluster, 95 }), 96 ClusterName: cluster, 97 } 98 99 return &kp, nil 100 } 101 102 // Shutdown will clean up resources that needs to be cleaned up 103 func (kp *KafkaPublisher) Shutdown() { 104 kp.SyncProducer.Close() 105 } 106 107 // Save saves a batch of row objects into a destination 108 func (kp *KafkaPublisher) Save(destination Destination, rows []client.Row) error { 109 kp.Scope.Gauge("batchSize").Update(float64(len(rows))) 110 111 shards, rowsIgnored := Shard(rows, destination, kp.JobConfig) 112 if rowsIgnored != 0 { 113 kp.Scope.Counter("errors.shard").Inc(int64(rowsIgnored)) 114 } 115 116 msgs := make([]*sarama.ProducerMessage, 0, len(shards)) 117 if shards == nil { 118 // case1: no sharding -- publish rows to random kafka partition 119 kp.buildKafkaMessage(msgs, &rowsIgnored, destination.Table, -1, destination.ColumnNames, rows, destination.AresUpdateModes...) 120 } else { 121 // case2: sharding -- publish rows to specified partition 122 for shardID, rowsInShard := range shards { 123 kp.buildKafkaMessage(msgs, &rowsIgnored, destination.Table, int32(shardID), destination.ColumnNames, rowsInShard, destination.AresUpdateModes...) 124 } 125 } 126 127 saveStart := utils.Now() 128 kp.ServiceConfig.Logger.Debug("saving", zap.Any("rows", rows)) 129 err := kp.SyncProducer.SendMessages(msgs) 130 if err != nil { 131 kp.Scope.Counter("errors.insert").Inc(1) 132 return utils.StackError(err, fmt.Sprintf("Failed to publish rows in table %s, columns: %+v", 133 destination.Table, destination.ColumnNames)) 134 } 135 numRows := len(rows) - rowsIgnored 136 kp.Scope.Timer("latency.ares.save").Record(utils.Now().Sub(saveStart)) 137 kp.Scope.Counter("rowsWritten").Inc(int64(numRows)) 138 kp.Scope.Counter("rowsIgnored").Inc(int64(rowsIgnored)) 139 kp.Scope.Gauge("upsertBatchSize").Update(float64(numRows)) 140 141 return nil 142 } 143 144 // Cluster returns the DB cluster name 145 func (kp *KafkaPublisher) Cluster() string { 146 return kp.ClusterName 147 } 148 149 func (kp *KafkaPublisher) buildKafkaMessage(msgs []*sarama.ProducerMessage, rowsIgnored *int, tableName string, shardID int32, columnNames []string, rows []client.Row, 150 updateModes ...memCom.ColumnUpdateMode) { 151 bytes, numRows, err := kp.UpsertBatchBuilder.PrepareUpsertBatch(tableName, columnNames, updateModes, rows) 152 if err != nil { 153 kp.Scope.Counter("errors.upsertBatchBuild").Inc(1) 154 utils.StackError(err, "Failed to prepare rows in table %s, columns: %+v", 155 tableName, columnNames) 156 } 157 158 msg := sarama.ProducerMessage{ 159 Topic: fmt.Sprintf("%s-%s", kp.Cluster(), tableName), 160 Value: sarama.ByteEncoder(bytes), 161 } 162 163 if shardID >= 0 { 164 msg.Partition = shardID 165 } 166 167 msgs = append(msgs, &msg) 168 *rowsIgnored = *rowsIgnored + (len(rows) - numRows) 169 return 170 }