github.com/Jeffail/benthos/v3@v3.65.0/public/service/output.go (about) 1 package service 2 3 import ( 4 "context" 5 "errors" 6 "sync" 7 "time" 8 9 "github.com/Jeffail/benthos/v3/internal/shutdown" 10 "github.com/Jeffail/benthos/v3/lib/message" 11 "github.com/Jeffail/benthos/v3/lib/output" 12 "github.com/Jeffail/benthos/v3/lib/types" 13 ) 14 15 // Output is an interface implemented by Benthos outputs that support single 16 // message writes. Each call to Write should block until either the message has 17 // been successfully or unsuccessfully sent, or the context is cancelled. 18 // 19 // Multiple write calls can be performed in parallel, and the constructor of an 20 // output must provide a MaxInFlight parameter indicating the maximum number of 21 // parallel write calls the output supports. 22 type Output interface { 23 // Establish a connection to the downstream service. Connect will always be 24 // called first when a writer is instantiated, and will be continuously 25 // called with back off until a nil error is returned. 26 // 27 // The provided context remains open only for the duration of the connecting 28 // phase, and should not be used to establish the lifetime of the connection 29 // itself. 30 // 31 // Once Connect returns a nil error the write method will be called until 32 // either ErrNotConnected is returned, or the writer is closed. 33 Connect(context.Context) error 34 35 // Write a message to a sink, or return an error if delivery is not 36 // possible. 37 // 38 // If this method returns ErrNotConnected then write will not be called 39 // again until Connect has returned a nil error. 40 Write(context.Context, *Message) error 41 42 Closer 43 } 44 45 //------------------------------------------------------------------------------ 46 47 // BatchOutput is an interface implemented by Benthos outputs that require 48 // Benthos to batch messages before dispatch in order to improve throughput. 49 // Each call to WriteBatch should block until either all messages in the batch 50 // have been successfully or unsuccessfully sent, or the context is cancelled. 51 // 52 // Multiple write calls can be performed in parallel, and the constructor of an 53 // output must provide a MaxInFlight parameter indicating the maximum number of 54 // parallel batched write calls the output supports. 55 type BatchOutput interface { 56 // Establish a connection to the downstream service. Connect will always be 57 // called first when a writer is instantiated, and will be continuously 58 // called with back off until a nil error is returned. 59 // 60 // Once Connect returns a nil error the write method will be called until 61 // either ErrNotConnected is returned, or the writer is closed. 62 Connect(context.Context) error 63 64 // Write a batch of messages to a sink, or return an error if delivery is 65 // not possible. 66 // 67 // If this method returns ErrNotConnected then write will not be called 68 // again until Connect has returned a nil error. 69 WriteBatch(context.Context, MessageBatch) error 70 71 Closer 72 } 73 74 //------------------------------------------------------------------------------ 75 76 // Implements output.AsyncSink 77 type airGapWriter struct { 78 w Output 79 80 sig *shutdown.Signaller 81 } 82 83 func newAirGapWriter(w Output) output.AsyncSink { 84 return &airGapWriter{w, shutdown.NewSignaller()} 85 } 86 87 func (a *airGapWriter) ConnectWithContext(ctx context.Context) error { 88 return a.w.Connect(ctx) 89 } 90 91 func (a *airGapWriter) WriteWithContext(ctx context.Context, msg types.Message) error { 92 err := a.w.Write(ctx, newMessageFromPart(msg.Get(0))) 93 if err != nil && errors.Is(err, ErrNotConnected) { 94 err = types.ErrNotConnected 95 } 96 return err 97 } 98 99 func (a *airGapWriter) CloseAsync() { 100 go func() { 101 // TODO: Determine whether to continue trying or log/exit. 102 _ = a.w.Close(context.Background()) 103 a.sig.ShutdownComplete() 104 }() 105 } 106 107 func (a *airGapWriter) WaitForClose(tout time.Duration) error { 108 select { 109 case <-a.sig.HasClosedChan(): 110 case <-time.After(tout): 111 return types.ErrTimeout 112 } 113 return nil 114 } 115 116 //------------------------------------------------------------------------------ 117 118 // Implements output.AsyncSink 119 type airGapBatchWriter struct { 120 w BatchOutput 121 122 sig *shutdown.Signaller 123 } 124 125 func newAirGapBatchWriter(w BatchOutput) output.AsyncSink { 126 return &airGapBatchWriter{w, shutdown.NewSignaller()} 127 } 128 129 func (a *airGapBatchWriter) ConnectWithContext(ctx context.Context) error { 130 return a.w.Connect(ctx) 131 } 132 133 func (a *airGapBatchWriter) WriteWithContext(ctx context.Context, msg types.Message) error { 134 parts := make([]*Message, msg.Len()) 135 _ = msg.Iter(func(i int, part types.Part) error { 136 parts[i] = newMessageFromPart(part) 137 return nil 138 }) 139 err := a.w.WriteBatch(ctx, parts) 140 if err != nil && errors.Is(err, ErrNotConnected) { 141 err = types.ErrNotConnected 142 } 143 return err 144 } 145 146 func (a *airGapBatchWriter) CloseAsync() { 147 go func() { 148 if err := a.w.Close(context.Background()); err == nil { 149 a.sig.ShutdownComplete() 150 } 151 }() 152 } 153 154 func (a *airGapBatchWriter) WaitForClose(tout time.Duration) error { 155 select { 156 case <-a.sig.HasClosedChan(): 157 case <-time.After(tout): 158 return types.ErrTimeout 159 } 160 return nil 161 } 162 163 //------------------------------------------------------------------------------ 164 165 // OwnedOutput provides direct ownership of an output extracted from a plugin 166 // config. Connectivity of the output is handled internally, and so the owner 167 // of this type should only be concerned with writing messages and eventually 168 // calling Close to terminate the output. 169 type OwnedOutput struct { 170 o types.Output 171 closeOnce sync.Once 172 t chan types.Transaction 173 } 174 175 func newOwnedOutput(o types.Output) (*OwnedOutput, error) { 176 tChan := make(chan types.Transaction) 177 if err := o.Consume(tChan); err != nil { 178 return nil, err 179 } 180 return &OwnedOutput{ 181 o: o, 182 t: tChan, 183 }, nil 184 } 185 186 // Write a message to the output, or return an error either if delivery is not 187 // possible or the context is cancelled. 188 func (o *OwnedOutput) Write(ctx context.Context, m *Message) error { 189 payload := message.New(nil) 190 payload.Append(m.part) 191 192 resChan := make(chan types.Response, 1) 193 select { 194 case o.t <- types.NewTransaction(payload, resChan): 195 case <-ctx.Done(): 196 return ctx.Err() 197 } 198 199 select { 200 case res := <-resChan: 201 return res.Error() 202 case <-ctx.Done(): 203 return ctx.Err() 204 } 205 } 206 207 // WriteBatch attempts to write a message batch to the output, and returns an 208 // error either if delivery is not possible or the context is cancelled. 209 func (o *OwnedOutput) WriteBatch(ctx context.Context, b MessageBatch) error { 210 payload := message.New(nil) 211 for _, m := range b { 212 payload.Append(m.part) 213 } 214 215 resChan := make(chan types.Response, 1) 216 select { 217 case o.t <- types.NewTransaction(payload, resChan): 218 case <-ctx.Done(): 219 return ctx.Err() 220 } 221 222 select { 223 case res := <-resChan: 224 return res.Error() 225 case <-ctx.Done(): 226 return ctx.Err() 227 } 228 } 229 230 // Close the output. 231 func (o *OwnedOutput) Close(ctx context.Context) error { 232 o.closeOnce.Do(func() { 233 close(o.t) 234 }) 235 for { 236 // Gross but will do for now until we replace these with context params. 237 if err := o.o.WaitForClose(time.Millisecond * 100); err == nil { 238 return nil 239 } 240 select { 241 case <-ctx.Done(): 242 return ctx.Err() 243 default: 244 } 245 } 246 }