-
Notifications
You must be signed in to change notification settings - Fork 100
/
sql_store.go
303 lines (266 loc) · 8.1 KB
/
sql_store.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
package snowflake
import (
"context"
"database/sql"
sqld "database/sql/driver"
"fmt"
"io"
"os"
"strconv"
"sync"
"time"
"github.com/apache/arrow/go/v14/arrow"
"github.com/apache/arrow/go/v14/arrow/memory"
"github.com/apache/arrow/go/v14/parquet"
"github.com/apache/arrow/go/v14/parquet/compress"
"github.com/apache/arrow/go/v14/parquet/pqarrow"
"github.com/c2h5oh/datasize"
"github.com/mitchellh/mapstructure"
"github.com/rilldata/rill/runtime/drivers"
"github.com/rilldata/rill/runtime/pkg/observability"
sf "github.com/snowflakedb/gosnowflake"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)
// recommended size is 512MB - 1GB, entire data is buffered in memory before its written to disk
const rowGroupBufferSize = int64(datasize.MB) * 512
// Query implements drivers.SQLStore
func (c *connection) Query(ctx context.Context, props map[string]any) (drivers.RowIterator, error) {
return nil, drivers.ErrNotImplemented
}
// QueryAsFiles implements drivers.SQLStore.
// Fetches query result in arrow batches.
// As an alternative (or in case of memory issues) consider utilizing Snowflake "COPY INTO <location>" feature,
// see https://docs.snowflake.com/en/sql-reference/sql/copy-into-location
func (c *connection) QueryAsFiles(ctx context.Context, props map[string]any, opt *drivers.QueryOption, p drivers.Progress) (drivers.FileIterator, error) {
srcProps, err := parseSourceProperties(props)
if err != nil {
return nil, err
}
var dsn string
if srcProps.DSN != "" { // get from src properties
dsn = srcProps.DSN
} else if url, ok := c.config["dsn"].(string); ok && url != "" { // get from driver configs
dsn = url
} else {
return nil, fmt.Errorf("the property 'dsn' is required for Snowflake. Provide 'dsn' in the YAML properties or pass '--var connector.snowflake.dsn=...' to 'rill start'")
}
parallelFetchLimit := 15
if limit, ok := c.config["parallel_fetch_limit"].(string); ok {
parallelFetchLimit, err = strconv.Atoi(limit)
if err != nil {
return nil, err
}
}
db, err := sql.Open("snowflake", dsn)
if err != nil {
return nil, err
}
ctx = sf.WithArrowAllocator(sf.WithArrowBatches(ctx), memory.DefaultAllocator)
conn, err := db.Conn(ctx)
if err != nil {
db.Close()
return nil, err
}
var rows sqld.Rows
err = conn.Raw(func(x interface{}) error {
rows, err = x.(sqld.QueryerContext).QueryContext(ctx, srcProps.SQL, nil)
return err
})
if err != nil {
conn.Close()
db.Close()
return nil, err
}
batches, err := rows.(sf.SnowflakeRows).GetArrowBatches()
if err != nil {
return nil, err
}
if len(batches) == 0 {
// empty result
return nil, drivers.ErrNoRows
}
// the number of returned rows is unknown at this point, only the number of batches and output files
p.Target(1, drivers.ProgressUnitFile)
return &fileIterator{
ctx: ctx,
db: db,
conn: conn,
rows: rows,
batches: batches,
progress: p,
limitInBytes: opt.TotalLimitInBytes,
parallelFetchLimit: parallelFetchLimit,
logger: c.logger,
}, nil
}
type fileIterator struct {
ctx context.Context
db *sql.DB
conn *sql.Conn
rows sqld.Rows
batches []*sf.ArrowBatch
progress drivers.Progress
limitInBytes int64
logger *zap.Logger
// Computed while iterating
totalRecords int64
tempFilePath string
downloaded bool
// Max number of batches to fetch in parallel
parallelFetchLimit int
}
// Close implements drivers.FileIterator.
func (f *fileIterator) Close() error {
return os.Remove(f.tempFilePath)
}
// Next implements drivers.FileIterator.
// Query result is written to a single parquet file.
func (f *fileIterator) Next() ([]string, error) {
if f.downloaded {
return nil, io.EOF
}
// close db resources early
defer func() {
f.rows.Close()
f.conn.Close()
f.db.Close()
}()
f.logger.Debug("downloading results in parquet file", observability.ZapCtx(f.ctx))
// create a temp file
fw, err := os.CreateTemp("", "temp*.parquet")
if err != nil {
return nil, err
}
defer fw.Close()
f.tempFilePath = fw.Name()
f.downloaded = true
tf := time.Now()
defer func() {
f.logger.Debug("time taken to write arrow records in parquet file", zap.Duration("duration", time.Since(tf)), observability.ZapCtx(f.ctx))
}()
firstBatch, err := f.batches[0].Fetch()
if err != nil {
return nil, err
}
if len(*firstBatch) == 0 {
// empty result
return nil, drivers.ErrNoRows
}
// common schema
schema := (*firstBatch)[0].Schema()
for _, f := range schema.Fields() {
if f.Type.ID() == arrow.TIME32 || f.Type.ID() == arrow.TIME64 {
return nil, fmt.Errorf("TIME data type (column %q) is not currently supported, "+
"consider excluding it or casting it to another data type", f.Name)
}
}
writer, err := pqarrow.NewFileWriter(schema, fw,
parquet.NewWriterProperties(
parquet.WithCompression(compress.Codecs.Snappy),
parquet.WithRootRepetition(parquet.Repetitions.Required),
// duckdb has issues reading statistics of string type generated with this write
// column statistics may not be useful if full file need to be ingested so better to disable to save computations
parquet.WithStats(false),
),
pqarrow.NewArrowWriterProperties(pqarrow.WithStoreSchema()))
if err != nil {
return nil, err
}
defer writer.Close()
// write arrow records to parquet file
// the following iteration might be memory intensive
// since batches are organized as a slice and every batch caches its content
f.logger.Debug("starting to fetch and process arrow batches",
zap.Int("batches", len(f.batches)), zap.Int("parallel_fetch_limit", f.parallelFetchLimit))
// Fetch batches async
errGrp, _ := errgroup.WithContext(f.ctx)
errGrp.SetLimit(f.parallelFetchLimit)
// mutex to protect file writes
var mu sync.Mutex
batchesLeft := len(f.batches)
for _, batch := range f.batches {
b := batch
errGrp.Go(func() error {
fetchStart := time.Now()
records, err := b.Fetch()
if err != nil {
return err
}
f.logger.Debug(
"fetched an arrow batch",
zap.Duration("duration", time.Since(fetchStart)),
zap.Int("row_count", b.GetRowCount()),
)
mu.Lock()
defer mu.Unlock()
writeStart := time.Now()
for _, rec := range *records {
if writer.RowGroupTotalBytesWritten() >= rowGroupBufferSize {
writer.NewBufferedRowGroup()
}
if err := writer.WriteBuffered(rec); err != nil {
return err
}
fileInfo, err := os.Stat(fw.Name())
if err == nil { // ignore error
if fileInfo.Size() > f.limitInBytes {
return drivers.ErrStorageLimitExceeded
}
}
}
batchesLeft--
f.logger.Debug(
"wrote an arrow batch to a parquet file",
zap.Float64("progress", float64(len(f.batches)-batchesLeft)/float64(len(f.batches))*100),
zap.Int("row_count", b.GetRowCount()),
zap.Duration("write_duration", time.Since(writeStart)),
)
f.totalRecords += int64(b.GetRowCount())
return nil
})
}
if err := errGrp.Wait(); err != nil {
return nil, err
}
writer.Close()
fw.Close()
fileInfo, err := os.Stat(fw.Name())
if err != nil {
return nil, err
}
f.progress.Observe(1, drivers.ProgressUnitFile)
f.logger.Debug("size of file", zap.String("size", datasize.ByteSize(fileInfo.Size()).HumanReadable()), observability.ZapCtx(f.ctx))
return []string{fw.Name()}, nil
}
// Size implements drivers.FileIterator.
func (f *fileIterator) Size(unit drivers.ProgressUnit) (int64, bool) {
switch unit {
case drivers.ProgressUnitFile:
return 1, true
// the number of records is unknown until the end of iteration
case drivers.ProgressUnitRecord:
return f.totalRecords, true
default:
return 0, false
}
}
func (f *fileIterator) Format() string {
return ""
}
var _ drivers.FileIterator = &fileIterator{}
type sourceProperties struct {
SQL string `mapstructure:"sql"`
DSN string `mapstructure:"dsn"`
}
func parseSourceProperties(props map[string]any) (*sourceProperties, error) {
conf := &sourceProperties{}
err := mapstructure.Decode(props, conf)
if err != nil {
return nil, err
}
if conf.SQL == "" {
return nil, fmt.Errorf("property 'sql' is mandatory for connector \"snowflake\"")
}
return conf, err
}