-
Notifications
You must be signed in to change notification settings - Fork 130
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Arrow Record Distributed Result Batches #544
Changes from 25 commits
9c26d05
a907324
857f8b6
a52ba51
fce1c82
4d0efe7
ee807bf
8943664
bb07f34
6dae32a
f56da50
ccfbe40
4874c89
701e083
b18a4de
5023beb
8b89936
82fbc2f
37799e5
d39e2e6
aedd242
da0f03b
f4e0b06
d9a0df1
3756cf8
b5a4331
ed3fe29
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ import ( | |
"sync" | ||
"time" | ||
|
||
"github.com/apache/arrow/go/arrow/array" | ||
"github.com/apache/arrow/go/arrow/ipc" | ||
"github.com/apache/arrow/go/arrow/memory" | ||
) | ||
|
@@ -34,6 +35,7 @@ type chunkDownloader interface { | |
getRowType() []execResponseRowType | ||
setNextChunkDownloader(downloader chunkDownloader) | ||
getNextChunkDownloader() chunkDownloader | ||
getResultBatches() []*ResultBatch | ||
} | ||
|
||
type snowflakeChunkDownloader struct { | ||
|
@@ -55,9 +57,11 @@ type snowflakeChunkDownloader struct { | |
ChunksFinalErrors []*chunkError | ||
ChunksMutex *sync.Mutex | ||
DoneDownloadCond *sync.Cond | ||
FirstBatch *ResultBatch | ||
NextDownloader chunkDownloader | ||
Qrmk string | ||
QueryResultFormat string | ||
ResultBatches []*ResultBatch | ||
RowSet rowSetType | ||
FuncDownload func(context.Context, *snowflakeChunkDownloader, int) | ||
FuncDownloadHelper func(context.Context, *snowflakeChunkDownloader, int) error | ||
|
@@ -88,6 +92,9 @@ func (scd *snowflakeChunkDownloader) nextResultSet() error { | |
} | ||
|
||
func (scd *snowflakeChunkDownloader) start() error { | ||
if usesDistributedBatches(scd.ctx) { | ||
return scd.startDistributedBatches() | ||
} | ||
scd.CurrentChunkSize = len(scd.RowSet.JSON) // cache the size | ||
scd.CurrentIndex = -1 // initial chunks idx | ||
scd.CurrentChunkIndex = -1 // initial chunk | ||
|
@@ -231,6 +238,13 @@ func (scd *snowflakeChunkDownloader) getRowType() []execResponseRowType { | |
return scd.RowSet.RowType | ||
} | ||
|
||
func (scd *snowflakeChunkDownloader) getResultBatches() []*ResultBatch { | ||
if scd.FirstBatch.Rec == nil { | ||
return scd.ResultBatches | ||
} | ||
return append([]*ResultBatch{scd.FirstBatch}, scd.ResultBatches...) | ||
} | ||
|
||
func getChunk( | ||
ctx context.Context, | ||
scd *snowflakeChunkDownloader, | ||
|
@@ -245,6 +259,33 @@ func getChunk( | |
return newRetryHTTP(ctx, scd.sc.rest.Client, http.NewRequest, u, headers, timeout).execute() | ||
} | ||
|
||
func (scd *snowflakeChunkDownloader) startDistributedBatches() error { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. startArrowBatches() |
||
var err error | ||
chunkMetaLen := len(scd.ChunkMetas) | ||
firstArrowChunk := buildFirstArrowChunk(scd.RowSet.RowSetBase64) | ||
scd.FirstBatch = &ResultBatch{ | ||
idx: 0, | ||
scd: scd, | ||
funcDownloadHelper: scd.FuncDownloadHelper, | ||
} | ||
// decode first chunk if possible | ||
if firstArrowChunk.allocator != nil { | ||
scd.FirstBatch.Rec, err = firstArrowChunk.decodeArrowBatch(scd) | ||
if err != nil { | ||
return err | ||
} | ||
} | ||
scd.ResultBatches = make([]*ResultBatch, chunkMetaLen) | ||
for i := 0; i < chunkMetaLen; i++ { | ||
scd.ResultBatches[i] = &ResultBatch{ | ||
idx: i, | ||
scd: scd, | ||
funcDownloadHelper: scd.FuncDownloadHelper, | ||
} | ||
} | ||
return nil | ||
} | ||
|
||
/* largeResultSetReader is a reader that wraps the large result set with leading and tailing brackets. */ | ||
type largeResultSetReader struct { | ||
status int | ||
|
@@ -380,6 +421,12 @@ func decodeChunk(scd *snowflakeChunkDownloader, idx int, bufStream *bufio.Reader | |
int(scd.totalUncompressedSize()), | ||
memory.NewGoAllocator(), | ||
} | ||
if usesDistributedBatches(scd.ctx) { | ||
if scd.ResultBatches[idx].Rec, err = arc.decodeArrowBatch(scd); err != nil { | ||
return err | ||
} | ||
return nil | ||
} | ||
highPrec := higherPrecisionEnabled(scd.ctx) | ||
respd, err = arc.decodeArrowChunk(scd.RowSet.RowType, highPrec) | ||
if err != nil { | ||
|
@@ -512,6 +559,10 @@ func (scd *streamChunkDownloader) getRowType() []execResponseRowType { | |
return scd.RowSet.RowType | ||
} | ||
|
||
func (scd *streamChunkDownloader) getResultBatches() []*ResultBatch { | ||
return nil | ||
} | ||
|
||
func useStreamDownloader(ctx context.Context) bool { | ||
val := ctx.Value(streamChunkDownload) | ||
if val == nil { | ||
|
@@ -630,3 +681,32 @@ func copyChunkStream(body io.Reader, rows chan<- []*string) error { | |
} | ||
return nil | ||
} | ||
|
||
// ResultBatch object represents a chunk of data, or subset of rows, retrievable in array.Record format | ||
type ResultBatch struct { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ArrowBatch |
||
Rec *[]array.Record | ||
idx int | ||
scd *snowflakeChunkDownloader | ||
funcDownloadHelper func(context.Context, *snowflakeChunkDownloader, int) error | ||
} | ||
sfc-gh-jbahk marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
// Fetch returns an array of records representing a chunk in the query | ||
func (rb *ResultBatch) Fetch() (*[]array.Record, error) { | ||
// chunk has already been downloaded | ||
if rb.Rec != nil { | ||
return rb.Rec, nil | ||
} | ||
if err := rb.funcDownloadHelper(context.Background(), rb.scd, rb.idx); err != nil { | ||
return nil, err | ||
} | ||
return rb.Rec, nil | ||
} | ||
|
||
func usesDistributedBatches(ctx context.Context) bool { | ||
val := ctx.Value(distributedResultBatches) | ||
if val == nil { | ||
return false | ||
} | ||
a, ok := val.(bool) | ||
return a && ok | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -5,6 +5,7 @@ package gosnowflake | |
import ( | ||
"bytes" | ||
"context" | ||
"database/sql/driver" | ||
"encoding/json" | ||
"fmt" | ||
"io" | ||
|
@@ -387,3 +388,44 @@ func TestWithStreamDownloader(t *testing.T) { | |
} | ||
}) | ||
} | ||
|
||
func TestWithDistributedResultBatches(t *testing.T) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This test succinctly demonstrates the API pathway, but is there another test that can be added to demonstrate that these batches are indeed fetched in a distributed manner? 100 rows is not a lot and would probably be fetched within one chunk. This test doesn't demonstrate to me that these batches can be called independently. More importantly, with these changes, do you see your memory issues improved? Solved? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. sounds good - I will have a first PR iteration with the logic changes, and a second only focusing on this test so that we can iterate on the former first! In terms of memory, it's fully up to the user to handle the workers as they wish - in our case we optimized the client side for parallel downloading, but releasing the records as soon as they've been read and written, and we have not seen any memory issue (queried all the way up to 10GB with very little memory pressure but really good performance). There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ^ changed it to 3000 rows which is approximately 6 ResultBatch objects (ran it a few times and, on average, it's 6 chunks of data) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The test seems better but is there a way to explicitly illustrate these batches being fetched on in different worker nodes? Something like multiple goroutines, perhaps? Maybe there are some other ways to demonstrate this but at face value, this test doesn't tell me much. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. just updated the test with clear download workers - it is now checking also that the metadata received (rowcount) is correct :) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. TestWithArrowBatches() |
||
ctx := WithDistributedResultBatches(context.Background()) | ||
numrows := 3000 // approximately 6 ResultBatch objects | ||
config, err := ParseDSN(dsn) | ||
if err != nil { | ||
t.Error(err) | ||
} | ||
sc, err := buildSnowflakeConn(ctx, *config) | ||
if err != nil { | ||
t.Error(err) | ||
} | ||
if err = authenticateWithConfig(sc); err != nil { | ||
t.Error(err) | ||
} | ||
|
||
query := fmt.Sprintf(selectRandomGenerator, numrows) | ||
rows, err := sc.QueryContext(ctx, query, []driver.NamedValue{}) | ||
if err != nil { | ||
t.Error(err) | ||
} | ||
defer rows.Close() | ||
batches, err := rows.(*snowflakeRows).GetBatches() | ||
if err != nil { | ||
t.Error(err) | ||
} | ||
|
||
cnt := 0 | ||
for _, b := range batches { | ||
_, err := b.Fetch() | ||
if err != nil { | ||
t.Error(err) | ||
} | ||
for _, r := range *b.Rec { | ||
cnt += int(r.NumRows()) | ||
} | ||
} | ||
if cnt != numrows { | ||
t.Errorf("number of rows didn't match. expected: %v, got: %v", numrows, cnt) | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
getArrowBatches() []*ArrowBatch