zeroshade commented on code in PR #152:
URL: https://github.com/apache/spark-connect-go/pull/152#discussion_r2808277505
##########
spark/client/client.go:
##########
@@ -434,6 +444,110 @@ func (c *ExecutePlanClient) ToTable() (*types.StructType,
arrow.Table, error) {
}
}
+// ToRecordSequence returns a single Seq2 iterator
+func (c *ExecutePlanClient) ToRecordSequence(ctx context.Context)
iter.Seq2[arrow.Record, error] {
+ // Return Seq2 iterator that directly yields results as they arrive,
upstream callers can convert this as needed
+ iterator := func(yield func(arrow.Record, error) bool) {
Review Comment:
just do `return func(yield func(arrow.Record, error) bool) {` instead of
`iterator := ....` and `return iterator`
##########
spark/sql/dataframe.go:
##########
@@ -936,6 +942,17 @@ func (df *dataFrameImpl) ToArrow(ctx context.Context)
(*arrow.Table, error) {
return &table, nil
}
+func (df *dataFrameImpl) StreamRows(ctx context.Context) (types.RowPull2,
error) {
+ responseClient, err := df.session.client.ExecutePlan(ctx,
df.createPlan())
+ if err != nil {
+ return nil, sparkerrors.WithType(fmt.Errorf("failed to execute
plan: %w", err), sparkerrors.ExecutionError)
+ }
+
+ seq2 := responseClient.ToRecordSequence(ctx)
+
+ return types.NewRowPull2(ctx, seq2), nil
Review Comment:
Why not just use `iter.Pull2`?
##########
spark/client/client.go:
##########
@@ -434,6 +443,151 @@ func (c *ExecutePlanClient) ToTable() (*types.StructType,
arrow.Table, error) {
}
}
+func (c *ExecutePlanClient) ToRecordBatches(ctx context.Context) (<-chan
arrow.Record, <-chan error, *types.StructType) {
+ recordChan := make(chan arrow.Record, 10)
+ errorChan := make(chan error, 1)
+
+ go func() {
+ defer func() {
+ // Ensure channels are always closed to prevent
goroutine leaks
+ close(recordChan)
+ close(errorChan)
+ }()
+
+ // Explicitly needed when tracking re-attachable execution.
+ c.done = false
Review Comment:
wouldn't this be a race condition? Should we be locking around accessing
`c.done`?
##########
spark/client/client.go:
##########
@@ -368,6 +369,15 @@ func (c *ExecutePlanClient) ToTable() (*types.StructType,
arrow.Table, error) {
c.done = false
for {
resp, err := c.responseStream.Recv()
+ if err != nil {
+ fmt.Printf("DEBUG: Recv error: %v, is EOF: %v\n", err,
errors.Is(err, io.EOF))
+ }
+ if err == nil && resp != nil {
+ fmt.Printf("DEBUG: Received response type: %T\n",
resp.ResponseType)
+ if _, ok :=
resp.ResponseType.(*proto.ExecutePlanResponse_ResultComplete_); ok {
+ fmt.Println("DEBUG: Got ResultComplete!")
+ }
+ }
Review Comment:
Probably shouldn't keep these debug lines in here. At minimum, use
`log.Println` if we're not gonna switch to something like zerolog yet.
##########
spark/sql/types/rowiterator.go:
##########
@@ -0,0 +1,88 @@
+package types
+
+import (
+ "context"
+ "errors"
+ "io"
+ "iter"
+ "sync/atomic"
+
+ "github.com/apache/arrow-go/v18/arrow"
+)
+
+type RowPull2 = iter.Seq2[Row, error]
+
+// NewRowSequence flattens record batches to a sequence of rows stream.
+func NewRowSequence(ctx context.Context, recordSeq iter.Seq2[arrow.Record,
error]) iter.Seq2[Row, error] {
+ return func(yield func(Row, error) bool) {
+ for rec, recErr := range recordSeq {
+ select {
+ case <-ctx.Done():
+ _ = yield(nil, ctx.Err())
+ return
+ default:
+ }
+ if recErr != nil {
+ // forward upstream error once, then stop
+ _ = yield(nil, recErr)
+ return
+ }
+ if rec == nil {
+ _ = yield(nil, errors.New("expected
arrow.Record to contain non-nil Rows, got nil"))
+ return
+ }
+
+ rows, err := func() ([]Row, error) {
+ defer rec.Release()
+ return ReadArrowRecordToRows(rec)
+ }()
+ if err != nil {
+ _ = yield(nil, err)
+ return
+ }
+ for _, row := range rows {
+ if !yield(row, nil) {
+ return
+ }
+ }
Review Comment:
It feels like we could simplify this if `ReadArrowRecordToRows` performed
the release inside the iterator etc.
Something like:
```go
for row := range rowIterFromRecord(rec) {
if !yield(row, nil) {
return
}
}
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]