arrow icon indicating copy to clipboard operation
arrow copied to clipboard

GH-37720: [Go][FlightSQL] Implement stateless prepared statements

Open erratic-pattern opened this issue 1 year ago • 5 comments

Rationale for this change

See discussion on https://github.com/apache/arrow/issues/37720 and mailing list: https://lists.apache.org/thread/3kb82ypx99q96g84qv555l6x8r0bppyq

What changes are included in this PR?

Changes the Go FlightSQL client and server implementations to support returning an updated prepared statement handle to the client as part of the DoPut(PreparedStatement) RPC call.

Are these changes tested?

Are there any user-facing changes?

See parent issue and docs PR #40243 for details of user facing changes.

This PR includes breaking changes to public APIs.

  • GitHub Issue: #37720

erratic-pattern avatar Mar 01 '24 19:03 erratic-pattern

Thanks for opening a pull request!

If this is not a minor PR. Could you open an issue for this pull request on GitHub? https://github.com/apache/arrow/issues/new/choose

Opening GitHub issues ahead of time contributes to the Openness of the Apache Arrow project.

Then could you also rename the pull request title in the following format?

GH-${GITHUB_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}

or

MINOR: [${COMPONENT}] ${SUMMARY}

In the case of PARQUET issues on JIRA the title also supports:

PARQUET-${JIRA_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}

See also:

github-actions[bot] avatar Mar 01 '24 19:03 github-actions[bot]

I mentioned here in the Rust PR that I would like to expand the Go tests to cover both the stateful and stateless cases, but I didn't spend much time learning how the Go test suite is structured, and I am less familiar with Go as a language in general.

I will take a look at it this week, but would appreciate recommendations of where to look from anyone.

erratic-pattern avatar Mar 04 '24 15:03 erratic-pattern

@erratic-pattern thanks for doing this. As far as how the tests are structured, there's three primary files specific to flightsql:

  • client_test.go: contains a mocked GrpcClientStream object and a mock FlightServiceClient object to isolate the FlightSQL logic. Everything is part of the FlightSqlClientSuite test object, so adding a new test is just adding a new method named using the pattern TestXXXXX() and it will automatically get picked up. You can look at SetupTest and TearDownTest for the setup and teardown which currently just consist of setting up the mocks and asserting the expectations happen.
  • server_test.go: You can look at FlightSqlServerSuite and its methods for the basic test suite that starts up a flight server using the testServer object in that file. Adding proper tests there would require simply adding any needed functionality to the testServer methods and corresponding test methods in the FlightSqlServerSuite and the FlightSqlServerSessionSuite if its relevant to test the Session based interactions too.
  • sqlite_server_test.go is a test suite which just tests the example SQLite FlightSQL server. You could add the stateless prepared statement support to the sqlite server in the example subdir, and then test it in this file.

Hope that helps! If you want me to look closely at this while it's still a draft let me know, otherwise I'll wait until you mark it as ready for review

zeroshade avatar Mar 06 '24 21:03 zeroshade

@zeroshade A review would be much appreciated. The PR is in draft state until the spec is voted on, but the code change is more or less "Done"

erratic-pattern avatar Mar 10 '24 17:03 erratic-pattern

@erratic-pattern in general this looks okay, but the build needs to be fixed as I'm seeing

# github.com/apache/arrow/go/v16/arrow/flight/flightsql
Error: flight\flightsql\server.go:979:16: undefined: flight.DoPutPreparedStatementResult

In the CI

zeroshade avatar Mar 11 '24 14:03 zeroshade

I must have forgotten to commit the protobuf generated code in this branch. I've run go generate locally on the format PR and committed the changes to this PR.

erratic-pattern avatar Mar 20 '24 01:03 erratic-pattern

@zeroshade Not sure how to interpret this CI failure. It seems to involve a Java client integration test. Can you take a look? https://github.com/apache/arrow/actions/runs/8352550401/job/22934920078?pr=40311

erratic-pattern avatar Mar 22 '24 16:03 erratic-pattern

The proposal for this specification change was approved on the Arrow dev mailing list. See thread here I've moved this PR out of draft status.

erratic-pattern avatar Mar 24 '24 17:03 erratic-pattern

@lidavidm @danepitkin Would either of you be able to look into the Java side here and confirm if the issue is on the Java client or if the Go side is doing something wrong?

The Java flight client appears to be reporting a memory leak in two scenarios.

zeroshade avatar Mar 25 '24 15:03 zeroshade

Shouldn't there be updates to client.go here?

lidavidm avatar Mar 25 '24 15:03 lidavidm

Probably, since we'd want to expose the new functionality for that. But that wouldn't explain the java failure though

zeroshade avatar Mar 25 '24 15:03 zeroshade

Right I was commenting before the page refreshed

lidavidm avatar Mar 25 '24 15:03 lidavidm

Most likely, you should split this into its own set of tests because it looks like the test is being changed in an incompatible way.

