-
Notifications
You must be signed in to change notification settings - Fork 89
/
bulk_ingestion.go
686 lines (591 loc) · 19.1 KB
/
bulk_ingestion.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package snowflake
import (
"bufio"
"bytes"
"compress/flate"
"context"
"database/sql"
"database/sql/driver"
"errors"
"fmt"
"io"
"math"
"path"
"runtime"
"slices"
"strings"
"sync"
"time"
"github.com/apache/arrow-adbc/go/adbc"
"github.com/apache/arrow/go/v18/arrow"
"github.com/apache/arrow/go/v18/arrow/array"
"github.com/apache/arrow/go/v18/arrow/memory"
"github.com/apache/arrow/go/v18/parquet"
"github.com/apache/arrow/go/v18/parquet/compress"
"github.com/apache/arrow/go/v18/parquet/pqarrow"
"github.com/snowflakedb/gosnowflake"
"golang.org/x/sync/errgroup"
)
const (
bindStageName = "ADBC$BIND"
createTemporaryStageStmt = "CREATE OR REPLACE TEMPORARY STAGE " + bindStageName + " FILE_FORMAT = (TYPE = PARQUET USE_LOGICAL_TYPE = TRUE BINARY_AS_TEXT = FALSE USE_VECTORIZED_SCANNER=TRUE REPLACE_INVALID_CHARACTERS = TRUE)"
putQueryTmpl = "PUT 'file:///tmp/placeholder/%s' @" + bindStageName + " OVERWRITE = TRUE"
copyQuery = "COPY INTO IDENTIFIER(?) FROM @" + bindStageName + " MATCH_BY_COLUMN_NAME = CASE_INSENSITIVE"
countQuery = "SELECT COUNT(*) FROM IDENTIFIER(?)"
megabyte = 1024 * 1024
)
var (
defaultTargetFileSize uint = 10 * megabyte
defaultWriterConcurrency uint = uint(runtime.NumCPU())
defaultUploadConcurrency uint = 8
defaultCopyConcurrency uint = 4
defaultCompressionCodec compress.Compression = compress.Codecs.Snappy
defaultCompressionLevel int = flate.DefaultCompression
ErrNoRecordsInStream = errors.New("no records in stream to write")
)
// Options for configuring bulk ingestion.
//
// Values should be updated with appropriate calls to stmt.SetOption().
type ingestOptions struct {
// Approximate size of Parquet files written during ingestion.
//
// Actual size will be slightly larger, depending on size of footer/metadata.
// Default is 10 MB. If set to 0, file size has no limit. Cannot be negative.
targetFileSize uint
// Number of Parquet files to write in parallel.
//
// Default attempts to maximize workers based on logical cores detected, but
// may need to be adjusted if running in a constrained environment.
// If set to 0, default value is used. Cannot be negative.
writerConcurrency uint
// Number of Parquet files to upload in parallel.
//
// Greater concurrency can smooth out TCP congestion and help make use of
// available network bandwith, but will increase memory utilization.
// Default is 8. If set to 0, default value is used. Cannot be negative.
uploadConcurrency uint
// Maximum number of COPY operations to run concurrently.
//
// Bulk ingestion performance is optimized by executing COPY queries as files are
// still being uploaded. Snowflake COPY speed scales with warehouse size, so smaller
// warehouses may benefit from setting this value higher to ensure long-running
// COPY queries do not block newly uploaded files from being loaded.
// Default is 4. If set to 0, only a single COPY query will be executed as part of ingestion,
// once all files have finished uploading. Cannot be negative.
copyConcurrency uint
// Compression codec to use for Parquet files.
//
// When network speeds are high, it is generally faster to use a faster codec with
// a lower compression ratio. The opposite is true if the network is slow by CPU is
// available.
// Default is Snappy.
compressionCodec compress.Compression
// Compression level for Parquet files.
//
// The compression level is codec-specific. Some codecs do not support setting it,
// notably Snappy.
// Default is the default level for the specified compressionCodec.
compressionLevel int
}
func DefaultIngestOptions() *ingestOptions {
return &ingestOptions{
targetFileSize: defaultTargetFileSize,
writerConcurrency: defaultWriterConcurrency,
uploadConcurrency: defaultUploadConcurrency,
copyConcurrency: defaultCopyConcurrency,
compressionCodec: defaultCompressionCodec,
compressionLevel: defaultCompressionLevel,
}
}
// ingestRecord performs bulk ingestion of a single Record and returns the
// number of rows affected.
//
// The Record must already be bound by calling stmt.Bind(), and will be released
// and reset upon completion.
func (st *statement) ingestRecord(ctx context.Context) (nrows int64, err error) {
defer func() {
// Record already released by writeParquet()
st.bound = nil
}()
var (
initialRows int64
target = quoteTblName(st.targetTable)
)
// Check final row count of target table to get definitive rows affected
initialRows, err = countRowsInTable(ctx, st.cnxn.sqldb, target)
if err != nil {
st.bound.Release()
return
}
parquetProps, arrowProps := newWriterProps(st.alloc, st.ingestOptions)
g := errgroup.Group{}
// writeParquet takes a channel of Records, but we only have one Record to write
recordCh := make(chan arrow.Record, 1)
recordCh <- st.bound
close(recordCh)
// Read the Record from the channel and write it into the provided writer
schema := st.bound.Schema()
r, w := io.Pipe()
bw := bufio.NewWriter(w)
g.Go(func() error {
defer r.Close()
defer bw.Flush()
err = writeParquet(schema, bw, recordCh, 0, parquetProps, arrowProps)
if err != io.EOF {
return err
}
return nil
})
// Create a temporary stage, we can't start uploading until it has been created
_, err = st.cnxn.cn.ExecContext(ctx, createTemporaryStageStmt, nil)
if err != nil {
return
}
// Start uploading the file to Snowflake
fileName := "0.parquet" // Only writing 1 file, so use same name as first file written by ingestStream() for consistency
err = uploadStream(ctx, st.cnxn.cn, r, fileName)
if err != nil {
return
}
// Parquet writing is already done if the upload finished, so we're just checking for any errors
err = g.Wait()
if err != nil {
return
}
// Load the uploaded file into the target table
_, err = st.cnxn.cn.ExecContext(ctx, copyQuery, []driver.NamedValue{{Value: target}})
if err != nil {
return
}
// Check final row count of target table to get definitive rows affected
nrows, err = countRowsInTable(ctx, st.cnxn.sqldb, target)
nrows = nrows - initialRows
return
}
// ingestStream performs bulk ingestion of a RecordReader and returns the
// number of rows affected.
//
// The RecordReader must already be bound by calling stmt.BindStream(), and will
// be released and reset upon completion.
func (st *statement) ingestStream(ctx context.Context) (nrows int64, err error) {
defer func() {
st.streamBind.Release()
st.streamBind = nil
}()
var (
initialRows int64
target = quoteTblName(st.targetTable)
)
// Check final row count of target table to get definitive rows affected
initialRows, err = countRowsInTable(ctx, st.cnxn.sqldb, target)
if err != nil {
return
}
defer func() {
// Always check the resulting row count, even in the case of an error. We may have ingested part of the data.
ctx := context.WithoutCancel(ctx)
n, countErr := countRowsInTable(ctx, st.cnxn.sqldb, target)
nrows = n - initialRows
// Ingestion, row-count check, or both could have failed
// Wrap any failures as ADBC errors
err = errors.Join(err, countErr)
if errors.Is(err, context.Canceled) {
err = errToAdbcErr(adbc.StatusCancelled, err)
} else {
err = errToAdbcErr(adbc.StatusInternal, err)
}
}()
parquetProps, arrowProps := newWriterProps(st.alloc, st.ingestOptions)
g, gCtx := errgroup.WithContext(ctx)
// Read records into channel
records := make(chan arrow.Record, st.ingestOptions.writerConcurrency)
g.Go(func() error {
return readRecords(gCtx, st.streamBind, records)
})
// Read records from channel and write Parquet files in parallel to buffer pool
schema := st.streamBind.Schema()
pool := newBufferPool(int(st.ingestOptions.targetFileSize))
buffers := make(chan *bytes.Buffer, st.ingestOptions.writerConcurrency)
g.Go(func() error {
return runParallelParquetWriters(
gCtx,
schema,
int(st.ingestOptions.targetFileSize),
int(st.ingestOptions.writerConcurrency),
parquetProps,
arrowProps,
pool,
records,
buffers,
)
})
// Create a temporary stage, we can't start uploading until it has been created
_, err = st.cnxn.cn.ExecContext(ctx, createTemporaryStageStmt, nil)
if err != nil {
return
}
// Kickoff background tasks to COPY Parquet files into Snowflake table as they are uploaded
fileReady, finishCopy, cancelCopy := runCopyTasks(ctx, st.cnxn.cn, target, int(st.ingestOptions.copyConcurrency))
// Read Parquet files from buffer pool and upload to Snowflake stage in parallel
g.Go(func() error {
return uploadAllStreams(gCtx, st.cnxn.cn, buffers, int(st.ingestOptions.uploadConcurrency), pool, fileReady)
})
// Wait until either all files have been uploaded to Snowflake or the pipeline has failed / been canceled
if err = g.Wait(); err != nil {
// If the pipeline failed, we can cancel in-progress COPYs and avoid starting the final one
cancelCopy()
return
}
// Start final COPY and wait for all COPY tasks to complete
err = finishCopy()
return
}
func newWriterProps(mem memory.Allocator, opts *ingestOptions) (*parquet.WriterProperties, pqarrow.ArrowWriterProperties) {
parquetProps := parquet.NewWriterProperties(
parquet.WithAllocator(mem),
parquet.WithCompression(opts.compressionCodec),
parquet.WithCompressionLevel(opts.compressionLevel),
// Overhead for dict building often reduces throughput more than filesize reduction benefits; may expose as config in future
parquet.WithDictionaryDefault(false),
// Stats won't be used since the file is dropped after ingestion completes
parquet.WithStats(false),
parquet.WithMaxRowGroupLength(math.MaxInt64),
)
arrowProps := pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem))
return parquetProps, arrowProps
}
func readRecords(ctx context.Context, rdr array.RecordReader, out chan<- arrow.Record) error {
defer close(out)
for rdr.Next() {
rec := rdr.Record()
rec.Retain()
select {
case out <- rec:
case <-ctx.Done():
return ctx.Err()
}
}
return nil
}
func writeRecordToParquet(wr *pqarrow.FileWriter, rec arrow.Record) (int64, error) {
if rec.NumRows() == 0 {
rec.Release()
return 0, nil
}
err := wr.Write(rec)
rec.Release()
if err != nil {
return 0, err
}
return wr.RowGroupTotalBytesWritten(), nil
}
func writeParquet(
schema *arrow.Schema,
w io.Writer,
in <-chan arrow.Record,
targetSize int,
parquetProps *parquet.WriterProperties,
arrowProps pqarrow.ArrowWriterProperties,
) error {
pqWriter, err := pqarrow.NewFileWriter(schema, w, parquetProps, arrowProps)
if err != nil {
return err
}
defer pqWriter.Close()
var bytesWritten int64
for rec := range in {
nbytes, err := writeRecordToParquet(pqWriter, rec)
if err != nil {
return err
}
bytesWritten += nbytes
if targetSize < 0 {
continue
}
if bytesWritten >= int64(targetSize) {
return nil
}
}
// let the caller know if the parquet file is empty, to avoid sending it any further in the pipeline
if bytesWritten == 0 {
return ErrNoRecordsInStream
}
// Input channel closed, signal that all parquet writing is done
return io.EOF
}
func runParallelParquetWriters(
ctx context.Context,
schema *arrow.Schema,
targetSize int,
concurrency int,
parquetProps *parquet.WriterProperties,
arrowProps pqarrow.ArrowWriterProperties,
buffers *bufferPool,
in <-chan arrow.Record,
out chan<- *bytes.Buffer,
) error {
var once sync.Once
defer close(out)
g, ctx := errgroup.WithContext(ctx)
g.SetLimit(concurrency)
done := make(chan interface{})
finished := func() {
once.Do(func() { close(done) })
}
tryWriteBuffer := func(buf *bytes.Buffer) {
select {
case out <- buf:
case <-ctx.Done():
// If the context is canceled, the buffer may be full but we don't want to block indefinitely
}
}
for {
select {
case <-done:
return g.Wait()
default:
}
g.Go(func() error {
select {
case <-done:
// Channel may have already closed while goroutine was waiting to get scheduled
return nil
case <-ctx.Done():
finished()
return ctx.Err()
default:
// Proceed to write a new file
}
buf := buffers.GetBuffer()
err := writeParquet(schema, buf, in, targetSize, parquetProps, arrowProps)
if err == io.EOF {
tryWriteBuffer(buf)
finished()
return nil
}
if errors.Is(err, ErrNoRecordsInStream) {
// no records were written to the parquet file, so just return the buffer
// to the pool instead of passing it along to the next pipeline stage
buffers.PutBuffer(buf)
finished()
return nil
}
if err == nil {
tryWriteBuffer(buf)
}
return err
})
}
}
func uploadStream(ctx context.Context, cn snowflakeConn, r io.Reader, name string) error {
putQuery := fmt.Sprintf(putQueryTmpl, name)
putQuery = strings.ReplaceAll(putQuery, "\\", "\\\\") // Windows compatibility
_, err := cn.ExecContext(gosnowflake.WithFileStream(ctx, r), putQuery, nil)
if err != nil {
return err
}
return nil
}
func uploadAllStreams(
ctx context.Context,
cn snowflakeConn,
streams <-chan *bytes.Buffer,
concurrency int,
buffers *bufferPool,
uploadCallback func(string),
) error {
g, ctx := errgroup.WithContext(ctx)
g.SetLimit(concurrency)
var i int
// Loop through buffers as they become ready and assign to available upload workers
for buf := range streams {
select {
case <-ctx.Done():
// The context is canceled on error, so we wait for graceful shutdown of in-progress uploads.
// The gosnowflake.snowflakeFileTransferAgent does not currently propogate context, so we
// have to wait for uploads to finish for proper shutdown. (https://github.com/snowflakedb/gosnowflake/issues/1028)
return g.Wait()
default:
}
buf := buf // mutable loop variable
fileName := fmt.Sprintf("%d.parquet", i)
g.Go(func() error {
defer buffers.PutBuffer(buf)
defer uploadCallback(fileName)
return uploadStream(ctx, cn, buf, fileName)
})
i++
}
return g.Wait()
}
func executeCopyQuery(ctx context.Context, cn snowflakeConn, tableName string, filesToCopy *fileSet) error {
rows, err := cn.QueryContext(ctx, copyQuery, []driver.NamedValue{{Value: tableName}})
if err != nil {
return err
}
defer rows.Close()
fileColIdx := slices.Index(rows.Columns(), "file")
if fileColIdx < 0 {
// COPY response does not include 'file' column if no files we copied
return nil
}
for {
vals := make([]driver.Value, len(rows.Columns()))
err := rows.Next(vals)
if err == io.EOF {
break
}
if err != nil {
return err
}
file, ok := vals[fileColIdx].(string)
if !ok {
return fmt.Errorf("unexpected response for COPY, expected 'file' to be string, found: %T", vals[fileColIdx])
}
filesToCopy.Remove(file)
}
return nil
}
func runCopyTasks(ctx context.Context, cn snowflakeConn, tableName string, concurrency int) (func(string), func() error, func()) {
var filesToCopy fileSet
ctx, cancel := context.WithCancel(ctx)
g, ctx := errgroup.WithContext(ctx)
g.SetLimit(concurrency)
readyCh := make(chan struct{}, 1)
stopCh := make(chan interface{})
// Handler to signal that new data has been uploaded.
// Executing COPY will be a no-op if this has not been called since the last COPY was dispatched, so we wait for the signal.
readyFn := func(filename string) {
// keep track of each file uploaded to the stage, until it has been copied into the table successfully
filesToCopy.Add(filename)
// readyFn is a no-op if the shutdown signal has already been recieved
select {
case _, ok := <-stopCh:
if !ok {
return
}
default:
// Continue
}
// readyFn is a no-op if it already knows that at least 1 file is ready to be loaded
select {
case readyCh <- struct{}{}:
default:
return
}
}
// Handler to signal that ingestion pipeline has completed successfully.
// Executes COPY to finalize ingestion (may no-op if all files already loaded by bg workers) and waits for all COPYs to complete.
stopFn := func() error {
defer cancel()
close(stopCh)
close(readyCh)
maxRetries := 5 // maybe make configurable?
for attempt := 0; attempt < maxRetries+1; attempt++ {
if attempt > 0 {
// backoff by 100ms, 200ms, 400ms, ...
factor := time.Duration(math.Pow(2, float64(attempt-1)))
backoff := 100 * factor * time.Millisecond
time.Sleep(backoff)
}
if err := executeCopyQuery(ctx, cn, tableName, &filesToCopy); err != nil {
return err
}
if filesToCopy.Len() == 0 {
// all files successfully copied
return g.Wait()
}
}
return fmt.Errorf("some files not loaded by COPY command, %d files remain after %d retries", filesToCopy.Len(), maxRetries)
}
// Handler to signal that ingestion pipeline failed and COPY operations should not proceed.
// Stops the dispatch of new bg workers and cancels all in-progress COPY operations.
cancelFn := func() {
defer cancel()
close(stopCh)
close(readyCh)
}
go func() {
for {
// Block until there is at least 1 new file available for copy, or it's time to shutdown
select {
case <-stopCh:
return
case <-ctx.Done():
return
case _, ok := <-readyCh:
if !ok {
return
}
// Proceed to start a new COPY job
}
g.Go(func() error {
return executeCopyQuery(ctx, cn, tableName, &filesToCopy)
})
}
}()
return readyFn, stopFn, cancelFn
}
func countRowsInTable(ctx context.Context, db *sql.DB, tableName string) (int64, error) {
var nrows int64
row := db.QueryRowContext(ctx, countQuery, tableName)
if err := row.Scan(&nrows); err != nil {
return 0, errToAdbcErr(adbc.StatusIO, err)
}
return nrows, nil
}
// Initializes a sync.Pool of *bytes.Buffer.
// Extra space is preallocated so that the Parquet footer can be written after reaching target file size without growing the buffer
func newBufferPool(size int) *bufferPool {
buffers := sync.Pool{
New: func() interface{} {
extraSpace := 1 * megabyte // TODO(joellubi): Generally works, but can this be smarter?
buf := make([]byte, 0, size+extraSpace)
return bytes.NewBuffer(buf)
},
}
return &bufferPool{&buffers}
}
type bufferPool struct {
*sync.Pool
}
func (bp *bufferPool) GetBuffer() *bytes.Buffer {
return bp.Pool.Get().(*bytes.Buffer)
}
func (bp *bufferPool) PutBuffer(buf *bytes.Buffer) {
buf.Reset()
bp.Pool.Put(buf)
}
type fileSet sync.Map
func (s *fileSet) Add(file string) {
basename := path.Base(file)
(*sync.Map)(s).Store(basename, nil)
}
func (s *fileSet) Remove(file string) {
basename := path.Base(file)
(*sync.Map)(s).Delete(basename)
}
func (s *fileSet) Len() int {
var items int
(*sync.Map)(s).Range(func(key, value any) bool {
items++
return true
})
return items
}