github.com/Jeffail/benthos/v3@v3.65.0/lib/processor/avro.go (about) 1 package processor 2 3 import ( 4 "fmt" 5 "io" 6 "net/http" 7 "strings" 8 "time" 9 10 "github.com/Jeffail/benthos/v3/internal/docs" 11 "github.com/Jeffail/benthos/v3/internal/tracing" 12 "github.com/Jeffail/benthos/v3/lib/log" 13 "github.com/Jeffail/benthos/v3/lib/metrics" 14 "github.com/Jeffail/benthos/v3/lib/types" 15 "github.com/linkedin/goavro/v2" 16 ) 17 18 //------------------------------------------------------------------------------ 19 20 func init() { 21 Constructors[TypeAvro] = TypeSpec{ 22 constructor: NewAvro, 23 Categories: []Category{ 24 CategoryParsing, 25 }, 26 Summary: ` 27 Performs Avro based operations on messages based on a schema.`, 28 Status: docs.StatusBeta, 29 Description: ` 30 WARNING: If you are consuming or generating messages using a schema registry service then it is likely this processor will fail as those services require messages to be prefixed with the identifier of the schema version being used. Instead, try the ` + "[`schema_registry_encode`](/docs/components/processors/schema_registry_encode) and [`schema_registry_decode`](/docs/components/processors/schema_registry_decode)" + ` processors. 31 32 ## Operators 33 34 ### ` + "`to_json`" + ` 35 36 Converts Avro documents into a JSON structure. This makes it easier to 37 manipulate the contents of the document within Benthos. The encoding field 38 specifies how the source documents are encoded. 39 40 ### ` + "`from_json`" + ` 41 42 Attempts to convert JSON documents into Avro documents according to the 43 specified encoding.`, 44 FieldSpecs: docs.FieldSpecs{ 45 docs.FieldCommon("operator", "The [operator](#operators) to execute").HasOptions("to_json", "from_json"), 46 docs.FieldCommon("encoding", "An Avro encoding format to use for conversions to and from a schema.").HasOptions("textual", "binary", "single"), 47 docs.FieldCommon("schema", "A full Avro schema to use."), 48 docs.FieldCommon( 49 "schema_path", "The path of a schema document to apply. Use either this or the `schema` field.", 50 "file://path/to/spec.avsc", 51 "http://localhost:8081/path/to/spec/versions/1", 52 ), 53 PartsFieldSpec, 54 }, 55 } 56 } 57 58 //------------------------------------------------------------------------------ 59 60 // AvroConfig contains configuration fields for the Avro processor. 61 type AvroConfig struct { 62 Parts []int `json:"parts" yaml:"parts"` 63 Operator string `json:"operator" yaml:"operator"` 64 Encoding string `json:"encoding" yaml:"encoding"` 65 Schema string `json:"schema" yaml:"schema"` 66 SchemaPath string `json:"schema_path" yaml:"schema_path"` 67 } 68 69 // NewAvroConfig returns a AvroConfig with default values. 70 func NewAvroConfig() AvroConfig { 71 return AvroConfig{ 72 Parts: []int{}, 73 Operator: "to_json", 74 Encoding: "textual", 75 Schema: "", 76 SchemaPath: "", 77 } 78 } 79 80 //------------------------------------------------------------------------------ 81 82 type avroOperator func(part types.Part) error 83 84 func newAvroToJSONOperator(encoding string, codec *goavro.Codec) (avroOperator, error) { 85 switch encoding { 86 case "textual": 87 return func(part types.Part) error { 88 jObj, _, err := codec.NativeFromTextual(part.Get()) 89 if err != nil { 90 return fmt.Errorf("failed to convert Avro document to JSON: %v", err) 91 } 92 if err = part.SetJSON(jObj); err != nil { 93 return fmt.Errorf("failed to set JSON: %v", err) 94 } 95 return nil 96 }, nil 97 case "binary": 98 return func(part types.Part) error { 99 jObj, _, err := codec.NativeFromBinary(part.Get()) 100 if err != nil { 101 return fmt.Errorf("failed to convert Avro document to JSON: %v", err) 102 } 103 if err = part.SetJSON(jObj); err != nil { 104 return fmt.Errorf("failed to set JSON: %v", err) 105 } 106 return nil 107 }, nil 108 case "single": 109 return func(part types.Part) error { 110 jObj, _, err := codec.NativeFromSingle(part.Get()) 111 if err != nil { 112 return fmt.Errorf("failed to convert Avro document to JSON: %v", err) 113 } 114 if err = part.SetJSON(jObj); err != nil { 115 return fmt.Errorf("failed to set JSON: %v", err) 116 } 117 return nil 118 }, nil 119 } 120 return nil, fmt.Errorf("encoding '%v' not recognised", encoding) 121 } 122 123 func newAvroFromJSONOperator(encoding string, codec *goavro.Codec) (avroOperator, error) { 124 switch encoding { 125 case "textual": 126 return func(part types.Part) error { 127 jObj, err := part.JSON() 128 if err != nil { 129 return fmt.Errorf("failed to parse message as JSON: %v", err) 130 } 131 var textual []byte 132 if textual, err = codec.TextualFromNative(nil, jObj); err != nil { 133 return fmt.Errorf("failed to convert JSON to Avro schema: %v", err) 134 } 135 part.Set(textual) 136 return nil 137 }, nil 138 case "binary": 139 return func(part types.Part) error { 140 jObj, err := part.JSON() 141 if err != nil { 142 return fmt.Errorf("failed to parse message as JSON: %v", err) 143 } 144 var binary []byte 145 if binary, err = codec.BinaryFromNative(nil, jObj); err != nil { 146 return fmt.Errorf("failed to convert JSON to Avro schema: %v", err) 147 } 148 part.Set(binary) 149 return nil 150 }, nil 151 case "single": 152 return func(part types.Part) error { 153 jObj, err := part.JSON() 154 if err != nil { 155 return fmt.Errorf("failed to parse message as JSON: %v", err) 156 } 157 var single []byte 158 if single, err = codec.SingleFromNative(nil, jObj); err != nil { 159 return fmt.Errorf("failed to convert JSON to Avro schema: %v", err) 160 } 161 part.Set(single) 162 return nil 163 }, nil 164 } 165 return nil, fmt.Errorf("encoding '%v' not recognised", encoding) 166 } 167 168 func strToAvroOperator(opStr, encoding string, codec *goavro.Codec) (avroOperator, error) { 169 switch opStr { 170 case "to_json": 171 return newAvroToJSONOperator(encoding, codec) 172 case "from_json": 173 return newAvroFromJSONOperator(encoding, codec) 174 } 175 return nil, fmt.Errorf("operator not recognised: %v", opStr) 176 } 177 178 func loadSchema(schemaPath string) (string, error) { 179 t := &http.Transport{} 180 t.RegisterProtocol("file", http.NewFileTransport(http.Dir("/"))) 181 c := &http.Client{Transport: t} 182 183 response, err := c.Get(schemaPath) 184 185 if err != nil { 186 return "", err 187 } 188 189 defer response.Body.Close() 190 191 body, err := io.ReadAll(response.Body) 192 193 if err != nil { 194 return "", err 195 } 196 197 return string(body), nil 198 } 199 200 //------------------------------------------------------------------------------ 201 202 // Avro is a processor that performs an operation on an Avro payload. 203 type Avro struct { 204 parts []int 205 operator avroOperator 206 207 conf Config 208 log log.Modular 209 stats metrics.Type 210 211 mCount metrics.StatCounter 212 mErr metrics.StatCounter 213 mSent metrics.StatCounter 214 mBatchSent metrics.StatCounter 215 } 216 217 // NewAvro returns an Avro processor. 218 func NewAvro( 219 conf Config, mgr types.Manager, log log.Modular, stats metrics.Type, 220 ) (Type, error) { 221 a := &Avro{ 222 parts: conf.Avro.Parts, 223 conf: conf, 224 log: log, 225 stats: stats, 226 227 mCount: stats.GetCounter("count"), 228 mErr: stats.GetCounter("error"), 229 mSent: stats.GetCounter("sent"), 230 mBatchSent: stats.GetCounter("batch.sent"), 231 } 232 var schema string 233 var err error 234 235 if schemaPath := conf.Avro.SchemaPath; schemaPath != "" { 236 if !(strings.HasPrefix(schemaPath, "file://") || strings.HasPrefix(schemaPath, "http://")) { 237 return nil, fmt.Errorf("invalid schema_path provided, must start with file:// or http://") 238 } 239 240 schema, err = loadSchema(schemaPath) 241 if err != nil { 242 return nil, fmt.Errorf("failed to load Avro schema definition: %v", err) 243 } 244 } else { 245 schema = conf.Avro.Schema 246 } 247 248 codec, err := goavro.NewCodec(schema) 249 if err != nil { 250 return nil, fmt.Errorf("failed to parse schema: %v", err) 251 } 252 253 if a.operator, err = strToAvroOperator(conf.Avro.Operator, conf.Avro.Encoding, codec); err != nil { 254 return nil, err 255 } 256 return a, nil 257 } 258 259 //------------------------------------------------------------------------------ 260 261 // ProcessMessage applies the processor to a message, either creating >0 262 // resulting messages or a response to be sent back to the message source. 263 func (p *Avro) ProcessMessage(msg types.Message) ([]types.Message, types.Response) { 264 p.mCount.Incr(1) 265 newMsg := msg.Copy() 266 267 proc := func(index int, span *tracing.Span, part types.Part) error { 268 if err := p.operator(part); err != nil { 269 p.mErr.Incr(1) 270 p.log.Debugf("Operator failed: %v\n", err) 271 return err 272 } 273 return nil 274 } 275 276 IteratePartsWithSpanV2(TypeAvro, p.parts, newMsg, proc) 277 278 p.mBatchSent.Incr(1) 279 p.mSent.Incr(int64(newMsg.Len())) 280 return []types.Message{newMsg}, nil 281 } 282 283 // CloseAsync shuts down the processor and stops processing requests. 284 func (p *Avro) CloseAsync() { 285 } 286 287 // WaitForClose blocks until the processor has closed down. 288 func (p *Avro) WaitForClose(timeout time.Duration) error { 289 return nil 290 } 291 292 //------------------------------------------------------------------------------