lidavidm avatar Mar 25 '24 15:03 lidavidm

oh wait hmm, we're not changing the handle yet?

lidavidm avatar Mar 25 '24 15:03 lidavidm

Shouldn't there be updates to client.go here?

Looks like I forgot to add this. I'll take a look.

erratic-pattern avatar Mar 25 '24 15:03 erratic-pattern

I see that writeBindParameters is where the client needs to be updated, but it's not clear to me how I should change it to read a single message from the stream. I don't see a NewRecordReader similar to the NewRecordWriter. Should I call pstream.Recv? That would appear to give us a PutResult which I could then unwrap to get the encoded app_metadata and decode to DoPutPreparedStatementResult.

erratic-pattern avatar Mar 26 '24 18:03 erratic-pattern

@erratic-pattern @lidavidm just checking on the status of this - besides @erratic-pattern's last question is anything else needed to get this merged?

matthewmturner avatar Apr 10 '24 22:04 matthewmturner

I see that writeBindParameters is where the client needs to be updated, but it's not clear to me how I should change it to read a single message from the stream. I don't see a NewRecordReader similar to the NewRecordWriter. Should I call pstream.Recv? That would appear to give us a PutResult which I could then unwrap to get the encoded app_metadata and decode to DoPutPreparedStatementResult.

Yes, I think pstream.Recv is the correct thing to do, though we have to handle EOF gracefully as well

lidavidm avatar Apr 10 '24 22:04 lidavidm

I've updated this PR with the client-side implementation, which waits for a PutResult and captures the stored prepared statement handle for subsequent requests.

There are some integration test failures related to this change that I am looking at right now, though it might take me some time to figure out if anyone wants to take a second look.

erratic-pattern avatar Apr 13 '24 21:04 erratic-pattern

Okay, tests have been updated and are passing locally. The new test failure in CI is a bit of a mystery.

https://github.com/apache/arrow/actions/runs/8676995733/job/23792073344?pr=40311#step:6:68

--- FAIL: TestSqliteBackend (19.24s)
    --- FAIL: TestSqliteBackend/TestRowsPrematureCloseDuringNextLoop (1.18s)
        driver_test.go:649: 
            	Error Trace:	/arrow/go/arrow/flight/flightsql/driver/driver_test.go:649
            	Error:      	Not equal: 
            	            	expected: 10
            	            	actual  : 0
            	Test:       	TestSqliteBackend/TestRowsPrematureCloseDuringNextLoop
FAIL

I can't see why this would be related to the changes here.

erratic-pattern avatar Apr 14 '24 00:04 erratic-pattern

@erratic-pattern that failure isn't related to this change, I've seen it a few times but haven't yet been able to figure out the cause. It appears to be a race condition of some sort but I'm unsure of the cause.

zeroshade avatar Apr 16 '24 14:04 zeroshade

@zeroshade just to confirm, I assume were blocked on merging this until that unrelated change is resolved?

matthewmturner avatar Apr 17 '24 14:04 matthewmturner

@matthewmturner We don't need to wait for that unrelated issue to be addressed. Once the merge conflict is addressed, i'll merge this

zeroshade avatar Apr 17 '24 16:04 zeroshade

@zeroshade awesome thanks!

@erratic-pattern are you able to take care of that conflict or would you like some help?

matthewmturner avatar Apr 17 '24 16:04 matthewmturner

@zeroshade @erratic-pattern i can give this another couple days but after that might make a separate PR with these changes copied and then ill see if i can resolve conflicts (not familiar with this repo though and havent looked at the conflicts yet so unsure).

the reason im pressing is that having this feature is a blocker for us and we would like to avoid having to fork.

matthewmturner avatar Apr 22 '24 16:04 matthewmturner

Done. I resolved the conflict by re-running go generate again @matthewmturner @zeroshade Let me know if that worked.

erratic-pattern avatar Apr 23 '24 01:04 erratic-pattern

@erratic-pattern awesome thanks!

matthewmturner avatar Apr 23 '24 01:04 matthewmturner

After merging your PR, Conbench analyzed the 5 benchmarking runs that have been run so far on merge-commit 2cf844d4386e1ccad8592b421c18dfe9a82d2e42.

There were no benchmark performance regressions. 🎉

The full Conbench report has more details. It also includes information about 3 possible false positives for unstable benchmarks that are known to sometimes produce them.

This doesn't seem to actually work @erratic-pattern @zeroshade

https://github.com/erratic-pattern/arrow/blob/af0765eebfc67579b369e80b5c3d600b580cfcf8/go/arrow/flight/flightsql/client.go#L1110-L1141

We capture the new value, but we continue to use the old desc in the final call to getFlightInfo

lidavidm avatar Apr 29 '24 06:04 lidavidm

https://github.com/apache/arrow/issues/41427

lidavidm avatar Apr 29 '24 06:04 lidavidm