This is an automated email from the ASF dual-hosted git repository.
lidavidm pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-adbc.git
The following commit(s) were added to refs/heads/main by this push:
new 84b96dfae feat(go/adbc/driver/snowflake): Keep track of all files
copied and skip empty files in bulk_ingestion (#2106)
84b96dfae is described below
commit 84b96dfaeb80be64643cf5ff2d0e71fa17547a46
Author: Joel Lubinitsky <[email protected]>
AuthorDate: Thu Aug 29 20:59:44 2024 -0400
feat(go/adbc/driver/snowflake): Keep track of all files copied and skip
empty files in bulk_ingestion (#2106)
Fixes: #2094
Two primary changes:
- Skip uploading parquet files that never had any rows written to them
- This reduces number of files Snowflake must keep track of, improving
success rate of the COPY command
- Keep track of all files uploaded to stage and ensure they are actually
copied
- Retry the COPY command up to 5 times with backoff if files any files
haven't been copied
**Snowflake Test Output**
```
➜ adbc git:(gh-2094) ✗ go test -v ./driver/snowflake/...
=== RUN TestIngestBatchedParquetWithFileLimit
--- PASS: TestIngestBatchedParquetWithFileLimit (0.00s)
=== RUN TestValidation
=== RUN TestValidation/TestNewDatabase
=== RUN TestValidation/TestAutocommitDefault
=== RUN TestValidation/TestAutocommitToggle
=== RUN TestValidation/TestCloseConnTwice
=== RUN TestValidation/TestConcurrent
=== RUN TestValidation/TestGetSetOptions
=== RUN TestValidation/TestMetadataCurrentCatalog
=== RUN TestValidation/TestMetadataCurrentDbSchema
=== RUN TestValidation/TestMetadataGetInfo
=== RUN TestValidation/TestMetadataGetObjectsColumns
=== RUN
TestValidation/TestMetadataGetObjectsColumns/depth_catalog_no_filter
=== RUN
TestValidation/TestMetadataGetObjectsColumns/depth_dbSchema_no_filter
=== RUN TestValidation/TestMetadataGetObjectsColumns/depth_table_no_filter
=== RUN
TestValidation/TestMetadataGetObjectsColumns/depth_column_no_filter
=== RUN TestValidation/TestMetadataGetObjectsColumns/filter_catalog_valid
=== RUN
TestValidation/TestMetadataGetObjectsColumns/filter_catalog_invalid
=== RUN TestValidation/TestMetadataGetObjectsColumns/filter_dbSchema_valid
=== RUN
TestValidation/TestMetadataGetObjectsColumns/filter_dbSchema_invalid
=== RUN TestValidation/TestMetadataGetObjectsColumns/filter_table_valid
=== RUN TestValidation/TestMetadataGetObjectsColumns/filter_table_invalid
=== RUN TestValidation/TestMetadataGetObjectsColumns/filter_column:_in%
=== RUN TestValidation/TestMetadataGetStatistics
=== RUN TestValidation/TestMetadataGetTableSchema
=== RUN TestValidation/TestMetadataGetTableTypes
=== RUN TestValidation/TestNewConn
=== RUN TestValidation/TestNewStatement
=== RUN TestValidation/TestSQLPrepareGetParameterSchema
=== RUN TestValidation/TestSQLPrepareSelectNoParams
=== RUN TestValidation/TestSQLPrepareSelectParams
=== RUN TestValidation/TestSqlExecuteSchema
=== RUN TestValidation/TestSqlExecuteSchema/no_query
=== RUN TestValidation/TestSqlExecuteSchema/query
=== RUN TestValidation/TestSqlExecuteSchema/prepared
=== RUN TestValidation/TestSqlIngestAppend
=== RUN TestValidation/TestSqlIngestCreateAppend
=== RUN TestValidation/TestSqlIngestErrors
=== RUN TestValidation/TestSqlIngestErrors/ingest_without_bind
=== RUN TestValidation/TestSqlIngestErrors/append_to_nonexistent_table
time="2024-08-28T10:58:21-04:00" level=error msg="error: 002003 (42S02):
SQL compilation error:\nObject '\"bulk_ingest\"' does not exist or not
authorized." func="gosnowflake.(*snowflakeConn).queryContextInternal"
file="connection.go:398"
=== RUN
TestValidation/TestSqlIngestErrors/overwrite_and_incompatible_schema
=== RUN TestValidation/TestSqlIngestInts
=== RUN TestValidation/TestSqlIngestReplace
=== RUN TestValidation/TestSqlPartitionedInts
=== RUN TestValidation/TestSqlPrepareErrorParamCountMismatch
--- PASS: TestValidation (80.67s)
--- PASS: TestValidation/TestNewDatabase (0.00s)
--- PASS: TestValidation/TestAutocommitDefault (0.33s)
--- PASS: TestValidation/TestAutocommitToggle (1.03s)
--- PASS: TestValidation/TestCloseConnTwice (0.33s)
--- PASS: TestValidation/TestConcurrent (0.95s)
--- PASS: TestValidation/TestGetSetOptions (0.29s)
--- PASS: TestValidation/TestMetadataCurrentCatalog (0.45s)
--- PASS: TestValidation/TestMetadataCurrentDbSchema (0.44s)
--- PASS: TestValidation/TestMetadataGetInfo (0.29s)
--- PASS: TestValidation/TestMetadataGetObjectsColumns (45.51s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/depth_catalog_no_filter (0.88s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/depth_dbSchema_no_filter (0.92s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/depth_table_no_filter (3.05s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/depth_column_no_filter (5.61s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/filter_catalog_valid (4.90s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/filter_catalog_invalid (4.23s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/filter_dbSchema_valid (4.92s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/filter_dbSchema_invalid (3.70s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/filter_table_valid (5.59s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/filter_table_invalid (4.42s)
--- PASS:
TestValidation/TestMetadataGetObjectsColumns/filter_column:_in% (5.65s)
--- PASS: TestValidation/TestMetadataGetStatistics (0.33s)
--- PASS: TestValidation/TestMetadataGetTableSchema (1.92s)
--- PASS: TestValidation/TestMetadataGetTableTypes (0.29s)
--- PASS: TestValidation/TestNewConn (0.30s)
--- PASS: TestValidation/TestNewStatement (0.39s)
--- PASS: TestValidation/TestSQLPrepareGetParameterSchema (0.30s)
--- PASS: TestValidation/TestSQLPrepareSelectNoParams (0.38s)
--- SKIP: TestValidation/TestSQLPrepareSelectParams (0.29s)
--- PASS: TestValidation/TestSqlExecuteSchema (0.52s)
--- PASS: TestValidation/TestSqlExecuteSchema/no_query (0.00s)
--- PASS: TestValidation/TestSqlExecuteSchema/query (0.07s)
--- PASS: TestValidation/TestSqlExecuteSchema/prepared (0.06s)
--- PASS: TestValidation/TestSqlIngestAppend (5.82s)
--- PASS: TestValidation/TestSqlIngestCreateAppend (5.72s)
--- PASS: TestValidation/TestSqlIngestErrors (1.08s)
--- PASS: TestValidation/TestSqlIngestErrors/ingest_without_bind
(0.00s)
--- PASS:
TestValidation/TestSqlIngestErrors/append_to_nonexistent_table (0.48s)
--- SKIP:
TestValidation/TestSqlIngestErrors/overwrite_and_incompatible_schema (0.00s)
--- PASS: TestValidation/TestSqlIngestInts (3.46s)
--- PASS: TestValidation/TestSqlIngestReplace (6.82s)
--- PASS: TestValidation/TestSqlPartitionedInts (0.29s)
--- SKIP: TestValidation/TestSqlPrepareErrorParamCountMismatch (0.31s)
=== RUN TestSnowflake
=== RUN TestSnowflake/TestAdditionalDriverInfo
=== RUN TestSnowflake/TestDecimalHighPrecision
=== RUN TestSnowflake/TestDescribeOnly
=== RUN TestSnowflake/TestEmptyResultSet
=== RUN TestSnowflake/TestIngestEmptyChunk
=== RUN TestSnowflake/TestIntDecimalLowPrecision
=== RUN TestSnowflake/TestJwtPrivateKey
driver_test.go:1888: apache/arrow-adbc#1364
=== RUN TestSnowflake/TestMetadataGetObjectsColumnsXdbc
=== RUN TestSnowflake/TestMetadataOnlyQuery
=== RUN TestSnowflake/TestNewDatabaseGetSetOptions
=== RUN TestSnowflake/TestNonIntDecimalLowPrecision
=== RUN TestSnowflake/TestSqlIngestDate64Type
=== RUN TestSnowflake/TestSqlIngestHighPrecision
=== RUN TestSnowflake/TestSqlIngestListType
=== RUN TestSnowflake/TestSqlIngestLowPrecision
=== RUN TestSnowflake/TestSqlIngestMapType
=== RUN TestSnowflake/TestSqlIngestRecordAndStreamAreEquivalent
=== RUN TestSnowflake/TestSqlIngestRoundtripTypes
=== RUN TestSnowflake/TestSqlIngestStructType
=== RUN TestSnowflake/TestSqlIngestTimestamp
=== RUN TestSnowflake/TestSqlIngestTimestampTypes
=== RUN TestSnowflake/TestStatementEmptyResultSet
=== RUN TestSnowflake/TestTimestampSnow
=== RUN TestSnowflake/TestUseHighPrecision
--- PASS: TestSnowflake (110.63s)
--- PASS: TestSnowflake/TestAdditionalDriverInfo (0.29s)
--- PASS: TestSnowflake/TestDecimalHighPrecision (27.19s)
--- PASS: TestSnowflake/TestDescribeOnly (0.53s)
--- PASS: TestSnowflake/TestEmptyResultSet (0.47s)
--- PASS: TestSnowflake/TestIngestEmptyChunk (4.00s)
--- PASS: TestSnowflake/TestIntDecimalLowPrecision (7.97s)
--- SKIP: TestSnowflake/TestJwtPrivateKey (0.50s)
--- PASS: TestSnowflake/TestMetadataGetObjectsColumnsXdbc (10.51s)
--- PASS: TestSnowflake/TestMetadataOnlyQuery (1.56s)
--- PASS: TestSnowflake/TestNewDatabaseGetSetOptions (0.30s)
--- PASS: TestSnowflake/TestNonIntDecimalLowPrecision (9.56s)
--- PASS: TestSnowflake/TestSqlIngestDate64Type (3.50s)
--- PASS: TestSnowflake/TestSqlIngestHighPrecision (3.63s)
--- PASS: TestSnowflake/TestSqlIngestListType (3.67s)
--- PASS: TestSnowflake/TestSqlIngestLowPrecision (3.46s)
--- PASS: TestSnowflake/TestSqlIngestMapType (3.21s)
--- PASS: TestSnowflake/TestSqlIngestRecordAndStreamAreEquivalent
(6.89s)
--- PASS: TestSnowflake/TestSqlIngestRoundtripTypes (5.33s)
--- PASS: TestSnowflake/TestSqlIngestStructType (4.11s)
--- PASS: TestSnowflake/TestSqlIngestTimestamp (5.05s)
--- PASS: TestSnowflake/TestSqlIngestTimestampTypes (4.17s)
--- PASS: TestSnowflake/TestStatementEmptyResultSet (0.66s)
--- PASS: TestSnowflake/TestTimestampSnow (0.79s)
--- PASS: TestSnowflake/TestUseHighPrecision (2.36s)
=== RUN TestJwtAuthenticationUnencryptedValue
driver_test.go:1810: Cannot find the `SNOWFLAKE_TEST_PKCS8_VALUE` value
--- SKIP: TestJwtAuthenticationUnencryptedValue (0.00s)
=== RUN TestJwtAuthenticationEncryptedValue
driver_test.go:1826: Cannot find the `SNOWFLAKE_TEST_PKCS8_EN_VALUE`
value
--- SKIP: TestJwtAuthenticationEncryptedValue (0.00s)
=== RUN TestIngestCancelContext
--- PASS: TestIngestCancelContext (6.40s)
PASS
ok github.com/apache/arrow-adbc/go/adbc/driver/snowflake 198.525s
```
---
go/adbc/driver/snowflake/bulk_ingestion.go | 157 ++++++++++++++++++++++-------
go/adbc/go.mod | 2 +-
go/adbc/go.sum | 4 +-
3 files changed, 126 insertions(+), 37 deletions(-)
diff --git a/go/adbc/driver/snowflake/bulk_ingestion.go
b/go/adbc/driver/snowflake/bulk_ingestion.go
index 7f3d6bbd2..7e12026e0 100644
--- a/go/adbc/driver/snowflake/bulk_ingestion.go
+++ b/go/adbc/driver/snowflake/bulk_ingestion.go
@@ -28,9 +28,12 @@ import (
"fmt"
"io"
"math"
+ "path"
"runtime"
+ "slices"
"strings"
"sync"
+ "time"
"github.com/apache/arrow-adbc/go/adbc"
"github.com/apache/arrow/go/v18/arrow"
@@ -60,6 +63,8 @@ var (
defaultCompressionCodec compress.Compression = compress.Codecs.Snappy
defaultCompressionLevel int = flate.DefaultCompression
+
+ ErrNoRecordsInStream = errors.New("no records in stream to write")
)
// Options for configuring bulk ingestion.
@@ -219,20 +224,13 @@ func (st *statement) ingestStream(ctx context.Context)
(nrows int64, err error)
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.Background() // TODO(joellubi): switch to
context.WithoutCancel(ctx) once we're on Go 1.21
+ 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
-
- // TODO(joellubi): simplify / improve with errors.Join(err,
countErr) once we're on Go 1.20
- if err == nil {
- err = errToAdbcErr(adbc.StatusInternal, countErr)
- return
- }
-
- // Failure in the pipeline itself
+ err = errors.Join(err, countErr)
if errors.Is(err, context.Canceled) {
err = errToAdbcErr(adbc.StatusCancelled, err)
} else {
@@ -261,7 +259,7 @@ func (st *statement) ingestStream(ctx context.Context)
(nrows int64, err error)
int(st.ingestOptions.writerConcurrency),
parquetProps,
arrowProps,
- pool.GetBuffer,
+ pool,
records,
buffers,
)
@@ -278,7 +276,7 @@ func (st *statement) ingestStream(ctx context.Context)
(nrows int64, err error)
// 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.PutBuffer, fileReady)
+ 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
@@ -326,6 +324,21 @@ func readRecords(ctx context.Context, rdr
array.RecordReader, out chan<- arrow.R
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,
@@ -342,27 +355,25 @@ func writeParquet(
var bytesWritten int64
for rec := range in {
- if rec.NumRows() == 0 {
- rec.Release()
- continue
- }
-
- err = pqWriter.Write(rec)
- rec.Release()
+ nbytes, err := writeRecordToParquet(pqWriter, rec)
if err != nil {
return err
}
+ bytesWritten += nbytes
if targetSize < 0 {
continue
}
-
- bytesWritten += pqWriter.RowGroupTotalBytesWritten()
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
}
@@ -374,7 +385,7 @@ func runParallelParquetWriters(
concurrency int,
parquetProps *parquet.WriterProperties,
arrowProps pqarrow.ArrowWriterProperties,
- newBuffer func() *bytes.Buffer,
+ buffers *bufferPool,
in <-chan arrow.Record,
out chan<- *bytes.Buffer,
) error {
@@ -416,13 +427,20 @@ func runParallelParquetWriters(
// Proceed to write a new file
}
- buf := newBuffer()
+ 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)
}
@@ -448,8 +466,8 @@ func uploadAllStreams(
cn snowflakeConn,
streams <-chan *bytes.Buffer,
concurrency int,
- freeBuffer func(*bytes.Buffer),
- uploadCallback func(),
+ buffers *bufferPool,
+ uploadCallback func(string),
) error {
g, ctx := errgroup.WithContext(ctx)
g.SetLimit(concurrency)
@@ -469,8 +487,8 @@ func uploadAllStreams(
buf := buf // mutable loop variable
fileName := fmt.Sprintf("%d.parquet", i)
g.Go(func() error {
- defer freeBuffer(buf)
- defer uploadCallback()
+ defer buffers.PutBuffer(buf)
+ defer uploadCallback(fileName)
return uploadStream(ctx, cn, buf, fileName)
})
@@ -479,7 +497,41 @@ func uploadAllStreams(
return g.Wait()
}
-func runCopyTasks(ctx context.Context, cn snowflakeConn, tableName string,
concurrency int) (func(), func() error, func()) {
+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)
@@ -489,7 +541,9 @@ func runCopyTasks(ctx context.Context, cn snowflakeConn,
tableName string, concu
// 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() {
+ 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 {
@@ -516,12 +570,26 @@ func runCopyTasks(ctx context.Context, cn snowflakeConn,
tableName string, concu
close(stopCh)
close(readyCh)
- _, err := cn.ExecContext(ctx, copyQuery,
[]driver.NamedValue{{Value: tableName}})
- if err != nil {
- return err
+ 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 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.
@@ -549,8 +617,7 @@ func runCopyTasks(ctx context.Context, cn snowflakeConn,
tableName string, concu
}
g.Go(func() error {
- _, err := cn.ExecContext(ctx, copyQuery,
[]driver.NamedValue{{Value: tableName}})
- return err
+ return executeCopyQuery(ctx, cn, tableName,
&filesToCopy)
})
}
}()
@@ -595,3 +662,25 @@ 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
+}
diff --git a/go/adbc/go.mod b/go/adbc/go.mod
index 8a169f623..ad9c05415 100644
--- a/go/adbc/go.mod
+++ b/go/adbc/go.mod
@@ -25,7 +25,7 @@ require (
github.com/bluele/gcache v0.0.2
github.com/golang/protobuf v1.5.4
github.com/google/uuid v1.6.0
- github.com/snowflakedb/gosnowflake v1.11.1-0.20240820132919-5649c7a3d6f6
+ github.com/snowflakedb/gosnowflake v1.11.1
github.com/stretchr/testify v1.9.0
github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a
golang.org/x/exp v0.0.0-20240808152545-0cdaa3abc0fa
diff --git a/go/adbc/go.sum b/go/adbc/go.sum
index 2b95dd435..0da064090 100644
--- a/go/adbc/go.sum
+++ b/go/adbc/go.sum
@@ -200,8 +200,8 @@ github.com/rogpeppe/go-internal v1.12.0
h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU
github.com/rogpeppe/go-internal v1.12.0/go.mod
h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4=
github.com/sirupsen/logrus v1.9.3
h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ=
github.com/sirupsen/logrus v1.9.3/go.mod
h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ=
-github.com/snowflakedb/gosnowflake v1.11.1-0.20240820132919-5649c7a3d6f6
h1:cNdbJtL8OMLwtQOWrprIHuJ/5egcfeQDrNBlF9f1cuE=
-github.com/snowflakedb/gosnowflake
v1.11.1-0.20240820132919-5649c7a3d6f6/go.mod
h1:WFe+8mpsapDaQjHX6BqJBKtfQCGlGD3lHKeDsKfpx2A=
+github.com/snowflakedb/gosnowflake v1.11.1
h1:E91s8vBOSroaSTLsyjO4QPkEuzGmZcCxEFQLg214mvk=
+github.com/snowflakedb/gosnowflake v1.11.1/go.mod
h1:WFe+8mpsapDaQjHX6BqJBKtfQCGlGD3lHKeDsKfpx2A=
github.com/stretchr/objx v0.1.0/go.mod
h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/objx v0.4.0/go.mod
h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
github.com/stretchr/objx v0.5.0/go.mod
h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=