Skip to content
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

GH-38255: [Go][C++] Implement Flight SQL Bulk Ingestion #38385

Merged
merged 13 commits into from
Apr 17, 2024

Conversation

joellubi
Copy link
Member

@joellubi joellubi commented Oct 22, 2023

Rationale for this change

It was suggested in the discussion around apache/arrow-adbc#1107 for the Flight SQL ADBC driver that an "Ingest" command would be a helpful addition to the Flight SQL specification. This command would enable a Flight SQL client to provide a FlightData stream to the server without needing to know its SQL syntax, and have that stream loaded into a target table by whichever means the server deems appropriate.

What changes are included in this PR?

  • Format:
    • Add CommandStatementIngest message type to Flight SQL proto definition
    • Add FLIGHT_SQL_SERVER_BULK_INGESTION and FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED options for SqlInfo
  • Go:
    • Generate pb
    • Server-side implementation
    • Client-side implementation
    • Unit + integration tests
  • C++:
    • Server-side implementation
    • Client-side implementation
    • Integration tests

Are these changes tested?

Yes, see server_test.go, scenario.go, and test_integration.cc.

Are there any user-facing changes?

Yes, new Flight SQL client and server functionality is being added. Changes are not expected to break existing users.

@kou
Copy link
Member

kou commented Oct 22, 2023

Could you also add a new integration test to

flight_scenarios = [
Scenario(
"auth:basic_proto",
description="Authenticate using the BasicAuth protobuf."),
Scenario(
"middleware",
description="Ensure headers are propagated via middleware.",
),
Scenario(
"ordered",
description="Ensure FlightInfo.ordered is supported.",
skip_testers={"JS", "C#", "Rust"},
),
Scenario(
"expiration_time:do_get",
description=("Ensure FlightEndpoint.expiration_time with "
"DoGet is working as expected."),
skip_testers={"JS", "C#", "Rust"},
),
Scenario(
"expiration_time:list_actions",
description=("Ensure FlightEndpoint.expiration_time related "
"pre-defined actions is working with ListActions "
"as expected."),
skip_testers={"JS", "C#", "Rust"},
),
Scenario(
"expiration_time:cancel_flight_info",
description=("Ensure FlightEndpoint.expiration_time and "
"CancelFlightInfo are working as expected."),
skip_testers={"JS", "C#", "Rust"},
),
Scenario(
"expiration_time:renew_flight_endpoint",
description=("Ensure FlightEndpoint.expiration_time and "
"RenewFlightEndpoint are working as expected."),
skip_testers={"JS", "C#", "Rust"},
),
Scenario(
"poll_flight_info",
description="Ensure PollFlightInfo is supported.",
skip_testers={"JS", "C#", "Rust"}
),
Scenario(
"app_metadata_flight_info_endpoint",
description="Ensure support FlightInfo and Endpoint app_metadata",
skip_testers={"JS", "C#", "Rust", "Java"}
),
Scenario(
"flight_sql",
description="Ensure Flight SQL protocol is working as expected.",
skip_testers={"Rust"}
),
Scenario(
"flight_sql:extension",
description="Ensure Flight SQL extensions work as expected.",
skip_testers={"Rust"}
),
]
to run the integration test in our CI?

@joellubi
Copy link
Member Author

@kou I added my tests under the existing scenario flight_sql, so my understanding was that they are already getting run. However I do see they're failing for implementations besides Go. Do you mean to create a new scenario in which other implementations besides Go are excluded?

Copy link
Member

@lidavidm lidavidm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should create a new integration test scenario so that it can be individually skipped/enabled for each implementation

Comment on lines 820 to 830

/*
* Retrieves a boolean value indicating whether transactions are supported for bulk ingestion. If not, invoking
* the method commit in the context of a bulk ingestion is a noop, and the isolation level is
* `arrow.flight.protocol.sql.SqlTransactionIsolationLevel.TRANSACTION_NONE`.
*
* Returns:
* - false: if bulk ingestion transactions are unsupported;
* - true: if bulk ingestion transactions are supported.
*/
INGEST_TRANSACTIONS_SUPPORTED = 577;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this should go in the 0-500 range

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense. Done.

Comment on lines 1792 to 1796
/*
* Represents a bulk ingestion request. Used in the command member of FlightDescriptor
* for the the RPC call DoPut to cause the server load the contents of the stream's
* FlightData into the target destination.
*/
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: why the extra indent?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed this

