github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/producer/pulsar/producer.go (about) 1 // Copyright 2020 PingCAP, 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 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package pulsar 15 16 import ( 17 "context" 18 "net/url" 19 "strconv" 20 21 "github.com/apache/pulsar-client-go/pulsar" 22 "github.com/pingcap/failpoint" 23 "github.com/pingcap/log" 24 "github.com/pingcap/ticdc/cdc/sink/codec" 25 cerror "github.com/pingcap/ticdc/pkg/errors" 26 "go.uber.org/zap" 27 ) 28 29 // NewProducer create a pulsar producer. 30 func NewProducer(u *url.URL, errCh chan error) (*Producer, error) { 31 failpoint.Inject("MockPulsar", func() { 32 failpoint.Return(&Producer{ 33 errCh: errCh, 34 partitions: 4, 35 }, nil) 36 }) 37 38 opt, err := parseSinkOptions(u) 39 if err != nil { 40 return nil, cerror.WrapError(cerror.ErrPulsarNewProducer, err) 41 } 42 client, err := pulsar.NewClient(*opt.clientOptions) 43 if err != nil { 44 return nil, cerror.WrapError(cerror.ErrPulsarNewProducer, err) 45 } 46 producer, err := client.CreateProducer(*opt.producerOptions) 47 if err != nil { 48 client.Close() 49 return nil, cerror.WrapError(cerror.ErrPulsarNewProducer, err) 50 } 51 partitions, err := client.TopicPartitions(opt.producerOptions.Topic) 52 if err != nil { 53 client.Close() 54 return nil, cerror.WrapError(cerror.ErrPulsarNewProducer, err) 55 } 56 return &Producer{ 57 errCh: errCh, 58 opt: *opt, 59 client: client, 60 producer: producer, 61 partitions: len(partitions), 62 }, nil 63 } 64 65 // Producer provide a way to send msg to pulsar. 66 type Producer struct { 67 opt Option 68 client pulsar.Client 69 producer pulsar.Producer 70 errCh chan error 71 partitions int 72 } 73 74 func createProperties(message *codec.MQMessage, partition int32) map[string]string { 75 properties := map[string]string{route: strconv.Itoa(int(partition))} 76 properties["ts"] = strconv.FormatUint(message.Ts, 10) 77 properties["type"] = strconv.Itoa(int(message.Type)) 78 properties["protocol"] = strconv.Itoa(int(message.Protocol)) 79 if message.Schema != nil { 80 properties["schema"] = *message.Schema 81 } 82 if message.Table != nil { 83 properties["table"] = *message.Table 84 } 85 return properties 86 } 87 88 // SendMessage send key-value msg to target partition. 89 func (p *Producer) SendMessage(ctx context.Context, message *codec.MQMessage, partition int32) error { 90 p.producer.SendAsync(ctx, &pulsar.ProducerMessage{ 91 Payload: message.Value, 92 Key: string(message.Key), 93 Properties: createProperties(message, partition), 94 EventTime: message.PhysicalTime(), 95 }, p.errors) 96 return nil 97 } 98 99 func (p *Producer) errors(_ pulsar.MessageID, _ *pulsar.ProducerMessage, err error) { 100 if err != nil { 101 select { 102 case p.errCh <- cerror.WrapError(cerror.ErrPulsarSendMessage, err): 103 default: 104 log.Error("error channel is full", zap.Error(err)) 105 } 106 } 107 } 108 109 // SyncBroadcastMessage send key-value msg to all partition. 110 func (p *Producer) SyncBroadcastMessage(ctx context.Context, message *codec.MQMessage) error { 111 for partition := 0; partition < p.partitions; partition++ { 112 _, err := p.producer.Send(ctx, &pulsar.ProducerMessage{ 113 Payload: message.Value, 114 Key: string(message.Key), 115 Properties: createProperties(message, int32(partition)), 116 EventTime: message.PhysicalTime(), 117 }) 118 if err != nil { 119 return cerror.WrapError(cerror.ErrPulsarSendMessage, p.producer.Flush()) 120 } 121 } 122 return nil 123 } 124 125 // Flush flush all in memory msgs to server. 126 func (p *Producer) Flush(_ context.Context) error { 127 return cerror.WrapError(cerror.ErrPulsarSendMessage, p.producer.Flush()) 128 } 129 130 // GetPartitionNum got current topic's partitions size. 131 func (p *Producer) GetPartitionNum() int32 { 132 return int32(p.partitions) 133 } 134 135 // Close close the producer. 136 func (p *Producer) Close() error { 137 err := p.producer.Flush() 138 if err != nil { 139 return cerror.WrapError(cerror.ErrPulsarSendMessage, err) 140 } 141 p.producer.Close() 142 p.client.Close() 143 return nil 144 }