github.com/Jeffail/benthos/v3@v3.65.0/lib/output/writer/gcp_pubsub.go (about) 1 package writer 2 3 import ( 4 "context" 5 "fmt" 6 "sync" 7 "time" 8 9 "cloud.google.com/go/pubsub" 10 "github.com/Jeffail/benthos/v3/internal/batch" 11 "github.com/Jeffail/benthos/v3/internal/bloblang/field" 12 "github.com/Jeffail/benthos/v3/internal/interop" 13 "github.com/Jeffail/benthos/v3/internal/metadata" 14 "github.com/Jeffail/benthos/v3/lib/log" 15 "github.com/Jeffail/benthos/v3/lib/metrics" 16 "github.com/Jeffail/benthos/v3/lib/types" 17 ) 18 19 //------------------------------------------------------------------------------ 20 21 // GCPPubSubConfig contains configuration fields for the output GCPPubSub type. 22 type GCPPubSubConfig struct { 23 ProjectID string `json:"project" yaml:"project"` 24 TopicID string `json:"topic" yaml:"topic"` 25 MaxInFlight int `json:"max_in_flight" yaml:"max_in_flight"` 26 PublishTimeout string `json:"publish_timeout" yaml:"publish_timeout"` 27 Metadata metadata.ExcludeFilterConfig `json:"metadata" yaml:"metadata"` 28 OrderingKey string `json:"ordering_key" yaml:"ordering_key"` 29 } 30 31 // NewGCPPubSubConfig creates a new Config with default values. 32 func NewGCPPubSubConfig() GCPPubSubConfig { 33 return GCPPubSubConfig{ 34 ProjectID: "", 35 TopicID: "", 36 MaxInFlight: 1, 37 PublishTimeout: "60s", 38 Metadata: metadata.NewExcludeFilterConfig(), 39 OrderingKey: "", 40 } 41 } 42 43 //------------------------------------------------------------------------------ 44 45 // GCPPubSub is a benthos writer.Type implementation that writes messages to a 46 // GCP Pub/Sub topic. 47 type GCPPubSub struct { 48 conf GCPPubSubConfig 49 50 client *pubsub.Client 51 publishTimeout time.Duration 52 metaFilter *metadata.ExcludeFilter 53 54 orderingEnabled bool 55 orderingKey *field.Expression 56 57 topicID *field.Expression 58 topics map[string]*pubsub.Topic 59 topicMut sync.Mutex 60 61 log log.Modular 62 stats metrics.Type 63 } 64 65 // NewGCPPubSub creates a new GCP Cloud Pub/Sub writer.Type. 66 // 67 // Deprecated: use the V2 API instead. 68 func NewGCPPubSub( 69 conf GCPPubSubConfig, 70 log log.Modular, 71 stats metrics.Type, 72 ) (*GCPPubSub, error) { 73 return NewGCPPubSubV2(conf, types.NoopMgr(), log, stats) 74 } 75 76 // NewGCPPubSubV2 creates a new GCP Cloud Pub/Sub writer.Type. 77 func NewGCPPubSubV2( 78 conf GCPPubSubConfig, 79 mgr types.Manager, 80 log log.Modular, 81 stats metrics.Type, 82 ) (*GCPPubSub, error) { 83 client, err := pubsub.NewClient(context.Background(), conf.ProjectID) 84 if err != nil { 85 return nil, err 86 } 87 topic, err := interop.NewBloblangField(mgr, conf.TopicID) 88 if err != nil { 89 return nil, fmt.Errorf("failed to parse topic expression: %v", err) 90 } 91 orderingKey, err := interop.NewBloblangField(mgr, conf.OrderingKey) 92 if err != nil { 93 return nil, fmt.Errorf("failed to parse ordering key: %v", err) 94 } 95 pubTimeout, err := time.ParseDuration(conf.PublishTimeout) 96 if err != nil { 97 return nil, fmt.Errorf("failed to parse publish timeout duration: %w", err) 98 } 99 metaFilter, err := conf.Metadata.Filter() 100 if err != nil { 101 return nil, fmt.Errorf("failed to construct metadata filter: %w", err) 102 } 103 return &GCPPubSub{ 104 conf: conf, 105 log: log, 106 metaFilter: metaFilter, 107 client: client, 108 publishTimeout: pubTimeout, 109 stats: stats, 110 topicID: topic, 111 orderingKey: orderingKey, 112 orderingEnabled: len(conf.OrderingKey) > 0, 113 }, nil 114 } 115 116 // ConnectWithContext attempts to establish a connection to the target GCP 117 // Pub/Sub topic. 118 func (c *GCPPubSub) ConnectWithContext(ctx context.Context) error { 119 c.topicMut.Lock() 120 defer c.topicMut.Unlock() 121 if c.topics != nil { 122 return nil 123 } 124 125 c.topics = map[string]*pubsub.Topic{} 126 c.log.Infof("Sending GCP Cloud Pub/Sub messages to project '%v' and topic '%v'\n", c.conf.ProjectID, c.conf.TopicID) 127 return nil 128 } 129 130 func (c *GCPPubSub) getTopic(ctx context.Context, t string) (*pubsub.Topic, error) { 131 c.topicMut.Lock() 132 defer c.topicMut.Unlock() 133 if c.topics == nil { 134 return nil, types.ErrNotConnected 135 } 136 if t, exists := c.topics[t]; exists { 137 return t, nil 138 } 139 140 topic := c.client.Topic(t) 141 exists, err := topic.Exists(ctx) 142 if err != nil { 143 return nil, fmt.Errorf("failed to validate topic '%v': %v", t, err) 144 } 145 if !exists { 146 return nil, fmt.Errorf("topic '%v' does not exist", t) 147 } 148 topic.PublishSettings.Timeout = c.publishTimeout 149 topic.EnableMessageOrdering = c.orderingEnabled 150 c.topics[t] = topic 151 return topic, nil 152 } 153 154 // Connect attempts to establish a connection to the target GCP Pub/Sub topic. 155 func (c *GCPPubSub) Connect() error { 156 ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) 157 defer cancel() 158 return c.ConnectWithContext(ctx) 159 } 160 161 // WriteWithContext attempts to write message contents to a target topic. 162 func (c *GCPPubSub) WriteWithContext(ctx context.Context, msg types.Message) error { 163 topics := make([]*pubsub.Topic, msg.Len()) 164 if err := msg.Iter(func(i int, _ types.Part) error { 165 var tErr error 166 topics[i], tErr = c.getTopic(ctx, c.topicID.String(i, msg)) 167 return tErr 168 }); err != nil { 169 return err 170 } 171 172 results := make([]*pubsub.PublishResult, msg.Len()) 173 msg.Iter(func(i int, part types.Part) error { 174 topic := topics[i] 175 attr := map[string]string{} 176 c.metaFilter.Iter(part.Metadata(), func(k, v string) error { 177 attr[k] = v 178 return nil 179 }) 180 gmsg := &pubsub.Message{ 181 Data: part.Get(), 182 } 183 if c.orderingEnabled { 184 gmsg.OrderingKey = c.orderingKey.String(i, msg) 185 } 186 if len(attr) > 0 { 187 gmsg.Attributes = attr 188 } 189 results[i] = topic.Publish(ctx, gmsg) 190 return nil 191 }) 192 193 var batchErr *batch.Error 194 for i, r := range results { 195 if _, err := r.Get(ctx); err != nil { 196 if batchErr == nil { 197 batchErr = batch.NewError(msg, err) 198 } 199 batchErr.Failed(i, err) 200 } 201 } 202 if batchErr != nil { 203 return batchErr 204 } 205 return nil 206 } 207 208 // Write attempts to write message contents to a target topic. 209 func (c *GCPPubSub) Write(msg types.Message) error { 210 return c.WriteWithContext(context.Background(), msg) 211 } 212 213 // CloseAsync begins cleaning up resources used by this reader asynchronously. 214 func (c *GCPPubSub) CloseAsync() { 215 go func() { 216 c.topicMut.Lock() 217 defer c.topicMut.Unlock() 218 if c.topics != nil { 219 for _, t := range c.topics { 220 t.Stop() 221 } 222 c.topics = nil 223 } 224 }() 225 } 226 227 // WaitForClose will block until either the reader is closed or a specified 228 // timeout occurs. 229 func (c *GCPPubSub) WaitForClose(time.Duration) error { 230 return nil 231 } 232 233 //------------------------------------------------------------------------------