github.com/Jeffail/benthos/v3@v3.65.0/lib/input/csv.go (about) 1 package input 2 3 import ( 4 "context" 5 "encoding/csv" 6 "errors" 7 "fmt" 8 "io" 9 "os" 10 "sync" 11 "time" 12 13 "github.com/Jeffail/benthos/v3/internal/docs" 14 "github.com/Jeffail/benthos/v3/internal/filepath" 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 ) 21 22 //------------------------------------------------------------------------------ 23 24 func init() { 25 Constructors[TypeCSVFile] = TypeSpec{ 26 constructor: fromSimpleConstructor(NewCSVFile), 27 Status: docs.StatusStable, 28 Summary: "Reads one or more CSV files as structured records following the format described in RFC 4180.", 29 FieldSpecs: docs.FieldSpecs{ 30 docs.FieldString( 31 "paths", "A list of file paths to read from. Each file will be read sequentially until the list is exhausted, at which point the input will close. Glob patterns are supported, including super globs (double star).", 32 []string{"/tmp/foo.csv", "/tmp/bar/*.csv", "/tmp/data/**/*.csv"}, 33 ).Array(), 34 docs.FieldCommon("parse_header_row", "Whether to reference the first row as a header row. If set to true the output structure for messages will be an object where field keys are determined by the header row."), 35 docs.FieldCommon("delimiter", `The delimiter to use for splitting values in each record, must be a single character.`), 36 docs.FieldAdvanced("batch_count", `Optionally process records in batches. This can help to speed up the consumption of exceptionally large CSV files. When the end of the file is reached the remaining records are processed as a (potentially smaller) batch.`), 37 }, 38 Description: ` 39 This input offers more control over CSV parsing than the ` + "[`file` input](/docs/components/inputs/file)" + `. 40 41 When parsing with a header row each line of the file will be consumed as a structured object, where the key names are determined from the header now. For example, the following CSV file: 42 43 ` + "```csv" + ` 44 foo,bar,baz 45 first foo,first bar,first baz 46 second foo,second bar,second baz 47 ` + "```" + ` 48 49 Would produce the following messages: 50 51 ` + "```json" + ` 52 {"foo":"first foo","bar":"first bar","baz":"first baz"} 53 {"foo":"second foo","bar":"second bar","baz":"second baz"} 54 ` + "```" + ` 55 56 If, however, the field ` + "`parse_header_row` is set to `false`" + ` then arrays are produced instead, like follows: 57 58 ` + "```json" + ` 59 ["first foo","first bar","first baz"] 60 ["second foo","second bar","second baz"] 61 ` + "```" + ``, 62 Categories: []Category{ 63 CategoryLocal, 64 }, 65 Footnotes: ` 66 This input is particularly useful when consuming CSV from files too large to 67 parse entirely within memory. However, in cases where CSV is consumed from other 68 input types it's also possible to parse them using the 69 ` + "[Bloblang `parse_csv` method](/docs/guides/bloblang/methods#parse_csv)" + `.`, 70 } 71 } 72 73 //------------------------------------------------------------------------------ 74 75 // CSVFileConfig contains configuration values for the CSVFile input type. 76 type CSVFileConfig struct { 77 Paths []string `json:"paths" yaml:"paths"` 78 ParseHeaderRow bool `json:"parse_header_row" yaml:"parse_header_row"` 79 Delim string `json:"delimiter" yaml:"delimiter"` 80 BatchCount int `json:"batch_count" yaml:"batch_count"` 81 } 82 83 // NewCSVFileConfig creates a new CSVFileConfig with default values. 84 func NewCSVFileConfig() CSVFileConfig { 85 return CSVFileConfig{ 86 Paths: []string{}, 87 ParseHeaderRow: true, 88 Delim: ",", 89 BatchCount: 1, 90 } 91 } 92 93 //------------------------------------------------------------------------------ 94 95 // NewCSVFile creates a new CSV file input type. 96 func NewCSVFile(conf Config, mgr types.Manager, log log.Modular, stats metrics.Type) (Type, error) { 97 delimRunes := []rune(conf.CSVFile.Delim) 98 if len(delimRunes) != 1 { 99 return nil, errors.New("delimiter value must be exactly one character") 100 } 101 102 comma := delimRunes[0] 103 104 pathsRemaining, err := filepath.Globs(conf.CSVFile.Paths) 105 if err != nil { 106 return nil, fmt.Errorf("failed to resolve path glob: %w", err) 107 } 108 if len(pathsRemaining) == 0 { 109 return nil, errors.New("requires at least one input file path") 110 } 111 112 if conf.CSVFile.BatchCount < 1 { 113 return nil, errors.New("batch_count must be at least 1") 114 } 115 116 rdr, err := newCSVReader( 117 func(context.Context) (io.Reader, error) { 118 if len(pathsRemaining) == 0 { 119 return nil, io.EOF 120 } 121 122 path := pathsRemaining[0] 123 handle, err := os.Open(path) 124 if err != nil { 125 return nil, err 126 } 127 pathsRemaining = pathsRemaining[1:] 128 129 return handle, nil 130 }, 131 func(context.Context) {}, 132 optCSVSetComma(comma), 133 optCSVSetExpectHeaders(conf.CSVFile.ParseHeaderRow), 134 optCSVSetGroupCount(conf.CSVFile.BatchCount), 135 ) 136 if err != nil { 137 return nil, err 138 } 139 140 return NewAsyncReader(TypeFile, true, reader.NewAsyncPreserver(rdr), log, stats) 141 } 142 143 //------------------------------------------------------------------------------ 144 145 // csvReader is an reader. implementation that consumes an io.Reader and parses 146 // it as a CSV file. 147 type csvReader struct { 148 handleCtor func(ctx context.Context) (io.Reader, error) 149 onClose func(ctx context.Context) 150 151 mut sync.Mutex 152 handle io.Reader 153 scanner *csv.Reader 154 headers []string 155 156 expectHeaders bool 157 comma rune 158 strict bool 159 groupCount int 160 } 161 162 // NewCSV creates a new reader input type able to create a feed of line 163 // delimited CSV records from an io.Reader. 164 // 165 // Callers must provide a constructor function for the target io.Reader, which 166 // is called on start up and again each time a reader is exhausted. If the 167 // constructor is called but there is no more content to create a Reader for 168 // then the error `io.EOF` should be returned and the CSV will close. 169 // 170 // Callers must also provide an onClose function, which will be called if the 171 // CSV has been instructed to shut down. This function should unblock any 172 // blocked Read calls. 173 func newCSVReader( 174 handleCtor func(ctx context.Context) (io.Reader, error), 175 onClose func(ctx context.Context), 176 options ...func(r *csvReader), 177 ) (*csvReader, error) { 178 r := csvReader{ 179 handleCtor: handleCtor, 180 onClose: onClose, 181 comma: ',', 182 expectHeaders: true, 183 strict: false, 184 groupCount: 1, 185 } 186 187 for _, opt := range options { 188 opt(&r) 189 } 190 191 return &r, nil 192 } 193 194 //------------------------------------------------------------------------------ 195 196 // OptCSVSetComma is a option func that sets the comma character (default ',') 197 // to be used to divide records. 198 func optCSVSetComma(comma rune) func(r *csvReader) { 199 return func(r *csvReader) { 200 r.comma = comma 201 } 202 } 203 204 // OptCSVSetGroupCount is a option func that sets the group count used to batch 205 // process records. 206 func optCSVSetGroupCount(groupCount int) func(r *csvReader) { 207 return func(r *csvReader) { 208 r.groupCount = groupCount 209 } 210 } 211 212 // OptCSVSetExpectHeaders is a option func that determines whether the first 213 // record from the CSV input outlines the names of columns. 214 func optCSVSetExpectHeaders(expect bool) func(r *csvReader) { 215 return func(r *csvReader) { 216 r.expectHeaders = expect 217 } 218 } 219 220 // OptCSVSetStrict is a option func that determines whether records with 221 // misaligned numbers of fields should be rejected. 222 func optCSVSetStrict(strict bool) func(r *csvReader) { 223 return func(r *csvReader) { 224 r.strict = strict 225 } 226 } 227 228 //------------------------------------------------------------------------------ 229 230 func (r *csvReader) closeHandle() { 231 if r.handle != nil { 232 if closer, ok := r.handle.(io.ReadCloser); ok { 233 closer.Close() 234 } 235 r.handle = nil 236 } 237 } 238 239 //------------------------------------------------------------------------------ 240 241 // ConnectWithContext attempts to establish a new scanner for an io.Reader. 242 func (r *csvReader) ConnectWithContext(ctx context.Context) error { 243 r.mut.Lock() 244 defer r.mut.Unlock() 245 if r.scanner != nil { 246 return nil 247 } 248 249 handle, err := r.handleCtor(ctx) 250 if err != nil { 251 if err == io.EOF { 252 return types.ErrTypeClosed 253 } 254 return err 255 } 256 257 scanner := csv.NewReader(handle) 258 scanner.Comma = r.comma 259 scanner.ReuseRecord = true 260 261 r.scanner = scanner 262 r.handle = handle 263 264 return nil 265 } 266 267 func (r *csvReader) readNext(reader *csv.Reader) ([]string, error) { 268 records, err := reader.Read() 269 if err != nil && (r.strict || len(records) == 0) { 270 if err == io.EOF { 271 r.mut.Lock() 272 r.scanner = nil 273 r.headers = nil 274 r.mut.Unlock() 275 return nil, types.ErrNotConnected 276 } 277 return nil, err 278 } 279 return records, nil 280 } 281 282 // ReadWithContext attempts to read a new line from the io.Reader. 283 func (r *csvReader) ReadWithContext(ctx context.Context) (types.Message, reader.AsyncAckFn, error) { 284 r.mut.Lock() 285 scanner := r.scanner 286 headers := r.headers 287 r.mut.Unlock() 288 289 if scanner == nil { 290 return nil, nil, types.ErrNotConnected 291 } 292 293 msg := message.New(nil) 294 295 for i := 0; i < r.groupCount; i++ { 296 records, err := r.readNext(scanner) 297 if err != nil { 298 if i == 0 { 299 return nil, nil, err 300 } 301 break 302 } 303 304 if r.expectHeaders && headers == nil { 305 headers = make([]string, 0, len(records)) 306 headers = append(headers, records...) 307 308 r.mut.Lock() 309 r.headers = headers 310 r.mut.Unlock() 311 312 if records, err = r.readNext(scanner); err != nil { 313 return nil, nil, err 314 } 315 } 316 317 part := message.NewPart(nil) 318 319 var structured interface{} 320 if len(headers) == 0 || len(headers) < len(records) { 321 slice := make([]interface{}, 0, len(records)) 322 for _, r := range records { 323 slice = append(slice, r) 324 } 325 structured = slice 326 } else { 327 obj := make(map[string]interface{}, len(records)) 328 for i, r := range records { 329 obj[headers[i]] = r 330 } 331 structured = obj 332 } 333 334 if err := part.SetJSON(structured); err != nil { 335 return nil, nil, err 336 } 337 msg.Append(part) 338 } 339 340 return msg, func(context.Context, types.Response) error { return nil }, nil 341 } 342 343 // CloseAsync shuts down the reader input and stops processing requests. 344 func (r *csvReader) CloseAsync() { 345 go func() { 346 r.mut.Lock() 347 r.onClose(context.Background()) 348 r.closeHandle() 349 r.mut.Unlock() 350 }() 351 } 352 353 // WaitForClose blocks until the reader input has closed down. 354 func (r *csvReader) WaitForClose(timeout time.Duration) error { 355 return nil 356 } 357 358 //------------------------------------------------------------------------------