if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil {
return
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe you have to drain the read-side of the stream here, even if technically the server should only have sent one message

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated. Let me know what you think of the approach. We don't currently ensure the stream is drained in the other DoPut methods FWIW, but I can make changes there as well if it's preferred.

@github-actions github-actions bot added awaiting changes Awaiting changes and removed awaiting review Awaiting review labels Oct 23, 2023
@kou
Copy link
Member

kou commented Oct 23, 2023

I added my tests under the existing scenario flight_sql, so my understanding was that they are already getting run. However I do see they're failing for implementations besides Go. Do you mean to create a new scenario in which other implementations besides Go are excluded?

Yes. Sorry for not clarifying this.

@github-actions github-actions bot added awaiting change review Awaiting change review and removed awaiting changes Awaiting changes labels Oct 23, 2023
@joellubi
Copy link
Member Author

Yes. Sorry for not clarifying this.

@kou No problem, I created a separate scenario.

Comment on lines 2646 to 2957
func (m *flightSqlIngestionScenarioTester) flightInfoForCommand(desc *flight.FlightDescriptor, schema *arrow.Schema) *flight.FlightInfo {
return &flight.FlightInfo{
Endpoint: []*flight.FlightEndpoint{
{Ticket: &flight.Ticket{Ticket: desc.Cmd}},
},
Schema: flight.SerializeSchema(schema, memory.DefaultAllocator),
FlightDescriptor: desc,
TotalRecords: -1,
TotalBytes: -1,
}
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this isn't actually being used anywhere, that's why the CI is failing

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, just removed.

@github-actions github-actions bot added awaiting changes Awaiting changes and removed awaiting change review Awaiting change review labels Oct 24, 2023
Comment on lines 240 to 284
wr = flight.NewRecordWriter(stream, ipc.WithAllocator(c.Alloc), ipc.WithSchema(rdr.Schema()))
wr.SetFlightDescriptor(desc)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

don't we need defer wr.Close() ?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch, forgot to do so. Just added.

@github-actions github-actions bot added awaiting change review Awaiting change review and removed awaiting changes Awaiting changes labels Oct 24, 2023
Copy link
Member

@zeroshade zeroshade left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This LGTM generally, though we need a second implementation before it can be voted on in the Mailing List to officially add this to the spec.

@github-actions github-actions bot added awaiting changes Awaiting changes and removed awaiting change review Awaiting change review labels Oct 25, 2023
@joellubi
Copy link
Member Author

This LGTM generally, though we need a second implementation before it can be voted on in the Mailing List to officially add this to the spec.

Thanks @zeroshade. I'm planning to get another implementation out in the next week or so.

@joellubi joellubi force-pushed the add-flight-sql-ingestion-go-impl branch from 434d537 to db70c20 Compare October 31, 2023 10:14
@github-actions github-actions bot added the awaiting change review Awaiting change review label Apr 9, 2024
@lidavidm
Copy link
Member

lidavidm commented Apr 9, 2024

@zeroshade or others, any last comments?

}

// Options to assert before/after mocked ingest call
func getIngestOptions() flightsql.ExecuteIngestOpts {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shouldn't this return a pointer to avoid the error about copying the mutex?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated

Comment on lines 259 to 260
// Servers cannot infer defaults for these parameters, so we validate the request to ensure they are set.
if reqOptions.TableDefinitionOptions == nil {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we validate that reqOptions is not nil first to avoid a potential panic?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point. Since we cast to the protobuf command type anyway, I did so a bit earlier to let it take care of nil handling for us. I added a nil test for completeness too.

@github-actions github-actions bot added awaiting changes Awaiting changes and removed awaiting change review Awaiting change review labels Apr 11, 2024
Comment on lines 267 to 268
rdr.Retain()
defer rdr.Release()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

since you aren't passing the reader to anything outside of this function, I don't think this is necessary at all

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch, updated

Comment on lines 882 to 887
type CreatePreparedStatementResult = pb.ActionCreatePreparedStatementResult

type TableDefinitionOptions = pb.CommandStatementIngest_TableDefinitionOptions

type TableDefinitionOptionsTableNotExistOption = pb.CommandStatementIngest_TableDefinitionOptions_TableNotExistOption
type TableDefinitionOptionsTableExistsOption = pb.CommandStatementIngest_TableDefinitionOptions_TableExistsOption
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

for clarity should we gather these into a single declaration block?

Suggested change
type CreatePreparedStatementResult = pb.ActionCreatePreparedStatementResult
type TableDefinitionOptions = pb.CommandStatementIngest_TableDefinitionOptions
type TableDefinitionOptionsTableNotExistOption = pb.CommandStatementIngest_TableDefinitionOptions_TableNotExistOption
type TableDefinitionOptionsTableExistsOption = pb.CommandStatementIngest_TableDefinitionOptions_TableExistsOption
type (
CreatePreparedStatementResult = pb.ActionCreatePreparedStatementResult
TableDefinitionOptions = pb.CommandStatementIngest_TableDefinitionOptions
TableDefinitionOptionsTableNotExistOption = pb.CommandStatementIngest_TableDefinitionOptions_TableNotExistOption
TableDefinitionOptionsTableExistsOption = pb.CommandStatementIngest_TableDefinitionOptions_TableExistsOption
)

?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I gathered the ones related to TableDefinitionOptions.

The CreatePreparedStatementResult type is unrelated.

@zeroshade
Copy link
Member

Outside of my nitpicks, this all LGTM

@github-actions github-actions bot added awaiting change review Awaiting change review and removed awaiting changes Awaiting changes labels Apr 12, 2024
@joellubi joellubi requested a review from zeroshade April 12, 2024 17:53
@joellubi
Copy link
Member Author

@zeroshade @lidavidm

I addressed the remaining comments. There are 2 failing workflows but they appear unrelated. If this is fine, would one of you please merge this or let me know if I should do so?

@zeroshade
Copy link
Member

You're right, the failures are unrelated to your changes. This looks good to me. If no one else objects I'll merge this later this evening

@joellubi
Copy link
Member Author

@zeroshade It looks like you're good with this merging and there have been no objections. I'll merge this shortly.

@zeroshade zeroshade merged commit 0d1ea5d into apache:main Apr 17, 2024
56 of 58 checks passed
@zeroshade zeroshade removed the awaiting change review Awaiting change review label Apr 17, 2024
Copy link

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

There were no benchmark performance regressions. 🎉

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

tolleybot pushed a commit to tmct/arrow that referenced this pull request May 2, 2024
…#38385)

### Rationale for this change

It was suggested in the discussion around apache/arrow-adbc#1107 for the Flight SQL ADBC driver that an "Ingest" command would be a helpful addition to the Flight SQL specification. This command would enable a Flight SQL client to provide a FlightData stream to the server without needing to know its SQL syntax, and have that stream loaded into a target table by whichever means the server deems appropriate.

### What changes are included in this PR?

- Format:
  - Add CommandStatementIngest message type to Flight SQL proto definition
  - Add FLIGHT_SQL_SERVER_BULK_INGESTION and FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED options for SqlInfo
- Go:
  - Generate pb
  - Server-side implementation
  - Client-side implementation
  - Unit + integration tests
- C++:
  - Server-side implementation
  - Client-side implementation
  - Integration tests

### Are these changes tested?

Yes, see `server_test.go`, `scenario.go`, and `test_integration.cc`.

### Are there any user-facing changes?

Yes, new Flight SQL client and server functionality is being added. Changes are not expected to break existing users.

* Closes: apache#38255

Lead-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>
Signed-off-by: Matt Topol <[email protected]>
tolleybot pushed a commit to tmct/arrow that referenced this pull request May 4, 2024
…#38385)

### Rationale for this change

It was suggested in the discussion around apache/arrow-adbc#1107 for the Flight SQL ADBC driver that an "Ingest" command would be a helpful addition to the Flight SQL specification. This command would enable a Flight SQL client to provide a FlightData stream to the server without needing to know its SQL syntax, and have that stream loaded into a target table by whichever means the server deems appropriate.

### What changes are included in this PR?

- Format:
  - Add CommandStatementIngest message type to Flight SQL proto definition
  - Add FLIGHT_SQL_SERVER_BULK_INGESTION and FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED options for SqlInfo
- Go:
  - Generate pb
  - Server-side implementation
  - Client-side implementation
  - Unit + integration tests
- C++:
  - Server-side implementation
  - Client-side implementation
  - Integration tests

### Are these changes tested?

Yes, see `server_test.go`, `scenario.go`, and `test_integration.cc`.

### Are there any user-facing changes?

Yes, new Flight SQL client and server functionality is being added. Changes are not expected to break existing users.

* Closes: apache#38255

Lead-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>
Signed-off-by: Matt Topol <[email protected]>
rok pushed a commit to tmct/arrow that referenced this pull request May 8, 2024
…#38385)

### Rationale for this change

It was suggested in the discussion around apache/arrow-adbc#1107 for the Flight SQL ADBC driver that an "Ingest" command would be a helpful addition to the Flight SQL specification. This command would enable a Flight SQL client to provide a FlightData stream to the server without needing to know its SQL syntax, and have that stream loaded into a target table by whichever means the server deems appropriate.

### What changes are included in this PR?

- Format:
  - Add CommandStatementIngest message type to Flight SQL proto definition
  - Add FLIGHT_SQL_SERVER_BULK_INGESTION and FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED options for SqlInfo
- Go:
  - Generate pb
  - Server-side implementation
  - Client-side implementation
  - Unit + integration tests
- C++:
  - Server-side implementation
  - Client-side implementation
  - Integration tests

### Are these changes tested?

Yes, see `server_test.go`, `scenario.go`, and `test_integration.cc`.

### Are there any user-facing changes?

Yes, new Flight SQL client and server functionality is being added. Changes are not expected to break existing users.

* Closes: apache#38255

Lead-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>
Signed-off-by: Matt Topol <[email protected]>
rok pushed a commit to tmct/arrow that referenced this pull request May 8, 2024
…#38385)

### Rationale for this change

It was suggested in the discussion around apache/arrow-adbc#1107 for the Flight SQL ADBC driver that an "Ingest" command would be a helpful addition to the Flight SQL specification. This command would enable a Flight SQL client to provide a FlightData stream to the server without needing to know its SQL syntax, and have that stream loaded into a target table by whichever means the server deems appropriate.

### What changes are included in this PR?

- Format:
  - Add CommandStatementIngest message type to Flight SQL proto definition
  - Add FLIGHT_SQL_SERVER_BULK_INGESTION and FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED options for SqlInfo
- Go:
  - Generate pb
  - Server-side implementation
  - Client-side implementation
  - Unit + integration tests
- C++:
  - Server-side implementation
  - Client-side implementation
  - Integration tests

### Are these changes tested?

Yes, see `server_test.go`, `scenario.go`, and `test_integration.cc`.

### Are there any user-facing changes?

Yes, new Flight SQL client and server functionality is being added. Changes are not expected to break existing users.

* Closes: apache#38255

Lead-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>
Signed-off-by: Matt Topol <[email protected]>
vibhatha pushed a commit to vibhatha/arrow that referenced this pull request May 25, 2024
…#38385)

### Rationale for this change

It was suggested in the discussion around apache/arrow-adbc#1107 for the Flight SQL ADBC driver that an "Ingest" command would be a helpful addition to the Flight SQL specification. This command would enable a Flight SQL client to provide a FlightData stream to the server without needing to know its SQL syntax, and have that stream loaded into a target table by whichever means the server deems appropriate.

### What changes are included in this PR?

- Format:
  - Add CommandStatementIngest message type to Flight SQL proto definition
  - Add FLIGHT_SQL_SERVER_BULK_INGESTION and FLIGHT_SQL_SERVER_INGEST_TRANSACTIONS_SUPPORTED options for SqlInfo
- Go:
  - Generate pb
  - Server-side implementation
  - Client-side implementation
  - Unit + integration tests
- C++:
  - Server-side implementation
  - Client-side implementation
  - Integration tests

### Are these changes tested?

Yes, see `server_test.go`, `scenario.go`, and `test_integration.cc`.

### Are there any user-facing changes?

Yes, new Flight SQL client and server functionality is being added. Changes are not expected to break existing users.

* Closes: apache#38255

Lead-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Joel Lubinitsky <[email protected]>
Co-authored-by: Antoine Pitrou <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>
Signed-off-by: Matt Topol <[email protected]>
lidavidm pushed a commit that referenced this pull request Sep 5, 2024
Please look at #38255 for details on this functionality. Support for Go and C++ was added as part of #38385.
This pull request is to add the required support for Java.
* GitHub Issue: #38255

Lead-authored-by: Amit Mittal <[email protected]>
Co-authored-by: Amit Mittal <[email protected]>
Signed-off-by: David Li <[email protected]>
zanmato1984 pushed a commit to zanmato1984/arrow that referenced this pull request Sep 6, 2024
)

Please look at apache#38255 for details on this functionality. Support for Go and C++ was added as part of apache#38385.
This pull request is to add the required support for Java.
* GitHub Issue: apache#38255

Lead-authored-by: Amit Mittal <[email protected]>
Co-authored-by: Amit Mittal <[email protected]>
Signed-off-by: David Li <[email protected]>
khwilson pushed a commit to khwilson/arrow that referenced this pull request Sep 14, 2024
)

Please look at apache#38255 for details on this functionality. Support for Go and C++ was added as part of apache#38385.
This pull request is to add the required support for Java.
* GitHub Issue: apache#38255

Lead-authored-by: Amit Mittal <[email protected]>
Co-authored-by: Amit Mittal <[email protected]>
Signed-off-by: David Li <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Format][Flight SQL] Add Flight SQL Command for Bulk Ingestion
7 participants