github.com/Jeffail/benthos/v3@v3.65.0/lib/input/http_client.go (about) 1 package input 2 3 import ( 4 "context" 5 "errors" 6 "io" 7 "strings" 8 "sync" 9 "time" 10 11 "github.com/Jeffail/benthos/v3/internal/codec" 12 "github.com/Jeffail/benthos/v3/internal/docs" 13 "github.com/Jeffail/benthos/v3/internal/http" 14 "github.com/Jeffail/benthos/v3/internal/interop" 15 "github.com/Jeffail/benthos/v3/lib/input/reader" 16 "github.com/Jeffail/benthos/v3/lib/log" 17 "github.com/Jeffail/benthos/v3/lib/message" 18 "github.com/Jeffail/benthos/v3/lib/metrics" 19 "github.com/Jeffail/benthos/v3/lib/types" 20 "github.com/Jeffail/benthos/v3/lib/util/http/client" 21 ) 22 23 func httpClientSpec() docs.FieldSpec { 24 codecDocs := codec.ReaderDocs.AtVersion("3.42.0") 25 codecDocs.Description = "The way in which the bytes of a continuous stream are converted into messages. It's possible to consume lines using a custom delimiter with the `delim:x` codec, where x is the character sequence custom delimiter. It's not necessary to add gzip in the codec when the response headers specify it as it will be decompressed automatically." 26 codecDocs.Examples = []interface{}{"lines", "delim:\t", "delim:foobar", "csv"} 27 28 streamSpecs := docs.FieldSpecs{ 29 docs.FieldBool("enabled", "Enables streaming mode."), 30 docs.FieldBool("reconnect", "Sets whether to re-establish the connection once it is lost."), 31 codecDocs, 32 docs.FieldInt("max_buffer", "Must be larger than the largest line of the stream.").Advanced(), 33 docs.FieldDeprecated("multipart"), 34 docs.FieldDeprecated("delimiter"), 35 } 36 37 return client.FieldSpec( 38 docs.FieldCommon("payload", "An optional payload to deliver for each request."), 39 docs.FieldAdvanced("drop_empty_bodies", "Whether empty payloads received from the target server should be dropped."), 40 docs.FieldCommon( 41 "stream", "Allows you to set streaming mode, where requests are kept open and messages are processed line-by-line.", 42 ).WithChildren(streamSpecs...), 43 ) 44 } 45 46 func init() { 47 Constructors[TypeHTTPClient] = TypeSpec{ 48 constructor: fromSimpleConstructor(NewHTTPClient), 49 Summary: ` 50 Connects to a server and continuously performs requests for a single message.`, 51 Description: ` 52 The URL and header values of this type can be dynamically set using function 53 interpolations described [here](/docs/configuration/interpolation#bloblang-queries). 54 55 ### Streaming 56 57 If you enable streaming then Benthos will consume the body of the response as a continuous stream of data, breaking messages out following a chosen codec. This allows you to consume APIs that provide long lived streamed data feeds (such as Twitter). 58 59 ### Pagination 60 61 This input supports interpolation functions in the ` + "`url` and `headers`" + ` fields where data from the previous successfully consumed message (if there was one) can be referenced. This can be used in order to support basic levels of pagination. However, in cases where pagination depends on logic it is recommended that you use an ` + "[`http` processor](/docs/components/processors/http) instead, often combined with a [`generate` input](/docs/components/inputs/generate)" + ` in order to schedule the processor.`, 62 config: httpClientSpec(), 63 Categories: []Category{ 64 CategoryNetwork, 65 }, 66 Examples: []docs.AnnotatedExample{ 67 { 68 Title: "Basic Pagination", 69 Summary: "Interpolation functions within the `url` and `headers` fields can be used to reference the previously consumed message, which allows simple pagination.", 70 Config: ` 71 input: 72 http_client: 73 url: >- 74 https://api.example.com/search?query=allmyfoos&start_time=${! ( 75 (timestamp_unix()-300).format_timestamp("2006-01-02T15:04:05Z","UTC").escape_url_query() 76 ) }${! ("&next_token="+this.meta.next_token.not_null()) | "" } 77 verb: GET 78 rate_limit: foo_searches 79 oauth2: 80 enabled: true 81 token_url: https://api.example.com/oauth2/token 82 client_key: "${EXAMPLE_KEY}" 83 client_secret: "${EXAMPLE_SECRET}" 84 85 rate_limit_resources: 86 - label: foo_searches 87 local: 88 count: 1 89 interval: 30s 90 `, 91 }, 92 }, 93 } 94 } 95 96 //------------------------------------------------------------------------------ 97 98 // StreamConfig contains fields for specifying consumption behaviour when the 99 // body of a request is a constant stream of bytes. 100 type StreamConfig struct { 101 Enabled bool `json:"enabled" yaml:"enabled"` 102 Reconnect bool `json:"reconnect" yaml:"reconnect"` 103 Codec string `json:"codec" yaml:"codec"` 104 Multipart bool `json:"multipart" yaml:"multipart"` 105 MaxBuffer int `json:"max_buffer" yaml:"max_buffer"` 106 Delim string `json:"delimiter" yaml:"delimiter"` 107 } 108 109 // HTTPClientConfig contains configuration for the HTTPClient output type. 110 type HTTPClientConfig struct { 111 client.Config `json:",inline" yaml:",inline"` 112 Payload string `json:"payload" yaml:"payload"` 113 DropEmptyBodies bool `json:"drop_empty_bodies" yaml:"drop_empty_bodies"` 114 Stream StreamConfig `json:"stream" yaml:"stream"` 115 } 116 117 // NewHTTPClientConfig creates a new HTTPClientConfig with default values. 118 func NewHTTPClientConfig() HTTPClientConfig { 119 cConf := client.NewConfig() 120 cConf.Verb = "GET" 121 cConf.URL = "http://localhost:4195/get" 122 return HTTPClientConfig{ 123 Config: cConf, 124 Payload: "", 125 DropEmptyBodies: true, 126 Stream: StreamConfig{ 127 Enabled: false, 128 Reconnect: true, 129 Codec: "lines", 130 Multipart: false, 131 MaxBuffer: 1000000, 132 Delim: "", 133 }, 134 } 135 } 136 137 //------------------------------------------------------------------------------ 138 139 // HTTPClient is an input type that continuously makes HTTP requests and reads 140 // the response bodies as message payloads. 141 type HTTPClient struct { 142 conf HTTPClientConfig 143 144 client *http.Client 145 payload types.Message 146 prevResponse types.Message 147 148 codecCtor codec.ReaderConstructor 149 150 codecMut sync.Mutex 151 codec codec.Reader 152 } 153 154 // NewHTTPClient creates a new HTTPClient input type. 155 func NewHTTPClient(conf Config, mgr types.Manager, log log.Modular, stats metrics.Type) (Type, error) { 156 rdr, err := newHTTPClient(conf.HTTPClient, mgr, log, stats) 157 if err != nil { 158 return nil, err 159 } 160 return NewAsyncReader(TypeHTTPClient, true, reader.NewAsyncPreserver(rdr), log, stats) 161 } 162 163 func newHTTPClient(conf HTTPClientConfig, mgr types.Manager, log log.Modular, stats metrics.Type) (*HTTPClient, error) { 164 var codecCtor codec.ReaderConstructor 165 166 if conf.Stream.Enabled { 167 // Timeout should be left at zero if we are streaming. 168 conf.Timeout = "" 169 if len(conf.Stream.Delim) > 0 { 170 conf.Stream.Codec = "delim:" + conf.Stream.Delim 171 } 172 if conf.Stream.Multipart && !strings.HasSuffix(conf.Stream.Codec, "/multipart") { 173 conf.Stream.Codec += "/multipart" 174 } 175 codecConf := codec.NewReaderConfig() 176 codecConf.MaxScanTokenSize = conf.Stream.MaxBuffer 177 178 var err error 179 if codecCtor, err = codec.GetReader(conf.Stream.Codec, codecConf); err != nil { 180 return nil, err 181 } 182 } 183 184 var payload types.Message = message.New(nil) 185 if len(conf.Payload) > 0 { 186 payload = message.New([][]byte{[]byte(conf.Payload)}) 187 } 188 189 cMgr, cLog, cStats := interop.LabelChild("client", mgr, log, stats) 190 client, err := http.NewClient( 191 conf.Config, 192 http.OptSetManager(cMgr), 193 http.OptSetLogger(cLog), 194 http.OptSetStats(cStats), 195 ) 196 if err != nil { 197 return nil, err 198 } 199 200 return &HTTPClient{ 201 conf: conf, 202 payload: payload, 203 prevResponse: message.New(nil), 204 client: client, 205 206 codecCtor: codecCtor, 207 }, nil 208 } 209 210 //------------------------------------------------------------------------------ 211 212 // ConnectWithContext establishes a connection. 213 func (h *HTTPClient) ConnectWithContext(ctx context.Context) (err error) { 214 if !h.conf.Stream.Enabled { 215 return nil 216 } 217 218 h.codecMut.Lock() 219 defer h.codecMut.Unlock() 220 221 if h.codec != nil { 222 return nil 223 } 224 225 res, err := h.client.SendToResponse(context.Background(), h.payload, h.prevResponse) 226 if err != nil { 227 if strings.Contains(err.Error(), "(Client.Timeout exceeded while awaiting headers)") { 228 err = types.ErrTimeout 229 } 230 return err 231 } 232 233 p := message.NewPart(nil) 234 meta := p.Metadata() 235 for k, values := range res.Header { 236 if len(values) > 0 { 237 meta.Set(strings.ToLower(k), values[0]) 238 } 239 } 240 h.prevResponse = message.New(nil) 241 h.prevResponse.Append(p) 242 243 if h.codec, err = h.codecCtor("", res.Body, func(ctx context.Context, err error) error { 244 return nil 245 }); err != nil { 246 res.Body.Close() 247 return err 248 } 249 return nil 250 } 251 252 // ReadWithContext a new HTTPClient message. 253 func (h *HTTPClient) ReadWithContext(ctx context.Context) (types.Message, reader.AsyncAckFn, error) { 254 if h.conf.Stream.Enabled { 255 return h.readStreamed(ctx) 256 } 257 return h.readNotStreamed(ctx) 258 } 259 260 func (h *HTTPClient) readStreamed(ctx context.Context) (types.Message, reader.AsyncAckFn, error) { 261 h.codecMut.Lock() 262 defer h.codecMut.Unlock() 263 264 if h.codec == nil { 265 return nil, nil, types.ErrNotConnected 266 } 267 268 parts, codecAckFn, err := h.codec.Next(ctx) 269 if err != nil { 270 if errors.Is(err, context.Canceled) || 271 errors.Is(err, context.DeadlineExceeded) { 272 err = types.ErrTimeout 273 } 274 if err != types.ErrTimeout { 275 h.codec.Close(ctx) 276 h.codec = nil 277 } 278 if errors.Is(err, io.EOF) { 279 if !h.conf.Stream.Reconnect { 280 return nil, nil, types.ErrTypeClosed 281 } 282 return nil, nil, types.ErrTimeout 283 } 284 return nil, nil, err 285 } 286 287 msg := message.New(nil) 288 msg.Append(parts...) 289 290 if msg.Len() == 1 && msg.Get(0).IsEmpty() && h.conf.DropEmptyBodies { 291 _ = codecAckFn(ctx, nil) 292 return nil, nil, types.ErrTimeout 293 } 294 if msg.Len() == 0 { 295 _ = codecAckFn(ctx, nil) 296 return nil, nil, types.ErrTimeout 297 } 298 299 meta := h.prevResponse.Get(0).Metadata() 300 resParts := make([]types.Part, 0, msg.Len()) 301 msg.Iter(func(i int, p types.Part) error { 302 part := message.NewPart(p.Get()) 303 part.SetMetadata(meta) 304 resParts = append(resParts, part) 305 return nil 306 }) 307 h.prevResponse.SetAll(resParts) 308 309 return msg, func(rctx context.Context, res types.Response) error { 310 return codecAckFn(rctx, res.Error()) 311 }, nil 312 } 313 314 func (h *HTTPClient) readNotStreamed(ctx context.Context) (types.Message, reader.AsyncAckFn, error) { 315 msg, err := h.client.Send(ctx, h.payload, h.prevResponse) 316 if err != nil { 317 if strings.Contains(err.Error(), "(Client.Timeout exceeded while awaiting headers)") { 318 err = types.ErrTimeout 319 } 320 return nil, nil, err 321 } 322 323 if msg.Len() == 0 { 324 return nil, nil, types.ErrTimeout 325 } 326 if msg.Len() == 1 && msg.Get(0).IsEmpty() && h.conf.DropEmptyBodies { 327 return nil, nil, types.ErrTimeout 328 } 329 330 h.prevResponse = msg 331 return msg.Copy(), func(context.Context, types.Response) error { 332 return nil 333 }, nil 334 } 335 336 // CloseAsync shuts down the HTTPClient input and stops processing requests. 337 func (h *HTTPClient) CloseAsync() { 338 h.client.Close(context.Background()) 339 go func() { 340 h.codecMut.Lock() 341 if h.codec != nil { 342 h.codec.Close(context.Background()) 343 h.codec = nil 344 } 345 h.codecMut.Unlock() 346 }() 347 } 348 349 // WaitForClose blocks until the HTTPClient input has closed down. 350 func (h *HTTPClient) WaitForClose(timeout time.Duration) error { 351 return nil 352 }