[SPARK-52780] Add StreamRows and Arrow Record Streaming#152
[SPARK-52780] Add StreamRows and Arrow Record Streaming#152caldempsey wants to merge 21 commits intoapache:masterfrom
Conversation
caldempsey
left a comment
There was a problem hiding this comment.
There are a number of places we can improve the performance of the Spark Connect Go Client where it matters, namely by pre-allocating resources and re-using pointers to those allocations to reduce GC pressure. I think this could speed things up substantially.
spark/client/base/base.go
Outdated
|
|
||
| type ExecuteResponseStream interface { | ||
| ToTable() (*types.StructType, arrow.Table, error) | ||
| ToRecordBatches(ctx context.Context) (<-chan arrow.Record, <-chan error, *types.StructType) |
There was a problem hiding this comment.
Added so we can consume Apache Arrow record batches rather than the entire table at once, in truth, ToTable should re-use private methods w/ ToRecordBatches, but I've avoided this to avoid breaking clients for now.
There was a problem hiding this comment.
Rather than exposing a channel directly, it might make more sense to use either an array.RecordReader or an iter.Seq2[arrow.Record, error].
Either that, or create something that wraps the two channels and produces one of those, simply to make it easier to consume for users.
There was a problem hiding this comment.
Thanks! I'll apply this comment (and all the others) in the next round of commits 🥳
|
Seem to be some issues, will resolve. |
|
I will take a look at the PR. It would be great to compare to #103 and see how the native go iterators are different from what you propose here. Thanks for your contributions! |
|
@grundprinzip Hey dude! Thanks a lot! This works really well but failing one edge case in my E2E test (real spark cluster, on Java (not Photon) Databricks, which is sending EOF as its 'I'm done' cue). Happy paths work, and most of the time if you cancel the context mid-stream, everything cancels gracefully, so I believe we're doing everything right on our end. Infrequently I'm observing a panic: I've moved this to WIP while I pepper it with debug statements (on my own master branch so I can easily pull my own go.mod) and find the root cause. Something doesn't implement signals right. I intend to find out what. To repro:
I took a look at your PR. Happy to accept Hoping to put some elbow grease in to get SC for Go off the ground. |
spark/client/client.go
Outdated
| case recordChan <- record: | ||
| // Successfully sent |
There was a problem hiding this comment.
All this effort just so we can do this. Again, we should opt in to a DRY implementation with ToTable.
There was a problem hiding this comment.
Any ideas how to improve this? I'm not crazy familiar with golang and the idiomatic implementation of channels.
There was a problem hiding this comment.
Will have a think, and make a proposal, then run that by you in this thread before implementation.
There was a problem hiding this comment.
left this for now, I think we can punt this down the road, mainly because I don't think it needs to be the focus of this PR but maybe the focus of a following one
…atabricks/Spark signal done processing rows)
There was a problem hiding this comment.
@grundprinzip Hey! I worked out the panic. It looks like the error I mentioned before only occurs when running the Databricks ODBC driver adjacent to it. There has to be some kind of signal handling collision between Spark Connect and that instance. Removing the Databricks ODBC driver instance from my main thread (a pretty big testing app) resolved the issue.
signal 13 received but handler not on signal stack
mp.gsignal stack [0xc000584000 0xc00058c000], mp.g0 stack [0x7fffb07f9280 0x7fffb0ff8e80], sp=0xc000815e20
fatal error: non-Go code set up signal handler without SA_ONSTACK flag
The introduction of context cancellation to Spark Connect only came in at about v40, and I was running v34 before, which means I think it might be worth internal testing the combination of cancelled context in Spark Connect on v40 with the Databricks ODBC Driver. A pretty rare edge case, I know, but you seem to work at Databricks and might want to shout at the guys at Simba to make sure they are handling Signal gracefully. Tagging @alexott onto this one.
I think we can unblock this PR right now. I'll see if I can get a core dump to prove out the issue being related to Simba Spark ODBC, but the problem cannot be reproduced without it, and the PR otherwise seems feature complete.
spark/client/base/base.go
Outdated
|
|
||
| type ExecuteResponseStream interface { | ||
| ToTable() (*types.StructType, arrow.Table, error) | ||
| ToRecordBatches(ctx context.Context) (<-chan arrow.Record, <-chan error, *types.StructType) |
There was a problem hiding this comment.
Would be good to add comments on the how this is intended to be used and what the difference to the ToTable is.
There was a problem hiding this comment.
100% I'll add this to the next commit.
spark/client/client.go
Outdated
| close(recordChan) | ||
| close(errorChan) |
There was a problem hiding this comment.
Does close require any error handling or is this final state.
There was a problem hiding this comment.
Final state here. Once closed, no more sends are allowed (sending on a closed channel panics), but you can still receive any buffered items, and subsequent receives yield the zero value immediately. Then attempting to close a closed channel will panic.
There was a problem hiding this comment.
this is outdated now, we use seq2 for pulls which simplifies a bit
spark/client/client.go
Outdated
| }() | ||
|
|
||
| // Explicitly needed when tracking re-attachable execution. | ||
| c.done = false |
There was a problem hiding this comment.
re-attachable execution is optional, we need to make sure it works with both modes.
There was a problem hiding this comment.
I'm not exactly sure how this is supposed to be implemented, any similar code or resources you can point out before I run over this again?
There was a problem hiding this comment.
wouldn't this be a race condition? Should we be locking around accessing c.done?
There was a problem hiding this comment.
Taking another look at this. I've spliced in the approach from ToTable(), originally written by @grundprinzip. Instead of writing to the shared c.done field, ToRecordSequence now tracks completion with a local done variable inside the closure.
After EOF, it checks if c.opts.ReattachExecution && !done and yields an error, the same way ToTable() does. This removes the race condition because nothing shared is being mutated, and it behaves correctly in both reattachable and non-reattachable modes.
I do think we should DRY this up eventually, but judiciously. I've kept the code WET for now intentionally. I want both code paths to remain directly comparable until I fully understand the domain. Having @grundprinzip's original logic in ToTable() sitting side by side against my equivalent in ToRecordSequence makes it much easier to reason about correctness and spot differences.
So rather than DRY it up in this PR, I'd like to take ownership of a fast follow-up where I consolidate ToRecordSequence and ToTable once this gets merged.
That said, happy to take a crack. I'm being cautious since this is an open source project, and I'd rather not introduce a negative diff that risks breaking users when the current approach is correct, comparable, and maintains parity between the two separate critical paths.
spark/sql/dataframe.go
Outdated
| Take(ctx context.Context, limit int32) ([]types.Row, error) | ||
| // ToArrow returns the Arrow representation of the DataFrame. | ||
| ToArrow(ctx context.Context) (*arrow.Table, error) | ||
| ToLocalIterator(ctx context.Context) (types.RowIterator, error) |
There was a problem hiding this comment.
The name local iterator is a bit misleading because this has a specific meaning in the context of Spark on the driver. I'm not opposed to naming it like this but it might lead to confusion. What about just calling it Iter().
There was a problem hiding this comment.
The second question I had was about using Golangs native iterator types to support iter.Seq or iter.Seq2 essentially.
One good improvement to the current code (but can be done in a follow-up) would be to replace the All() implementation using this iterator based one so that we avoid buffering the whole table before returning the iterator.
There was a problem hiding this comment.
Yeah, I'd prefer to rename this. I don't like the name ToLocalIterator because it doesn't tell us anything about the underlying mechanism. When I tested with Scala, I was surprised to find I can use ToLocalIterator in Scala Spark Connect on streams of Arrow batches. IterArrowStream or StreamRows would be so much better. Let me know what you prefer, I'm glad we're on the same page here (for different reasons)!
There was a problem hiding this comment.
I would be in favor of using iter.Seq and iter.Seq2
For reference, Arrow also allows using iter directly: https://pkg.go.dev/github.com/apache/arrow-go/v18/arrow/array#IterFromReader and https://pkg.go.dev/github.com/apache/arrow-go/v18/arrow/array#ReaderFromIter
There was a problem hiding this comment.
Updated to use Seq2, seemed to better match the Iterator semantics we wanted to expose
|
@caldempsey, thanks a lot for your contribution! I will take another look in early next week, but I don't see a lot of contentious pieces. |
|
I looked into the panic I added some socket monitoring with ODBC Connections: 172.17.0.2:42562->44.234.192.45:443
172.17.0.2:42568->44.234.192.45:443
Spark Connect: 172.17.0.2:60452->44.234.192.46:443
172.17.0.2:41742->44.234.192.45:443I THINK the issue is becaauuuuse POSIX signals get delivered to the entire process, not individual sockets. I'm guessing the ODBC driver ( With Spark Connect v40's new streaming and context cancellation support, SIGPIPE happens more often - cancel a context mid-stream and you'll likely trigger it. The ODBC driver's process-wide signal handler intercepts these signals regardless of which library's socket actually broke, while Spark Connect seems to handle the termination gracefully. The 'fix' here is really sweeping the issue under the rug by adding package main
func init() {
signal.Ignore(syscall.SIGPIPE)
}When you call signal.Ignore(syscall.SIGPIPE), the entire process ignores SIGPIPE signals. The kernel simply doesn't deliver them to the Go process at all. Both libraries already handle EPIPE errors properly through return values, so they don't need the signal. A C++ dev I know (who I can't mention) said this is actually the modern way to handle broken pipes - check return values instead of relying on signals. So with this, Spark Connect streaming and ODBC can coexist in the same process without issues. Pretty important for services that need both traditional ODBC connectivity (faster than gRPC) and Spark Connect's DataFrame capabilities (easier to use than ODBC). |
grundprinzip
left a comment
There was a problem hiding this comment.
While working on a side project, I think this change could come in handy. I think the code is in a good spot, but would be good to add an integration test as well to show as well how to use the API.
|
@grundprinzip Thanks Martin, yes I will get to the feedback as soon as I can, this Saturday is my next best available op. I've been vacationing + had a surgery, sorry for the delay! |
51f79d1 to
216f3a3
Compare
216f3a3 to
c7a0c05
Compare
5cf7f46 to
917ce9f
Compare
|
I’ve made the changes we discussed @grundprinzip @zeroshade. Few thoughts:
So please don't crucify me if this isn't exactly right!!!!! |
Will take a look at writing the integration test next step, missed this comment! |
spark/sql/types/rowiterator.go
Outdated
| // Treat io.EOF as clean termination (don’t forward). | ||
| if errors.Is(err, io.EOF) { | ||
| return | ||
| } |
There was a problem hiding this comment.
Since we're also pruning downstream maybe remove this
tests for channel-based processing, filtering, error handling, empty datasets, multiple columns, and large datasets.
|
@grundprinzip @zeroshade Added a Spark test that works fine locally, but looks like the Github action is failing on I'd normally address concerns in separate PRs, but I've bumped the version inline to keep things moving. Also added an integration test demonstrating how this works with unary RPCs. All current feedback should be addressed. Let me know if anything else needs attention 🙂 Separately, I'd love to connect more with the community on this project. If there are any channels where folks discuss features, use cases, or development, I'd like to be part of them. I've been productionizing Spark Connect Go for a while now (a vote of confidence), including my own implementation of this feature with Databricks to synchronize the control loop between Spark requests and infrastructure (previously asynchronous Spark submit jobs). Would be good to share and listen to insights and experiences in the wild and smooth problems out! Thanks ✌️ |
zeroshade
left a comment
There was a problem hiding this comment.
Most of my comments are nitpicks, there's a few things that I think are pretty important but overall this looks pretty good to me!
spark/client/client.go
Outdated
| 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!") | ||
| } | ||
| } |
There was a problem hiding this 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/client/client.go
Outdated
| // 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) { |
There was a problem hiding this comment.
just do return func(yield func(arrow.Record, error) bool) { instead of iterator := .... and return iterator
spark/client/client.go
Outdated
| }() | ||
|
|
||
| // Explicitly needed when tracking re-attachable execution. | ||
| c.done = false |
There was a problem hiding this comment.
wouldn't this be a race condition? Should we be locking around accessing c.done?
spark/sql/types/rowiterator.go
Outdated
| 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 | ||
| } | ||
| } |
There was a problem hiding this comment.
It feels like we could simplify this if ReadArrowRecordToRows performed the release inside the iterator etc.
Something like:
for row := range rowIterFromRecord(rec) {
if !yield(row, nil) {
return
}
}There was a problem hiding this comment.
So i've extracted ReadArrowRecordToRows, and that let us range over it directly as you suggested.
There was an error check that needs to be handled since ReadArrowRecordToRows can still fail, so made that explicit:
for row, err := range rowIterFromRecord(rec) {
if err != nil {
_ = yield(nil, err)
return
}
if !yield(row, nil) {
return
}
}
spark/sql/dataframe.go
Outdated
| seq2 := responseClient.ToRecordSequence(ctx) | ||
|
|
||
| return types.NewRowPull2(ctx, seq2), nil |
There was a problem hiding this comment.
So, as I mentioned briefly, this is my first time using Go iterators. I was originally inspired to build a Pull abstraction from the Go range functions blog post to give callers on-demand 'pull' semantics over the tuple. I used that as my 'north star', so to speak.
But yeah, I think the iter.Pull2 conversion here is overengineering: looking at this a few months down the line this is just creating a push→pull→push, and so a roundtrip that seems completely redundant and less concise. Seems the underlying gRPC stream is inherently single-use, so we don't need to guard termination either.
I'll deal with this by taking the useful part, the EOF handler and bubble it directly into NewRowSequence and drop the iter.Pull2 conversion.
There was a problem hiding this comment.
Done, let me know if this looks like the right idiom to you. Apologies for the greenness.
|
Updated the PR addressing feedback. Removed |
spark/sql/types/rowiterator.go
Outdated
| return | ||
| } | ||
| if rec == nil { | ||
| _ = yield(nil, errors.New("expected arrow.Record to contain non-nil Rows, got nil")) |
There was a problem hiding this comment.
The error here isn't quite correct, should probably be "expected non-nil arrow.Record" or something to that effect. It's not a case of "non-nil Rows" but rather the entire arrow.Record object is nil
spark/sql/types/rowiterator.go
Outdated
| if err != nil { | ||
| _ = yield(nil, err) | ||
| return | ||
| } | ||
| if !yield(row, nil) { | ||
| return | ||
| } |
There was a problem hiding this comment.
Maybe something like this?
| if err != nil { | |
| _ = yield(nil, err) | |
| return | |
| } | |
| if !yield(row, nil) { | |
| return | |
| } | |
| cont := yield(row, err) | |
| if err != nil || !cont { | |
| return | |
| } |
There was a problem hiding this comment.
yeah! i wasn't sure if we wanted the explicit form of this where you make the err != nil flow painfully clear
for row, err := range rowIterFromRecord(rec) {
if !yield(row, err) || err != nil {
return
}
}is another way to hash this. but honestly? i like your version most because it preserves the semantics from a human readability standpoint better.
for row, err := range rowIterFromRecord(rec) {
cont := yield(row, err)
if err != nil || !cont {
return
}
}will update
zeroshade
left a comment
There was a problem hiding this comment.
Just a couple comments, but otherwise looks great to me!
What changes were proposed in this pull request?
This PR adds streaming support to the Spark Connect Go client by introducing
ToLocalIterator(ctx)on the DataFrame API, matching the functionality available in Scala's implementation. The implementation includes a newtypes.RowIteratorinterface that streams Arrow records batch-by-batch, and aToRecordBatches()method onExecutePlanClientthat returns channels for streaming Arrow records and errors. Context cancellation is handled throughout the streaming pipeline to ensure proper resource cleanup. We use EOF as the default implementation to signal we are done processing. This seems to be the default from a Databricks Cluster running on Spark. Seems OK on localhost Spark too.Why are the changes needed?
The Go client currently only supports
Collect(), which loads all rows into memory at once. This approach doesn't work for large OLAP queries that return gigabytes of data, and prevents lightweight services like API gateways or serverless functions from using Spark Connect effectively.This PR brings the same streaming capability that Scala and Java clients have through
toLocalIterator(). Rows are deserialised batch-by-batch and released immediately after consumption. SinceToTablewas pulling entire tables of Arrow batches, we needed a new method to stream batches individually. For now, I've reusedReadArrowTableToRowsfor the batch-to-row conversion, though this could be optimised in a follow-up PR.The changes are purely additive to maintain backward compatibility. External consumers can pull data at their own pace, making it suitable for streaming over HTTP or gRPC. I believe there are many opportunities to improve performance by reducing allocs and GC pressure, which can also be addressed in a follow up.
Does this PR introduce any user-facing change?
Yes, it adds a new optional API:
Existing code using
Collect()orToArrow()remains unchanged.How was this patch tested?
go test ./...Demo:
Showing the effects of a complete stream and context cancellation (against a UC catalog table I get ~15MB/s):
Screen.Recording.2025-07-13.at.16.40.21.mov
From the server: