From b5cd8e0b3673b16b5c02d78e45cb48fcc49331e5 Mon Sep 17 00:00:00 2001 From: Ryan Nicholson Date: Fri, 21 Aug 2020 17:32:46 -0700 Subject: [PATCH 01/16] [FlightRPC] Flight SQL POC MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add extensions in the Apache Arrow project’s Arrow Flight modules to provide a standard way for clients and servers to communicate with SQL-like semantics. Do not pull to master. A message to the mailing list will accompany this and another proposal in the coming days for discussion. --- format/FlightSQL.proto | 226 +++++++ .../arrow/flight/FlightRuntimeException.java | 2 +- java/flight/flight-sql/pom.xml | 178 ++++++ .../flight/sql/FlightSQLClientUtils.java | 219 +++++++ .../arrow/flight/sql/FlightSQLProducer.java | 339 ++++++++++ .../arrow/flight/sql/FlightSQLUtils.java | 203 ++++++ .../apache/arrow/flight/TestFlightSQL.java | 262 ++++++++ .../arrow/flight/sql/FlightSQLExample.java | 601 ++++++++++++++++++ .../flight/sql/PreparedStatementCacheKey.java | 83 +++ .../flight/sql/PreparedStatementContext.java | 65 ++ .../src/test/protobuf/flightSQLExample.proto | 26 + java/pom.xml | 1 + 12 files changed, 2204 insertions(+), 1 deletion(-) create mode 100644 format/FlightSQL.proto create mode 100644 java/flight/flight-sql/pom.xml create mode 100644 java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLClientUtils.java create mode 100644 java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java create mode 100644 java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java create mode 100644 java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java create mode 100644 java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java create mode 100644 java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementCacheKey.java create mode 100644 java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementContext.java create mode 100644 java/flight/flight-sql/src/test/protobuf/flightSQLExample.proto diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto new file mode 100644 index 00000000000..2ef7299becb --- /dev/null +++ b/format/FlightSQL.proto @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +syntax = "proto3"; + +option java_package = "org.apache.arrow.flight.sql.impl"; +package arrow.flight.protocol.sql; + +/* + * Wrap the result of a "GetSQLCapabilities" action. + */ +message ActionGetSQLCapabilitiesResult{ + string identifierQuoteString = 1; + bool supportsExpressionsInOrderBy = 2; + // TODO add more capabilities. +} + +/* + * Request message for the "GetCatalogs" action on a + * Flight SQL enabled backend. + * Requests a list of catalogs available in the server. + */ +message ActionGetCatalogsRequest { + /* + * True will ensure results are ordered alphabetically. + * False will not enforce ordering. + */ + bool orderResultsAlphabetically = 1; +} + +/* + * Wrap the result of a "GetCatalogs" action. + */ +message ActionGetCatalogsResult { + repeated string catalogNames = 1; +} + +/* + * Request message for the "GetSchemas" action on a + * Flight SQL enabled backend. + * Requests a list of schemas available in the server. + */ +message ActionGetSchemasRequest { + /* + * True will ensure results are ordered alphabetically. + * False will not enforce ordering. + */ + bool orderResultsAlphabetically = 1; + + /* + * Specifies the Catalog to search for schemas. + */ + string catalog = 2; + + // Specifies a filter pattern for schemas to search for. + string schemaFilterPattern = 3; +} + +/* + * Wrap the result of a "GetSchemas" action. + */ +message ActionGetSchemasResult { + string catalog = 1; + string schema = 2; +} + +/* + * Request message for the "GetTables" action on a + * Flight SQL enabled backend. + * Requests a list of tables available in the server. + */ +message ActionGetTablesRequest { + /* + * True will ensure results are ordered alphabetically. + * False will not enforce ordering. + */ + bool orderResultsAlphabetically = 1; + + // Specifies the Catalog to search for schemas. + string catalog = 2; + + // Specifies a filter pattern for schemas to search for. + string schemaFilterPattern = 3; + + // Specifies a filter pattern for tables to search for. + string tableNameFilterPattern = 4; + + // Specifies a filter of table types which must match. + repeated string tableTypes = 5; + + // Specifies if the schema should be returned for found tables. + bool includeSchema = 6; +} + +/* + * Wrap the result of a "GetTables" action. + */ +message ActionGetTablesResult { + string catalog = 1; + string schema = 2; + string table = 3; + string tableType = 4; + + /* + * Schema of the dataset as described in Schema.fbs::Schema, + * Null if includeSchema on request is false. + */ + bytes schemaMetadata = 5; +} + +/* + * Wrap the result of a "GetTableTypes" action. + */ +message ActionGetTableTypesResult { + string tableType = 1; +} + +// SQL Execution Action Messages + +/* + * Request message for the "GetPreparedStatement" action on a + * Flight SQL enabled backend. + * Requests a list of tables available in the server. + */ +message ActionGetPreparedStatementRequest { + // The SQL syntax. + string query = 1; +} + +/* + * Wrap the result of a "GetPreparedStatement" action. + */ +message ActionGetPreparedStatementResult { + + // Opaque handle for the prepared statement on the server. + bytes preparedStatementHandle = 1; + + // schema of the dataset as described in Schema.fbs::Schema. + bytes datasetSchema = 2; + + // schema of the expected parameters, if any existed, as described in Schema.fbs::Schema. + bytes parameterSchema = 3; +} + +/* + * Request message for the "ClosePreparedStatement" action on a + * Flight SQL enabled backend. + * Closes server resources associated with the prepared statement handle. + */ +message ActionClosePreparedStatementRequest { + // Opaque handle for the prepared statement on the server. + string preparedStatementHandle = 1; +} + + +// SQL Execution Messages. + +/* + * Represents a SQL query. Used in the command member of FlightDescriptor + * for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the query. + */ +message CommandStatementQuery { + // The SQL syntax. + string query = 2; +} + +/* + * Represents an instance of executing a prepared statement. Used in the + * command member of FlightDescriptor for the following RPC calls: + * - DoPut: bind parameter values. + * - GetFlightInfo: execute the prepared statement instance. + */ +message CommandPreparedStatementQuery { + // Unique identifier for the instance of the prepared statement to execute. + bytes clientExecutionHandle = 2; + // Opaque handle for the prepared statement on the server. + bytes preparedStatementHandle = 3; +} + +/* + * Represents a SQL update query. Used in the command member of FlightDescriptor + * for the the RPC call DoPut to cause the server to execute the included + * SQL update. + */ +message CommandStatementUpdate { + // The SQL syntax. + string query = 2; +} + +/* + * Represents a SQL update query. Used in the command member of FlightDescriptor + * for the the RPC call DoPut to cause the server to execute the included + * prepared statement handle as an update. + */ +message CommandPreparedStatementUpdate { + // Unique identifier for the instance of the prepared statement to execute. + bytes clientExecutionHandle = 2; + // Opaque handle for the prepared statement on the server. + bytes preparedStatementHandle = 3; +} + +/* + * Returned from the RPC call DoPut when a CommandStatementUpdate + * CommandPreparedStatementUpdate was in the request, containing + * results from the update. + */ +message DoPutUpdateResult { + int64 recordCount = 1; +} diff --git a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightRuntimeException.java b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightRuntimeException.java index 76d3349a2c3..3abcce7b163 100644 --- a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightRuntimeException.java +++ b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightRuntimeException.java @@ -29,7 +29,7 @@ public class FlightRuntimeException extends RuntimeException { /** * Create a new exception from the given status. */ - FlightRuntimeException(CallStatus status) { + public FlightRuntimeException(CallStatus status) { super(status.description(), status.cause()); this.status = status; } diff --git a/java/flight/flight-sql/pom.xml b/java/flight/flight-sql/pom.xml new file mode 100644 index 00000000000..f7f5fda9096 --- /dev/null +++ b/java/flight/flight-sql/pom.xml @@ -0,0 +1,178 @@ + + + + + arrow-java-root + org.apache.arrow + 2.0.0-SNAPSHOT + ../../pom.xml + + 4.0.0 + + flight-sql + Arrow Flight SQL + (Experimental)Contains utility classes to expose Flight SQL semantics for clients and servers over Arrow Flight + jar + + + 1.30.2 + 3.7.1 + 1 + + + + + org.apache.arrow + flight-core + ${project.version} + + + io.netty + netty-transport-native-unix-common + + + io.netty + netty-transport-native-kqueue + + + io.netty + netty-transport-native-epoll + + + + + org.apache.arrow + arrow-memory-core + ${project.version} + + + org.apache.arrow + arrow-memory-netty + ${project.version} + runtime + + + io.grpc + grpc-protobuf + ${dep.grpc.version} + + + com.google.guava + guava + + + io.grpc + grpc-stub + ${dep.grpc.version} + + + com.google.protobuf + protobuf-java + ${dep.protobuf.version} + + + io.grpc + grpc-api + ${dep.grpc.version} + + + org.apache.arrow + flight-core + ${project.version} + tests + test + + + org.apache.arrow + arrow-vector + ${project.version} + ${arrow.vector.classifier} + + + org.slf4j + slf4j-api + + + org.apache.derby + derby + 10.14.2.0 + test + + + org.apache.commons + commons-dbcp2 + 2.7.0 + test + + + commons-logging + commons-logging + + + + + org.apache.commons + commons-pool2 + 2.8.1 + test + + + + + + + + kr.motd.maven + os-maven-plugin + 1.5.0.Final + + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.5.0 + + com.google.protobuf:protoc:${dep.protobuf.version}:exe:${os.detected.classifier} + false + grpc-java + io.grpc:protoc-gen-grpc-java:${dep.grpc.version}:exe:${os.detected.classifier} + + + + src + + ${basedir}/../../../format/ + ${project.build.directory}/generated-sources/protobuf + + + compile + compile-custom + + + + test + + ${basedir}/src/test/protobuf + ${project.build.directory}/generated-test-sources//protobuf + + + compile + compile-custom + + + + + + + + diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLClientUtils.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLClientUtils.java new file mode 100644 index 00000000000..3a462e106c2 --- /dev/null +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLClientUtils.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.sql; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.arrow.flight.Action; +import org.apache.arrow.flight.FlightClient; +import org.apache.arrow.flight.FlightDescriptor; +import org.apache.arrow.flight.FlightInfo; +import org.apache.arrow.flight.Result; +import org.apache.arrow.flight.sql.impl.FlightSQL; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesRequest; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementQuery; +import org.apache.arrow.vector.types.pojo.Schema; + +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; + +import io.grpc.Status; + +/** + * Client side utilities to work with Flight SQL semantics. + */ +public final class FlightSQLClientUtils { + + /** + * Helper method to request a list of tables from a Flight SQL enabled endpoint. + * + * @param client The Flight Client. + * @param catalog The catalog. + * @param schemaFilterPattern The schema filter pattern. + * @param tableFilterPattern The table filter pattern. + * @param tableTypes The table types to include. + * @param includeSchema True to include the schema upon return, false to not include the schema. + * @return A list of tables matching the criteria. + */ + public static List getTables(FlightClient client, String catalog, String schemaFilterPattern, + String tableFilterPattern, List tableTypes, boolean includeSchema) { + + final ActionGetTablesRequest.Builder requestBuilder = ActionGetTablesRequest + .newBuilder() + .setIncludeSchema(includeSchema); + + if (catalog != null) { + requestBuilder.setCatalog(catalog); + } + + if (schemaFilterPattern != null) { + requestBuilder.setSchemaFilterPattern(schemaFilterPattern); + } + + if (tableFilterPattern != null) { + requestBuilder.setTableNameFilterPattern(tableFilterPattern); + } + + if (tableTypes != null) { + requestBuilder.addAllTableTypes(tableTypes); + } + + final Iterator results = client.doAction(new Action( + "GetTables", Any.pack(requestBuilder.build()).toByteArray())); + + final List getTablesResults = new ArrayList<>(); + results.forEachRemaining(result -> { + ActionGetTablesResult actual = FlightSQLUtils.unpackAndParseOrThrow(result.getBody(), + ActionGetTablesResult.class); + getTablesResults.add(actual); + }); + + return getTablesResults; + } + + /** + * Helper method to create a prepared statement on the server. + * + * @param client The Flight Client. + * @param query The query to prepare. + * @return Metadata and handles to the prepared statement which exists on the server. + */ + public static FlightSQLPreparedStatement getPreparedStatement(FlightClient client, String query) { + return new FlightSQLPreparedStatement(client, query); + } + + /** + * Helper class to encapsulate Flight SQL prepared statement logic. + */ + public static class FlightSQLPreparedStatement implements Closeable { + private final FlightClient client; + private final ActionGetPreparedStatementResult preparedStatementResult; + private long invocationCount; + private boolean isClosed; + private Schema resultSetSchema = null; + private Schema parameterSchema = null; + + /** + * Constructor. + * + * @param client The client. FlightSQLPreparedStatement does not maintain this resource. + * @param sql The query. + */ + public FlightSQLPreparedStatement(FlightClient client, String sql) { + this.client = client; + + final Iterator preparedStatementResults = client.doAction(new Action("GetPreparedStatement", + Any.pack(FlightSQL.ActionGetPreparedStatementRequest + .newBuilder() + .setQuery(sql) + .build()) + .toByteArray())); + + preparedStatementResult = FlightSQLUtils.unpackAndParseOrThrow( + preparedStatementResults.next().getBody(), + ActionGetPreparedStatementResult.class); + + invocationCount = 0; + isClosed = false; + } + + /** + * Returns the Schema of the resultset. + * + * @return the Schema of the resultset. + */ + public Schema getResultSetSchema() { + if (resultSetSchema == null && preparedStatementResult.getDatasetSchema() != null) { + resultSetSchema = Schema.deserialize(preparedStatementResult.getDatasetSchema().asReadOnlyByteBuffer()); + } + return resultSetSchema; + } + + /** + * Returns the Schema of the parameters. + * + * @return the Schema of the parameters. + */ + public Schema getParameterSchema() { + if (parameterSchema == null && preparedStatementResult.getParameterSchema() != null) { + parameterSchema = Schema.deserialize(preparedStatementResult.getParameterSchema().asReadOnlyByteBuffer()); + } + return parameterSchema; + } + + /** + * Executes the prepared statement query on the server. + * + * @return a FlightInfo object representing the stream(s) to fetch. + * @throws IOException if the PreparedStatement is closed. + */ + public FlightInfo executeQuery() throws IOException { + if (isClosed) { + throw new IOException("Prepared statement has already been closed on the server."); + } + + final FlightDescriptor descriptor = FlightDescriptor + .command(Any.pack(CommandPreparedStatementQuery.newBuilder() + .setClientExecutionHandle( + ByteString.copyFrom(ByteBuffer.allocate(Long.BYTES).putLong(invocationCount++))) + .setPreparedStatementHandle(preparedStatementResult.getPreparedStatementHandle()) + .build()) + .toByteArray()); + + return client.getInfo(descriptor); + } + + /** + * Executes the prepared statement update on the server. + * + * @return the number of rows updated. + */ + public long executeUpdate() { + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public void close() { + isClosed = true; + final Iterator closePreparedStatementResults = client.doAction(new Action("ClosePreparedStatement", + Any.pack(FlightSQL.ActionClosePreparedStatementRequest + .newBuilder() + .setPreparedStatementHandleBytes(preparedStatementResult.getPreparedStatementHandle()) + .build()) + .toByteArray())); + closePreparedStatementResults.forEachRemaining(result -> { + }); + } + + /** + * Returns if the prepared statement is already closed. + * + * @return true if the prepared statement is already closed. + */ + public boolean isClosed() { + return isClosed; + } + } +} diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java new file mode 100644 index 00000000000..5effd82893a --- /dev/null +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java @@ -0,0 +1,339 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.sql; + +import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_ACTIONS; +import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_CLOSEPREPAREDSTATEMENT; +import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETCATALOGS; +import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETPREPAREDSTATEMENT; +import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETSCHEMAS; +import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETSQLCAPABILITIES; +import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETTABLES; +import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETTABLETYPES; + +import org.apache.arrow.flight.Action; +import org.apache.arrow.flight.ActionType; +import org.apache.arrow.flight.FlightDescriptor; +import org.apache.arrow.flight.FlightInfo; +import org.apache.arrow.flight.FlightProducer; +import org.apache.arrow.flight.FlightStream; +import org.apache.arrow.flight.PutResult; +import org.apache.arrow.flight.Result; +import org.apache.arrow.flight.SchemaResult; +import org.apache.arrow.flight.Ticket; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionClosePreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetCatalogsRequest; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSchemasRequest; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesRequest; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementUpdate; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandStatementUpdate; + +import com.google.protobuf.Any; +import com.google.protobuf.InvalidProtocolBufferException; + +import io.grpc.Status; + +/** + * API to Implement an Arrow Flight SQL producer. + */ +public abstract class FlightSQLProducer implements FlightProducer, AutoCloseable { + + @Override + public FlightInfo getFlightInfo(CallContext context, FlightDescriptor descriptor) { + final Any command = FlightSQLUtils.parseOrThrow(descriptor.getCommand()); + + if (command.is(CommandStatementQuery.class)) { + return getFlightInfoStatement(FlightSQLUtils.unpackOrThrow(command, CommandStatementQuery.class), descriptor, + context); + + } else if (command.is(CommandPreparedStatementQuery.class)) { + return getFlightInfoPreparedStatement( + FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), descriptor, context); + } + + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + + /** + * Get information about a particular SQL query based data stream. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoStatement(CommandStatementQuery command, FlightDescriptor descriptor, + CallContext context); + + /** + * Get information about a particular prepared statement data stream. + * + * @param command The prepared statement to generate the data stream. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoPreparedStatement(CommandPreparedStatementQuery command, + FlightDescriptor descriptor, CallContext context); + + @Override + public SchemaResult getSchema(CallContext context, FlightDescriptor descriptor) { + final Any command = FlightSQLUtils.parseOrThrow(descriptor.getCommand()); + + if (command.is(CommandStatementQuery.class)) { + return getSchemaStatement(FlightSQLUtils.unpackOrThrow(command, CommandStatementQuery.class), descriptor, + context); + } + + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + + /** + * Get schema about a particular SQL query based data stream. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Schema for the stream. + */ + public abstract SchemaResult getSchemaStatement(CommandStatementQuery command, FlightDescriptor descriptor, + CallContext context); + + @Override + public Runnable acceptPut(CallContext context, FlightStream flightStream, StreamListener ackStream) { + final Any command = FlightSQLUtils.parseOrThrow(flightStream.getDescriptor().getCommand()); + + if (command.is(CommandStatementUpdate.class)) { + return acceptPutStatement( + FlightSQLUtils.unpackOrThrow(command, CommandStatementUpdate.class), + context, flightStream, ackStream); + + } else if (command.is(CommandPreparedStatementUpdate.class)) { + return acceptPutPreparedStatementUpdate( + FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementUpdate.class), + context, flightStream, ackStream); + + } else if (command.is(CommandPreparedStatementQuery.class)) { + return acceptPutPreparedStatementQuery( + FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), + context, flightStream, ackStream); + } + + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + + /** + * Accept uploaded data for a particular SQL query based data stream. PutResults must be in the form of a + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.DoPutUpdateResult}. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The result data stream. + * @return A runnable to process the stream. + */ + public abstract Runnable acceptPutStatement(CommandStatementUpdate command, CallContext context, + FlightStream flightStream, StreamListener ackStream); + + /** + * Accept uploaded data for a particular prepared statement data stream. PutResults must be in the form of a + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.DoPutUpdateResult}. + * + * @param command The prepared statement to generate the data stream. + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The result data stream. + * @return A runnable to process the stream. + */ + public abstract Runnable acceptPutPreparedStatementUpdate(CommandPreparedStatementUpdate command, + CallContext context, FlightStream flightStream, StreamListener ackStream); + + /** + * Accept uploaded parameter values for a particular prepared statement query. + * + * @param command The prepared statement the parameter values will bind to. + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The result data stream. + * @return A runnable to process the stream. + */ + public abstract Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery command, + CallContext context, FlightStream flightStream, StreamListener ackStream); + + @Override + public void doAction(CallContext context, Action action, StreamListener listener) { + + if (action.getType().equals(FLIGHT_SQL_GETSQLCAPABILITIES.getType())) { + getSqlCapabilities(context, listener); + + } else if (action.getType().equals(FLIGHT_SQL_GETCATALOGS.getType())) { + final ActionGetCatalogsRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), + ActionGetCatalogsRequest.class); + getCatalogs(request, context, listener); + + } else if (action.getType().equals(FLIGHT_SQL_GETSCHEMAS.getType())) { + final ActionGetSchemasRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), + ActionGetSchemasRequest.class); + getSchemas(request, context, listener); + + } else if (action.getType().equals(FLIGHT_SQL_GETTABLES.getType())) { + final ActionGetTablesRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), + ActionGetTablesRequest.class); + getTables(request, context, listener); + + } else if (action.getType().equals(FLIGHT_SQL_GETTABLETYPES.getType())) { + getTableTypes(context, listener); + + } else if (action.getType().equals(FLIGHT_SQL_GETPREPAREDSTATEMENT.getType())) { + final ActionGetPreparedStatementRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), + ActionGetPreparedStatementRequest.class); + getPreparedStatement(request, context, listener); + + } else if (action.getType().equals(FLIGHT_SQL_CLOSEPREPAREDSTATEMENT.getType())) { + final ActionClosePreparedStatementRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), + ActionClosePreparedStatementRequest.class); + closePreparedStatement(request, context, listener); + } + } + + /** + * Returns the SQL Capabilities of the server by returning a + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSQLCapabilitiesResult} in a {@link Result}. + * + * @param context Per-call context. + * @param listener A stream of responses. + */ + public abstract void getSqlCapabilities(CallContext context, StreamListener listener); + + /** + * Returns the available catalogs by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetCatalogsResult} objects in {@link Result} objects. + * + * @param request request filter parameters. + * @param context Per-call context. + * @param listener A stream of responses. + */ + public abstract void getCatalogs(ActionGetCatalogsRequest request, CallContext context, + StreamListener listener); + + /** + * Returns the available schemas by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSchemasResult} objects in {@link Result} objects. + * + * @param request request filter parameters. + * @param context Per-call context. + * @param listener A stream of responses. + */ + public abstract void getSchemas(ActionGetSchemasRequest request, CallContext context, + StreamListener listener); + + /** + * Returns the available table types by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTableTypesResult} objects in {@link Result} objects. + * + * @param context Per-call context. + * @param listener A stream of responses. + */ + public abstract void getTableTypes(CallContext context, StreamListener listener); + + /** + * Returns the available tables by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult} objects in {@link Result} objects. + * + * @param request request filter parameters. + * @param context Per-call context. + * @param listener A stream of responses. + */ + public abstract void getTables(ActionGetTablesRequest request, CallContext context, StreamListener listener); + + /** + * Creates a prepared statement on the server and returns a handle and metadata for in a + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult} object in a {@link Result} + * object. + * + * @param request The sql command to generate the prepared statement. + * @param context Per-call context. + * @param listener A stream of responses. + */ + public abstract void getPreparedStatement(ActionGetPreparedStatementRequest request, CallContext context, + StreamListener listener); + + /** + * Closes a prepared statement on the server. No result is expected. + * + * @param request The sql command to generate the prepared statement. + * @param context Per-call context. + * @param listener A stream of responses. + */ + public abstract void closePreparedStatement(ActionClosePreparedStatementRequest request, CallContext context, + StreamListener listener); + + @Override + public void listActions(CallContext context, StreamListener listener) { + FLIGHT_SQL_ACTIONS.forEach(action -> listener.onNext(action)); + listener.onCompleted(); + } + + @Override + public void getStream(CallContext context, Ticket ticket, ServerStreamListener listener) { + final Any command; + + try { + command = Any.parseFrom(ticket.getBytes()); + } catch (InvalidProtocolBufferException e) { + listener.error(e); + return; + } + + if (command.is(CommandStatementQuery.class)) { + getStreamStatement(FlightSQLUtils.unpackOrThrow(command, CommandStatementQuery.class), + context, ticket, listener); + + } else if (command.is(CommandPreparedStatementQuery.class)) { + getStreamPreparedStatement(FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), + context, ticket, listener); + } + + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + + /** + * Return data for a SQL query based data stream. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamStatement(CommandStatementQuery command, CallContext context, Ticket ticket, + ServerStreamListener listener); + + /** + * Return data for a particular prepared statement query instance. + * + * @param command The prepared statement to generate the data stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamPreparedStatement(CommandPreparedStatementQuery command, CallContext context, + Ticket ticket, ServerStreamListener listener); +} diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java new file mode 100644 index 00000000000..9e77699f4c4 --- /dev/null +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.sql; + +import java.sql.Types; +import java.util.List; + +import org.apache.arrow.flight.ActionType; +import org.apache.arrow.vector.types.DateUnit; +import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.TimeUnit; +import org.apache.arrow.vector.types.pojo.ArrowType; + +import com.google.common.collect.ImmutableList; +import com.google.protobuf.Any; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.Message; + +/** + * Utilities to work with Flight SQL semantics. + */ +public final class FlightSQLUtils { + + private static final int BIT_WIDTH8 = 8; + private static final int BIT_WIDTH_16 = 16; + private static final int BIT_WIDTH_32 = 32; + private static final int BIT_WIDTH_64 = 64; + private static final boolean IS_SIGNED_FALSE = false; + private static final boolean IS_SIGNED_TRUE = true; + + public static final ActionType FLIGHT_SQL_GETSQLCAPABILITIES = new ActionType("GetSQLCapabilities", + "Retrieves details of SQL capabilities of the Flight server. \n" + + "Request Message: N/A\n" + + "Response Message: SQLCapabilitiesResult"); + + public static final ActionType FLIGHT_SQL_GETCATALOGS = new ActionType("GetCatalogs", + "Retrieves a list of all catalogs available on the server. \n" + + "Request Message: GetCatalogsRequest\n" + + "Response Message: GetCatalogsResult"); + + public static final ActionType FLIGHT_SQL_GETSCHEMAS = new ActionType("GetSchemas", + "Retrieves a list of schemas available on the server. \n" + + "Request Message: GetSchemasRequest\n" + + "Response Message: GetSchemasResult"); + + public static final ActionType FLIGHT_SQL_GETTABLES = new ActionType("GetTables", + "Retrieves a list of tables available on the server. \n" + + "Request Message: GetTablesRequest\n" + + "Response Message: GetTablesResult"); + + public static final ActionType FLIGHT_SQL_GETTABLETYPES = new ActionType("GetTableTypes", + "Retrieves a list of table types available on the server. \n" + + "Request Message: N/A\n" + + "Response Message: GetTableTypesResult"); + + public static final ActionType FLIGHT_SQL_GETPREPAREDSTATEMENT = new ActionType("GetPreparedStatement", + "Creates a reusable prepared statement resource on the server. \n" + + "Request Message: ActionRequestGetPreparedStatement\n" + + "Response Message: ActionResponseGetPreparedStatement"); + + public static final ActionType FLIGHT_SQL_CLOSEPREPAREDSTATEMENT = new ActionType("ClosePreparedStatement", + "Closes a reusable prepared statement resource on the server. \n" + + "Request Message: ActionRequestClosePreparedStatement\n" + + "Response Message: N/A"); + + public static final List FLIGHT_SQL_ACTIONS = ImmutableList.of( + FLIGHT_SQL_GETSQLCAPABILITIES, + FLIGHT_SQL_GETCATALOGS, + FLIGHT_SQL_GETSCHEMAS, + FLIGHT_SQL_GETTABLES, + FLIGHT_SQL_GETTABLETYPES, + FLIGHT_SQL_GETPREPAREDSTATEMENT, + FLIGHT_SQL_CLOSEPREPAREDSTATEMENT + ); + + /** + * Converts {@link java.sql.Types} values returned from JDBC Apis to Arrow types. + * + * @param jdbcDataType {@link java.sql.Types} value. + * @param precision Precision of the type. + * @param scale Scale of the type. + * @return The Arrow equivalent type. + */ + public static ArrowType getArrowTypeFromJDBCType(int jdbcDataType, int precision, int scale) { + + switch (jdbcDataType) { + case Types.BIT: + case Types.BOOLEAN: + return ArrowType.Bool.INSTANCE; + case Types.TINYINT: + return new ArrowType.Int(BIT_WIDTH8, IS_SIGNED_TRUE); + case Types.SMALLINT: + return new ArrowType.Int(BIT_WIDTH_16, IS_SIGNED_TRUE); + case Types.INTEGER: + return new ArrowType.Int(BIT_WIDTH_32, IS_SIGNED_TRUE); + case Types.BIGINT: + return new ArrowType.Int(BIT_WIDTH_64, IS_SIGNED_TRUE); + case Types.FLOAT: + case Types.REAL: + return new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE); + case Types.DOUBLE: + return new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE); + case Types.NUMERIC: + case Types.DECIMAL: + return new ArrowType.Decimal(precision, scale); + case Types.DATE: + return new ArrowType.Date(DateUnit.DAY); + case Types.TIME: + return new ArrowType.Time(TimeUnit.MILLISECOND, BIT_WIDTH_32); + case Types.TIMESTAMP: + return new ArrowType.Timestamp(TimeUnit.MILLISECOND, null); + case Types.BINARY: + case Types.VARBINARY: + case Types.LONGVARBINARY: + return ArrowType.Binary.INSTANCE; + case Types.NULL: + return ArrowType.Null.INSTANCE; + + case Types.CHAR: + case Types.VARCHAR: + case Types.LONGVARCHAR: + case Types.CLOB: + case Types.NCHAR: + case Types.NVARCHAR: + case Types.LONGNVARCHAR: + case Types.NCLOB: + + case Types.OTHER: + case Types.JAVA_OBJECT: + case Types.DISTINCT: + case Types.STRUCT: + case Types.ARRAY: + case Types.BLOB: + case Types.REF: + case Types.DATALINK: + case Types.ROWID: + case Types.SQLXML: + case Types.REF_CURSOR: + case Types.TIME_WITH_TIMEZONE: + case Types.TIMESTAMP_WITH_TIMEZONE: + default: + return ArrowType.Utf8.INSTANCE; + // throw new UnsupportedOperationException(); + } + } + + /** + * Helper to parse {@link com.google.protobuf.Any} objects to the specific protobuf object. + * + * @param source the raw bytes source value. + * @return the materialized protobuf object. + */ + public static Any parseOrThrow(byte[] source) { + try { + return Any.parseFrom(source); + } catch (InvalidProtocolBufferException e) { + throw new AssertionError(e.getMessage()); + } + } + + /** + * Helper to unpack {@link com.google.protobuf.Any} objects to the specific protobuf object. + * + * @param source the parsed Source value. + * @param as the class to unpack as. + * @param the class to unpack as. + * @return the materialized protobuf object. + */ + public static T unpackOrThrow(Any source, Class as) { + try { + return source.unpack(as); + } catch (InvalidProtocolBufferException e) { + throw new AssertionError(e.getMessage()); + } + } + + /** + * Helper to parse and unpack {@link com.google.protobuf.Any} objects to the specific protobuf object. + * + * @param source the raw bytes source value. + * @param as the class to unpack as. + * @param the class to unpack as. + * @return the materialized protobuf object. + */ + public static T unpackAndParseOrThrow(byte[] source, Class as) { + return unpackOrThrow(parseOrThrow(source), as); + } +} diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java new file mode 100644 index 00000000000..b775737decc --- /dev/null +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java @@ -0,0 +1,262 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight; + +import static org.apache.arrow.flight.sql.FlightSQLClientUtils.getPreparedStatement; +import static org.apache.arrow.flight.sql.FlightSQLClientUtils.getTables; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +import org.apache.arrow.flight.sql.FlightSQLClientUtils; +import org.apache.arrow.flight.sql.FlightSQLExample; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionClosePreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesRequest; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementQuery; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.memory.util.ArrowBufPointer; +import org.apache.arrow.util.AutoCloseables; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.arrow.vector.util.ElementAddressableVectorIterator; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; + +/** + * Test direct usage of Flight SQL workflows. + */ +public class TestFlightSQL { + private static BufferAllocator allocator; + private static FlightServer server; + + private static FlightClient client; + + protected static final Schema SCHEMA_INT_TABLE = new Schema(Arrays.asList( + new Field("KEYNAME", new + FieldType(true, ArrowType.Utf8.INSTANCE, null), + null), + new Field("VALUE", + new FieldType(true, new ArrowType.Int(32, true), null), + null))); + + @BeforeClass + public static void setUp() throws Exception { + allocator = new RootAllocator(Integer.MAX_VALUE); + + final Location serverLocation = Location.forGrpcInsecure(FlightTestUtil.LOCALHOST, 0); + server = FlightServer.builder(allocator, serverLocation, new FlightSQLExample(serverLocation)).build(); + server.start(); + + final Location clientLocation = Location.forGrpcInsecure(FlightTestUtil.LOCALHOST, server.getPort()); + client = FlightClient.builder(allocator, clientLocation).build(); + } + + @AfterClass + public static void tearDown() throws Exception { + AutoCloseables.close(client, server, allocator); + } + + @Test + public void testGetTables() throws Exception { + // Arrange + final ActionGetTablesResult expected = ActionGetTablesResult.newBuilder() + .setSchema("APP") + .setTable("INTTABLE") + .setTableType("TABLE") + .setSchemaMetadata(ByteString.copyFrom(SCHEMA_INT_TABLE.toByteArray())) + .build(); + + // Act + final Iterator results = client.doAction(new Action("GetTables", + Any.pack(ActionGetTablesRequest + .newBuilder() + .addTableTypes("TABLE") + .setIncludeSchema(true) + .build()) + .toByteArray())); + + // Assert + while (results.hasNext()) { + ActionGetTablesResult actual = Any.parseFrom(results.next().getBody()).unpack(ActionGetTablesResult.class); + assertEquals(expected, actual); + } + } + + @Test + public void testGetTablesWithFlightSQLClientUtils() throws Exception { + // Arrange + final ActionGetTablesResult expected = ActionGetTablesResult.newBuilder() + .setSchema("APP") + .setTable("INTTABLE") + .setTableType("TABLE") + .setSchemaMetadata(ByteString.copyFrom(SCHEMA_INT_TABLE.toByteArray())) + .build(); + + // Act + final List results = getTables(client, null, null, null, + Collections.singletonList("TABLE"), true); + + // Assert + assertEquals(1, results.size()); + assertEquals(expected, results.get(0)); + } + + @Test + public void testSimplePrepStmt() throws Exception { + final Iterator preparedStatementResults = client.doAction(new Action("GetPreparedStatement", + Any.pack(ActionGetPreparedStatementRequest + .newBuilder() + .setQuery("Select * from intTable") + .build()) + .toByteArray())); + + assertTrue(preparedStatementResults.hasNext()); + final ActionGetPreparedStatementResult preparedStatementResult = + Any.parseFrom(preparedStatementResults.next().getBody()).unpack(ActionGetPreparedStatementResult.class); + assertFalse(preparedStatementResults.hasNext()); + + final Schema actualSchema = Schema.deserialize(preparedStatementResult.getDatasetSchema().asReadOnlyByteBuffer()); + assertEquals(SCHEMA_INT_TABLE, actualSchema); + + final FlightDescriptor descriptor = FlightDescriptor + .command(Any.pack(CommandPreparedStatementQuery.newBuilder() + .setClientExecutionHandle(ByteString.copyFrom(new byte[]{1, 2, 3, 4})) + .setPreparedStatementHandle(preparedStatementResult.getPreparedStatementHandle()) + .build()) + .toByteArray()); + + final FlightInfo info = client.getInfo(descriptor); + assertEquals(SCHEMA_INT_TABLE, info.getSchema()); + + final FlightStream stream = client.getStream(info.getEndpoints().get(0).getTicket()); + assertEquals(SCHEMA_INT_TABLE, stream.getSchema()); + + List actualStringResults = new ArrayList<>(); + List actualIntResults = new ArrayList<>(); + while (stream.next()) { + final VectorSchemaRoot root = stream.getRoot(); + final long rowCount = root.getRowCount(); + + for (Field field : root.getSchema().getFields()) { + final FieldVector fieldVector = root.getVector(field.getName()); + + if (fieldVector instanceof VarCharVector) { + + final ElementAddressableVectorIterator it = + new ElementAddressableVectorIterator<>((VarCharVector) fieldVector); + + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + final ArrowBufPointer pt = it.next(); + final byte[] bytes = new byte[(int) pt.getLength()]; + pt.getBuf().getBytes(pt.getOffset(), bytes); + + actualStringResults.add(new String(bytes, StandardCharsets.UTF_8)); + } + } else if (fieldVector instanceof IntVector) { + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + actualIntResults.add(((IntVector) fieldVector).get(rowIndex)); + } + } + } + } + stream.getRoot().clear(); + + assertEquals(Arrays.asList("one", "zero", "negative one"), actualStringResults); + assertEquals(Arrays.asList(1, 0, -1), actualIntResults); + + final Iterator closePreparedStatementResults = client.doAction(new Action("ClosePreparedStatement", + Any.pack(ActionClosePreparedStatementRequest + .newBuilder() + .setPreparedStatementHandleBytes(preparedStatementResult.getPreparedStatementHandle()) + .build()) + .toByteArray())); + assertFalse(closePreparedStatementResults.hasNext()); + } + + @Test + public void testSimplePrepStmtWithFlightSQLClientUtils() throws Exception { + final FlightSQLClientUtils.FlightSQLPreparedStatement preparedStatement = + getPreparedStatement(client, "Select * from intTable"); + + final Schema actualSchema = preparedStatement.getResultSetSchema(); + assertEquals(SCHEMA_INT_TABLE, actualSchema); + + final FlightInfo info = preparedStatement.executeQuery(); + assertEquals(SCHEMA_INT_TABLE, info.getSchema()); + + final FlightStream stream = client.getStream(info.getEndpoints().get(0).getTicket()); + assertEquals(SCHEMA_INT_TABLE, stream.getSchema()); + + List actualStringResults = new ArrayList<>(); + List actualIntResults = new ArrayList<>(); + while (stream.next()) { + final VectorSchemaRoot root = stream.getRoot(); + final long rowCount = root.getRowCount(); + + for (Field field : root.getSchema().getFields()) { + final FieldVector fieldVector = root.getVector(field.getName()); + + if (fieldVector instanceof VarCharVector) { + + final ElementAddressableVectorIterator it = + new ElementAddressableVectorIterator<>((VarCharVector) fieldVector); + + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + final ArrowBufPointer pt = it.next(); + final byte[] bytes = new byte[(int) pt.getLength()]; + pt.getBuf().getBytes(pt.getOffset(), bytes); + + actualStringResults.add(new String(bytes, StandardCharsets.UTF_8)); + } + } else if (fieldVector instanceof IntVector) { + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + actualIntResults.add(((IntVector) fieldVector).get(rowIndex)); + } + } + } + } + stream.getRoot().clear(); + + assertEquals(Arrays.asList("one", "zero", "negative one"), actualStringResults); + assertEquals(Arrays.asList(1, 0, -1), actualIntResults); + + AutoCloseables.close(preparedStatement); + assertTrue(preparedStatement.isClosed()); + } +} diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java new file mode 100644 index 00000000000..b54621fa21f --- /dev/null +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java @@ -0,0 +1,601 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.sql; + +import static org.apache.arrow.flight.sql.FlightSQLUtils.getArrowTypeFromJDBCType; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ParameterMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.stream.Stream; + +import org.apache.arrow.flight.CallStatus; +import org.apache.arrow.flight.Criteria; +import org.apache.arrow.flight.FlightDescriptor; +import org.apache.arrow.flight.FlightEndpoint; +import org.apache.arrow.flight.FlightInfo; +import org.apache.arrow.flight.FlightRuntimeException; +import org.apache.arrow.flight.FlightStatusCode; +import org.apache.arrow.flight.FlightStream; +import org.apache.arrow.flight.Location; +import org.apache.arrow.flight.PutResult; +import org.apache.arrow.flight.Result; +import org.apache.arrow.flight.SchemaResult; +import org.apache.arrow.flight.Ticket; +import org.apache.arrow.flight.sql.impl.FlightSQL; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult; +import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementUpdate; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSQL.CommandStatementUpdate; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.util.AutoCloseables; +import org.apache.arrow.util.Preconditions; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.commons.dbcp2.ConnectionFactory; +import org.apache.commons.dbcp2.DriverManagerConnectionFactory; +import org.apache.commons.dbcp2.PoolableConnection; +import org.apache.commons.dbcp2.PoolableConnectionFactory; +import org.apache.commons.dbcp2.PoolingDataSource; +import org.apache.commons.pool2.ObjectPool; +import org.apache.commons.pool2.impl.GenericObjectPool; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.cache.RemovalListener; +import com.google.common.cache.RemovalNotification; +import com.google.common.collect.ImmutableList; +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; + +import io.grpc.Status; + +/** + * Proof of concept {@link FlightSQLProducer} implementation showing an Apache Derby backed Flight SQL server capable + * of the following workflows: + * - returning a list of tables from the action "GetTables". + * - creation of a prepared statement from the action "GetPreparedStatement". + * - execution of a prepared statement by using a {@link CommandPreparedStatementQuery} with getFlightInfo and + * getStream. + */ +public class FlightSQLExample extends FlightSQLProducer implements AutoCloseable { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FlightSQLExample.class); + + private static final int BATCH_ROW_SIZE = 1000; + + private final Location location; + private final PoolingDataSource dataSource; + + private final LoadingCache commandExecutePreparedStatementLoadingCache; + private final LoadingCache preparedStatementLoadingCache; + + public FlightSQLExample(Location location) { + removeDerbyDatabaseIfExists(); + populateDerbyDatabase(); + + final ConnectionFactory connectionFactory = + new DriverManagerConnectionFactory("jdbc:derby:target/derbyDB", null); + final PoolableConnectionFactory poolableConnectionFactory = new PoolableConnectionFactory(connectionFactory, null); + final ObjectPool connectionPool = new GenericObjectPool<>(poolableConnectionFactory); + poolableConnectionFactory.setPool(connectionPool); + + // PoolingDataSource takes ownership of connectionPool. + dataSource = new PoolingDataSource<>(connectionPool); + + preparedStatementLoadingCache = + CacheBuilder.newBuilder() + .maximumSize(100) + .expireAfterWrite(10, java.util.concurrent.TimeUnit.MINUTES) + .removalListener(new PreparedStatementRemovalListener()) + .build(new PreparedStatementCacheLoader(dataSource)); + + commandExecutePreparedStatementLoadingCache = + CacheBuilder.newBuilder() + .maximumSize(100) + .expireAfterWrite(10, java.util.concurrent.TimeUnit.MINUTES) + .removalListener(new CommandExecutePreparedStatementRemovalListener()) + .build(new CommandExecutePreparedStatementCacheLoader(preparedStatementLoadingCache)); + + this.location = location; + } + + @Override + public void getTables(FlightSQL.ActionGetTablesRequest request, CallContext context, + StreamListener listener) { + try { + final String catalog = (request.getCatalog().isEmpty() ? null : request.getCatalog()); + + final String schemaFilterPattern = + (request.getSchemaFilterPattern().isEmpty() ? null : request.getSchemaFilterPattern()); + + final String tableFilterPattern = + (request.getTableNameFilterPattern().isEmpty() ? null : request.getTableNameFilterPattern()); + + final String[] tableTypes = request.getTableTypesList().size() == 0 ? null : + request.getTableTypesList().toArray(new String[request.getTableTypesList().size()]); + + try (final Connection connection = dataSource.getConnection(); + final ResultSet tables = connection.getMetaData().getTables( + catalog, + schemaFilterPattern, + tableFilterPattern, + tableTypes)) { + while (tables.next()) { + listener.onNext(getTableResult(tables, request.getIncludeSchema())); + } + } + } catch (SQLException e) { + listener.onError(e); + } finally { + listener.onCompleted(); + } + } + + private Result getTableResult(final ResultSet tables, boolean includeSchema) throws SQLException { + + final String catalog = tables.getString("TABLE_CAT"); + final String schema = tables.getString("TABLE_SCHEM"); + final String table = tables.getString("TABLE_NAME"); + final String tableType = tables.getString("TABLE_TYPE"); + + final ActionGetTablesResult.Builder builder = ActionGetTablesResult.newBuilder() + .setCatalog(catalog) + .setSchema(schema) + .setTable(table) + .setTableType(tableType); + + if (includeSchema) { + final Schema pojoSchema = buildSchema(catalog, schema, table); + builder.setSchemaMetadata(ByteString.copyFrom(pojoSchema.toByteArray())); + } + + return new Result(Any.pack(builder.build()).toByteArray()); + } + + @Override + public void getPreparedStatement(FlightSQL.ActionGetPreparedStatementRequest request, CallContext context, + StreamListener listener) { + final PreparedStatementCacheKey handle = new PreparedStatementCacheKey( + UUID.randomUUID().toString(), request.getQuery()); + + try { + final PreparedStatementContext preparedStatementContext = preparedStatementLoadingCache.get(handle); + final PreparedStatement preparedStatement = preparedStatementContext.getPreparedStatement(); + + // todo + final Schema pojoParameterMetaDataSchema = buildSchema(preparedStatement.getParameterMetaData()); + final Schema pojoResultSetSchema = buildSchema(preparedStatement.getMetaData()); + + listener.onNext(new Result( + Any.pack(ActionGetPreparedStatementResult.newBuilder() + .setDatasetSchema(ByteString.copyFrom(pojoResultSetSchema.toByteArray())) + .setParameterSchema(ByteString.copyFrom(pojoParameterMetaDataSchema.toByteArray())) + .setPreparedStatementHandle(handle.toProtocol()) + .build()) + .toByteArray())); + + } catch (ExecutionException | SQLException e) { + listener.onError(e); + } finally { + listener.onCompleted(); + } + } + + @Override + public FlightInfo getFlightInfoPreparedStatement(CommandPreparedStatementQuery command, FlightDescriptor descriptor, + CallContext context) { + try { + final ResultSet resultSet = commandExecutePreparedStatementLoadingCache.get(command); + final Schema schema = buildSchema(resultSet.getMetaData()); + + final List endpoints = ImmutableList + .of(new FlightEndpoint(new Ticket(Any.pack(command).toByteArray()), location)); + + return new FlightInfo(schema, descriptor, endpoints, -1, -1); + } catch (ExecutionException | SQLException e) { + logger.error("There was a problem executing the prepared statement", e); + throw new FlightRuntimeException(new CallStatus(FlightStatusCode.INTERNAL, e, e.getMessage(), null)); + } + } + + private Schema buildSchema(String catalog, String schema, String table) throws SQLException { + final List fields = new ArrayList<>(); + + try (final Connection connection = dataSource.getConnection(); + final ResultSet columns = connection.getMetaData().getColumns( + catalog, + schema, + table, + null);) { + + while (columns.next()) { + final String columnName = columns.getString("COLUMN_NAME"); + final int jdbcDataType = columns.getInt("DATA_TYPE"); + final String jdbcDataTypeName = columns.getString("TYPE_NAME"); + final String jdbcIsNullable = columns.getString("IS_NULLABLE"); + final boolean arrowIsNullable = jdbcIsNullable.equals("YES"); + + final int precision = columns.getInt("DECIMAL_DIGITS"); + final int scale = columns.getInt("COLUMN_SIZE"); + final ArrowType arrowType = FlightSQLUtils.getArrowTypeFromJDBCType(jdbcDataType, precision, scale); + + final FieldType fieldType = new FieldType(arrowIsNullable, arrowType, /*dictionary=*/null); + fields.add(new Field(columnName, fieldType, null)); + } + } + + return new Schema(fields); + } + + @Override + public void getStreamPreparedStatement(CommandPreparedStatementQuery command, CallContext context, Ticket ticket, + ServerStreamListener listener) { + try { + final ResultSet resultSet = commandExecutePreparedStatementLoadingCache.get(command); + final ResultSetMetaData resultSetMetaData = resultSet.getMetaData(); + final Schema schema = buildSchema(resultSetMetaData); + final DictionaryProvider dictionaryProvider = new DictionaryProvider.MapDictionaryProvider(); + + try (final BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE); + final VectorSchemaRoot root = VectorSchemaRoot.create(schema, allocator)) { + + listener.start(root, dictionaryProvider); + final int columnCount = resultSetMetaData.getColumnCount(); + + while (resultSet.next()) { + final int rowCounter = readBatch(resultSet, resultSetMetaData, root, columnCount); + + for (int resultSetColumnCounter = 1; resultSetColumnCounter <= columnCount; resultSetColumnCounter++) { + final String columnName = resultSetMetaData.getColumnName(resultSetColumnCounter); + root.getVector(columnName).setValueCount(rowCounter); + } + + root.setRowCount(rowCounter); + listener.putNext(); + } + } + } catch (ExecutionException | SQLException e) { + listener.error(e); + } finally { + listener.completed(); + commandExecutePreparedStatementLoadingCache.invalidate(command); + } + } + + private int readBatch(ResultSet resultSet, ResultSetMetaData resultSetMetaData, VectorSchemaRoot root, + int columnCount) throws SQLException { + int rowCounter = 0; + do { + for (int resultSetColumnCounter = 1; resultSetColumnCounter <= columnCount; resultSetColumnCounter++) { + final String columnName = resultSetMetaData.getColumnName(resultSetColumnCounter); + + final FieldVector fieldVector = root.getVector(columnName); + + if (fieldVector instanceof VarCharVector) { + final String value = resultSet.getString(resultSetColumnCounter); + if (resultSet.wasNull()) { + // TODO handle null + } else { + ((VarCharVector) fieldVector).setSafe(rowCounter, value.getBytes(), 0, value.length()); + } + } else if (fieldVector instanceof IntVector) { + final int value = resultSet.getInt(resultSetColumnCounter); + + if (resultSet.wasNull()) { + // TODO handle null + } else { + ((IntVector) fieldVector).setSafe(rowCounter, value); + } + } else { + throw new UnsupportedOperationException(); + } + } + rowCounter++; + } + while (rowCounter < BATCH_ROW_SIZE && resultSet.next()); + + return rowCounter; + } + + + @Override + public void closePreparedStatement(FlightSQL.ActionClosePreparedStatementRequest request, CallContext context, + StreamListener listener) { + try { + preparedStatementLoadingCache.invalidate( + PreparedStatementCacheKey.fromProtocol(request.getPreparedStatementHandleBytes())); + } catch (InvalidProtocolBufferException e) { + listener.onError(e); + } finally { + listener.onCompleted(); + } + } + + private Schema buildSchema(ResultSetMetaData resultSetMetaData) throws SQLException { + Preconditions.checkNotNull(resultSetMetaData, "ResultSetMetaData object can't be null"); + final List resultSetFields = new ArrayList<>(); + + for (int resultSetCounter = 1; resultSetCounter <= resultSetMetaData.getColumnCount(); resultSetCounter++) { + final String name = resultSetMetaData.getColumnName(resultSetCounter); + + final int jdbcDataType = resultSetMetaData.getColumnType(resultSetCounter); + + final int jdbcIsNullable = resultSetMetaData.isNullable(resultSetCounter); + final boolean arrowIsNullable = jdbcIsNullable == ResultSetMetaData.columnNullable; + + final int precision = resultSetMetaData.getPrecision(resultSetCounter); + final int scale = resultSetMetaData.getScale(resultSetCounter); + + final ArrowType arrowType = getArrowTypeFromJDBCType(jdbcDataType, precision, scale); + + final FieldType fieldType = new FieldType(arrowIsNullable, arrowType, /*dictionary=*/null); + resultSetFields.add(new Field(name, fieldType, null)); + } + final Schema pojoResultSetSchema = new Schema(resultSetFields); + return pojoResultSetSchema; + } + + private Schema buildSchema(ParameterMetaData parameterMetaData) throws SQLException { + Preconditions.checkNotNull(parameterMetaData, "ParameterMetaData object can't be null"); + final List parameterFields = new ArrayList<>(); + + for (int parameterCounter = 1; parameterCounter <= parameterMetaData.getParameterCount(); parameterCounter++) { + final int jdbcDataType = parameterMetaData.getParameterType(parameterCounter); + + final int jdbcIsNullable = parameterMetaData.isNullable(parameterCounter); + final boolean arrowIsNullable = jdbcIsNullable == ParameterMetaData.parameterNullable; + + final int precision = parameterMetaData.getPrecision(parameterCounter); + final int scale = parameterMetaData.getScale(parameterCounter); + + final ArrowType arrowType = getArrowTypeFromJDBCType(jdbcDataType, precision, scale); + + final FieldType fieldType = new FieldType(arrowIsNullable, arrowType, /*dictionary=*/null); + parameterFields.add(new Field(null, fieldType, null)); + } + final Schema pojoParameterMetaDataSchema = new Schema(parameterFields); + return pojoParameterMetaDataSchema; + } + + @Override + public void close() throws Exception { + try { + commandExecutePreparedStatementLoadingCache.cleanUp(); + } catch (Throwable e) { + // Swallow + } + + try { + preparedStatementLoadingCache.cleanUp(); + } catch (Throwable e) { + // Swallow + } + + AutoCloseables.close(dataSource); + } + + private static class CommandExecutePreparedStatementRemovalListener + implements RemovalListener { + @Override + public void onRemoval(RemovalNotification notification) { + try { + AutoCloseables.close(notification.getValue()); + } catch (Throwable e) { + // Swallow + } + } + } + + private static class CommandExecutePreparedStatementCacheLoader + extends CacheLoader { + + private final LoadingCache preparedStatementLoadingCache; + + private CommandExecutePreparedStatementCacheLoader(LoadingCache preparedStatementLoadingCache) { + this.preparedStatementLoadingCache = preparedStatementLoadingCache; + } + + @Override + public ResultSet load(CommandPreparedStatementQuery commandExecutePreparedStatement) + throws SQLException, InvalidProtocolBufferException, ExecutionException { + final PreparedStatementCacheKey preparedStatementCacheKey = + PreparedStatementCacheKey.fromProtocol(commandExecutePreparedStatement.getPreparedStatementHandle()); + final PreparedStatementContext preparedStatementContext = preparedStatementLoadingCache + .get(preparedStatementCacheKey); + return preparedStatementContext.getPreparedStatement().executeQuery(); + } + } + + + private static class PreparedStatementRemovalListener implements RemovalListener { + @Override + public void onRemoval(RemovalNotification notification) { + try { + AutoCloseables.close(notification.getValue()); + } catch (Throwable e) { + // swallow + } + } + } + + private static class PreparedStatementCacheLoader extends CacheLoader { + + // Owned by parent class. + private final PoolingDataSource dataSource; + + private PreparedStatementCacheLoader(PoolingDataSource dataSource) { + this.dataSource = dataSource; + } + + @Override + public PreparedStatementContext load(PreparedStatementCacheKey key) throws SQLException { + + // Ownership of the connection will be passed to the context. + final Connection connection = dataSource.getConnection(); + try { + final PreparedStatement preparedStatement = connection.prepareStatement(key.getSql()); + return new PreparedStatementContext(connection, preparedStatement); + } catch (SQLException e) { + connection.close(); + throw e; + } + } + } + + private static void removeDerbyDatabaseIfExists() { + final Path path = Paths.get("target" + File.separator + "derbyDB"); + + try (final Stream walk = Files.walk(path)) { + walk.sorted(Comparator.reverseOrder()) + .map(Path::toFile) + .forEach(File::delete); + } catch (NoSuchFileException e) { + // Ignore as there was no data directory to clean up. + } catch (IOException e) { + throw new RuntimeException("Failed to remove derby data directory.", e); + } + } + + private static void populateDerbyDatabase() { + try (final Connection conn = DriverManager.getConnection("jdbc:derby:target/derbyDB;create=true")) { + conn.createStatement().execute("CREATE TABLE intTable (keyName varchar(100), value int)"); + conn.createStatement().execute("INSERT INTO intTable (keyName, value) VALUES ('one', 1)"); + conn.createStatement().execute("INSERT INTO intTable (keyName, value) VALUES ('zero', 0)"); + conn.createStatement().execute("INSERT INTO intTable (keyName, value) VALUES ('negative one', -1)"); + } catch (SQLException e) { + throw new RuntimeException("Failed to create derby database.", e); + } + } + + + @Override + public void listFlights(CallContext context, Criteria criteria, StreamListener listener) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public FlightInfo getFlightInfoStatement(CommandStatementQuery command, FlightDescriptor descriptor, + CallContext context) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public SchemaResult getSchema(CallContext context, FlightDescriptor descriptor) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public void doExchange(CallContext context, FlightStream reader, ServerStreamListener writer) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public void getSqlCapabilities(CallContext context, StreamListener listener) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public void getCatalogs(FlightSQL.ActionGetCatalogsRequest request, CallContext context, + StreamListener listener) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public void getSchemas(FlightSQL.ActionGetSchemasRequest request, CallContext context, + StreamListener listener) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public void getTableTypes(CallContext context, StreamListener listener) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public SchemaResult getSchemaStatement(CommandStatementQuery command, FlightDescriptor descriptor, + CallContext context) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public Runnable acceptPutStatement(CommandStatementUpdate command, + CallContext context, FlightStream flightStream, StreamListener ackStream) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public Runnable acceptPutPreparedStatementUpdate(CommandPreparedStatementUpdate command, CallContext context, + FlightStream flightStream, StreamListener ackStream) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery command, CallContext context, + FlightStream flightStream, StreamListener ackStream) { + // TODO - build example implementation + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + @Override + public void getStreamStatement(CommandStatementQuery command, CallContext context, Ticket ticket, + ServerStreamListener listener) { + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + +} diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementCacheKey.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementCacheKey.java new file mode 100644 index 00000000000..9c56e3162d2 --- /dev/null +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementCacheKey.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.sql; + +import java.util.Objects; + +import org.apache.arrow.flight.sql.impl.FlightSQLExample.PreparedStatementHandle; +import org.apache.arrow.util.Preconditions; + +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; + +class PreparedStatementCacheKey { + + private final String uuid; + private final String sql; + + PreparedStatementCacheKey(final String uuid, final String sql) { + this.uuid = uuid; + this.sql = sql; + } + + String getUuid() { + return uuid; + } + + String getSql() { + return sql; + } + + ByteString toProtocol() { + return Any.pack(org.apache.arrow.flight.sql.impl.FlightSQLExample.PreparedStatementHandle + .newBuilder() + .setSql(getSql()) + .setUuid(getUuid()) + .build()) + .toByteString(); + } + + static PreparedStatementCacheKey fromProtocol(ByteString byteString) throws InvalidProtocolBufferException { + final Any parsed = Any.parseFrom(byteString); + Preconditions.checkArgument(parsed.is(PreparedStatementHandle.class)); + + final PreparedStatementHandle preparedStatementHandle = parsed.unpack(PreparedStatementHandle.class); + return new PreparedStatementCacheKey(preparedStatementHandle.getUuid(), preparedStatementHandle.getSql()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof PreparedStatementCacheKey)) { + return false; + } + + PreparedStatementCacheKey that = (PreparedStatementCacheKey) o; + + return Objects.equals(uuid, that.uuid) && + Objects.equals(sql, that.sql); + } + + @Override + public int hashCode() { + return Objects.hash(uuid, sql); + } +} diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementContext.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementContext.java new file mode 100644 index 00000000000..cd38255fd03 --- /dev/null +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementContext.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.sql; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.util.Objects; + +import org.apache.arrow.util.AutoCloseables; + +class PreparedStatementContext implements AutoCloseable { + + private final Connection connection; + private final PreparedStatement preparedStatement; + + PreparedStatementContext(Connection connection, PreparedStatement preparedStatement) { + this.preparedStatement = preparedStatement; + this.connection = connection; + } + + PreparedStatement getPreparedStatement() { + return preparedStatement; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof PreparedStatementContext)) { + return false; + } + + PreparedStatementContext that = (PreparedStatementContext) o; + + return Objects.equals(connection, that.connection) && + Objects.equals(preparedStatement, that.preparedStatement); + } + + @Override + public int hashCode() { + return Objects.hash(connection, preparedStatement); + } + + @Override + public void close() throws Exception { + AutoCloseables.close(preparedStatement, connection); + } +} diff --git a/java/flight/flight-sql/src/test/protobuf/flightSQLExample.proto b/java/flight/flight-sql/src/test/protobuf/flightSQLExample.proto new file mode 100644 index 00000000000..c6ebfcabaf8 --- /dev/null +++ b/java/flight/flight-sql/src/test/protobuf/flightSQLExample.proto @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +syntax = "proto3"; + +option java_package = "org.apache.arrow.flight.sql.impl"; + +message PreparedStatementHandle { + string uuid = 1; + string sql = 2; +} diff --git a/java/pom.xml b/java/pom.xml index c8f79e77743..71376ee9935 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -673,6 +673,7 @@ plasma flight/flight-core flight/flight-grpc + flight/flight-sql performance algorithm adapter/avro From 026e9c71dd8de9b8b81564438cbd344f9b45e941 Mon Sep 17 00:00:00 2001 From: Ryan Nicholson Date: Mon, 31 Aug 2020 12:51:08 -0700 Subject: [PATCH 02/16] Update FlightSQL.proto Minor updates to fix field numbering and clarify usage in comments. --- format/FlightSQL.proto | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index 2ef7299becb..a7b68cd6dc4 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -150,10 +150,12 @@ message ActionGetPreparedStatementResult { // Opaque handle for the prepared statement on the server. bytes preparedStatementHandle = 1; + // If a result set generating query was provided, datasetSchema contains the // schema of the dataset as described in Schema.fbs::Schema. bytes datasetSchema = 2; - // schema of the expected parameters, if any existed, as described in Schema.fbs::Schema. + // If the query provided contained parameters, parameterSchema contains the + // Schema of the expected parameters as described in Schema.fbs::Schema. bytes parameterSchema = 3; } @@ -178,7 +180,7 @@ message ActionClosePreparedStatementRequest { */ message CommandStatementQuery { // The SQL syntax. - string query = 2; + string query = 1; } /* @@ -189,9 +191,9 @@ message CommandStatementQuery { */ message CommandPreparedStatementQuery { // Unique identifier for the instance of the prepared statement to execute. - bytes clientExecutionHandle = 2; + bytes clientExecutionHandle = 1; // Opaque handle for the prepared statement on the server. - bytes preparedStatementHandle = 3; + bytes preparedStatementHandle = 2; } /* @@ -201,7 +203,7 @@ message CommandPreparedStatementQuery { */ message CommandStatementUpdate { // The SQL syntax. - string query = 2; + string query = 1; } /* @@ -211,16 +213,18 @@ message CommandStatementUpdate { */ message CommandPreparedStatementUpdate { // Unique identifier for the instance of the prepared statement to execute. - bytes clientExecutionHandle = 2; + bytes clientExecutionHandle = 1; // Opaque handle for the prepared statement on the server. - bytes preparedStatementHandle = 3; + bytes preparedStatementHandle = 2; } /* * Returned from the RPC call DoPut when a CommandStatementUpdate * CommandPreparedStatementUpdate was in the request, containing - * results from the update. + * results from the update. */ message DoPutUpdateResult { + // The number of records updated. A return value of -1 represents + // an unknown updated record count. int64 recordCount = 1; } From 8b49dc9a44cad8470a040b6ed897fe0a3cbe5422 Mon Sep 17 00:00:00 2001 From: Ryan Nicholson Date: Tue, 1 Sep 2020 09:45:38 -0700 Subject: [PATCH 03/16] FlightSQL.proto formatting feedback Update with initial formatting/naming feedback and add ResultsOrder enum. --- format/FlightSQL.proto | 80 ++++++++++++------- .../apache/arrow/flight/TestFlightSQL.java | 4 +- .../arrow/flight/sql/FlightSQLExample.java | 2 +- 3 files changed, 52 insertions(+), 34 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index a7b68cd6dc4..ea7681288c0 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -25,8 +25,9 @@ package arrow.flight.protocol.sql; * Wrap the result of a "GetSQLCapabilities" action. */ message ActionGetSQLCapabilitiesResult{ - string identifierQuoteString = 1; - bool supportsExpressionsInOrderBy = 2; + map custom_fields = 1; + string identifier_quote_string = 2; + bool supports_expressions_in_order_by = 3; // TODO add more capabilities. } @@ -37,17 +38,16 @@ message ActionGetSQLCapabilitiesResult{ */ message ActionGetCatalogsRequest { /* - * True will ensure results are ordered alphabetically. - * False will not enforce ordering. + * Specifies the order of result values. */ - bool orderResultsAlphabetically = 1; + ResultsOrder order = 1; } /* * Wrap the result of a "GetCatalogs" action. */ message ActionGetCatalogsResult { - repeated string catalogNames = 1; + repeated string catalog = 1; } /* @@ -57,10 +57,11 @@ message ActionGetCatalogsResult { */ message ActionGetSchemasRequest { /* - * True will ensure results are ordered alphabetically. - * False will not enforce ordering. - */ - bool orderResultsAlphabetically = 1; + * Specifies the order of result values with prescendence: + * - catalog + * - schema + */ + ResultsOrder order = 1; /* * Specifies the Catalog to search for schemas. @@ -68,7 +69,7 @@ message ActionGetSchemasRequest { string catalog = 2; // Specifies a filter pattern for schemas to search for. - string schemaFilterPattern = 3; + string schema_filter_pattern = 3; } /* @@ -86,25 +87,28 @@ message ActionGetSchemasResult { */ message ActionGetTablesRequest { /* - * True will ensure results are ordered alphabetically. - * False will not enforce ordering. - */ - bool orderResultsAlphabetically = 1; + * Specifies the order of result values with prescendence: + * - catalog + * - schema + * - table_type + * - table + */ + ResultsOrder order = 1; // Specifies the Catalog to search for schemas. string catalog = 2; // Specifies a filter pattern for schemas to search for. - string schemaFilterPattern = 3; + string schema_filter_pattern = 3; // Specifies a filter pattern for tables to search for. - string tableNameFilterPattern = 4; + string table_name_filter_pattern = 4; // Specifies a filter of table types which must match. - repeated string tableTypes = 5; + repeated string table_types = 5; // Specifies if the schema should be returned for found tables. - bool includeSchema = 6; + bool include_schema = 6; } /* @@ -114,20 +118,20 @@ message ActionGetTablesResult { string catalog = 1; string schema = 2; string table = 3; - string tableType = 4; + string table_type = 4; /* * Schema of the dataset as described in Schema.fbs::Schema, * Null if includeSchema on request is false. */ - bytes schemaMetadata = 5; + bytes arrow_metadata = 5; } /* * Wrap the result of a "GetTableTypes" action. */ message ActionGetTableTypesResult { - string tableType = 1; + string table_type = 1; } // SQL Execution Action Messages @@ -148,15 +152,15 @@ message ActionGetPreparedStatementRequest { message ActionGetPreparedStatementResult { // Opaque handle for the prepared statement on the server. - bytes preparedStatementHandle = 1; + bytes prepared_statement_handle = 1; // If a result set generating query was provided, datasetSchema contains the // schema of the dataset as described in Schema.fbs::Schema. - bytes datasetSchema = 2; + bytes dataset_schema = 2; // If the query provided contained parameters, parameterSchema contains the // Schema of the expected parameters as described in Schema.fbs::Schema. - bytes parameterSchema = 3; + bytes parameter_schema = 3; } /* @@ -166,7 +170,7 @@ message ActionGetPreparedStatementResult { */ message ActionClosePreparedStatementRequest { // Opaque handle for the prepared statement on the server. - string preparedStatementHandle = 1; + string prepared_statement_handle = 1; } @@ -191,9 +195,9 @@ message CommandStatementQuery { */ message CommandPreparedStatementQuery { // Unique identifier for the instance of the prepared statement to execute. - bytes clientExecutionHandle = 1; + bytes client_execution_handle = 1; // Opaque handle for the prepared statement on the server. - bytes preparedStatementHandle = 2; + bytes prepared_statement_handle = 2; } /* @@ -213,9 +217,9 @@ message CommandStatementUpdate { */ message CommandPreparedStatementUpdate { // Unique identifier for the instance of the prepared statement to execute. - bytes clientExecutionHandle = 1; + bytes client_execution_handle = 1; // Opaque handle for the prepared statement on the server. - bytes preparedStatementHandle = 2; + bytes prepared_statement_handle = 2; } /* @@ -226,5 +230,19 @@ message CommandPreparedStatementUpdate { message DoPutUpdateResult { // The number of records updated. A return value of -1 represents // an unknown updated record count. - int64 recordCount = 1; + int64 record_count = 1; } + +enum ResultsOrder { + // Protobuf pattern, not used. + UNKNOWN = 0; + + // No ordering enforcement. + UNORDERED = 1; + + // Order results by ascending value order. + ASCENDING = 2; + + // Order results by descending value order. + DESCENDING = 3; +} \ No newline at end of file diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java index b775737decc..fd393472acf 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java @@ -99,7 +99,7 @@ public void testGetTables() throws Exception { .setSchema("APP") .setTable("INTTABLE") .setTableType("TABLE") - .setSchemaMetadata(ByteString.copyFrom(SCHEMA_INT_TABLE.toByteArray())) + .setArrowMetadata(ByteString.copyFrom(SCHEMA_INT_TABLE.toByteArray())) .build(); // Act @@ -125,7 +125,7 @@ public void testGetTablesWithFlightSQLClientUtils() throws Exception { .setSchema("APP") .setTable("INTTABLE") .setTableType("TABLE") - .setSchemaMetadata(ByteString.copyFrom(SCHEMA_INT_TABLE.toByteArray())) + .setArrowMetadata(ByteString.copyFrom(SCHEMA_INT_TABLE.toByteArray())) .build(); // Act diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java index b54621fa21f..99b1a9835c3 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java @@ -188,7 +188,7 @@ private Result getTableResult(final ResultSet tables, boolean includeSchema) thr if (includeSchema) { final Schema pojoSchema = buildSchema(catalog, schema, table); - builder.setSchemaMetadata(ByteString.copyFrom(pojoSchema.toByteArray())); + builder.setArrowMetadata(ByteString.copyFrom(pojoSchema.toByteArray())); } return new Result(Any.pack(builder.build()).toByteArray()); From bbf84696ca9b2007c3c270732a999c1ffafac23d Mon Sep 17 00:00:00 2001 From: Ryan Nicholson Date: Tue, 1 Sep 2020 17:44:18 -0700 Subject: [PATCH 04/16] Alter GetSQLCapabilies to GetSQLInfo pattern Alter GetSQLCapabilities to model after Hive Thrift's TCLIService. --- format/FlightSQL.proto | 55 +++++++++++++------ .../arrow/flight/sql/FlightSQLProducer.java | 4 +- .../arrow/flight/sql/FlightSQLUtils.java | 26 ++++----- 3 files changed, 54 insertions(+), 31 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index ea7681288c0..41cb7e19228 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -22,13 +22,34 @@ option java_package = "org.apache.arrow.flight.sql.impl"; package arrow.flight.protocol.sql; /* - * Wrap the result of a "GetSQLCapabilities" action. + * Wrap the result of a "GetSQLInfo" action. */ -message ActionGetSQLCapabilitiesResult{ - map custom_fields = 1; - string identifier_quote_string = 2; - bool supports_expressions_in_order_by = 3; - // TODO add more capabilities. +message ActionGetSQLInfoResult { + /* + * Values are based on Apache Hive's Thrift Service and + * ODBC's CLIGetInfo() function. Possible types include: + * - CLI_IDENTIFIER_QUOTE_CHAR + * - CLI_ORDER_BY_COLUMNS_IN_SELECT + * - TODO add more info types. + */ + map flight_sql_info = 1; + + // Implementation specific custom SQLInfo values. + map custom_info = 2; +} + +/* + * Wrapper for values returned in ActionGetSQLInfoResult. + */ +message TGetSQLInfoValue { + oneof value { + string stringValue = 1; + int32 integerValue = 2; + int32 integerBitmask = 3; + int32 integerFlag = 4; + bytes binaryValue = 5; + int64 longValue = 6; + } } /* @@ -233,16 +254,18 @@ message DoPutUpdateResult { int64 record_count = 1; } -enum ResultsOrder { - // Protobuf pattern, not used. - UNKNOWN = 0; +message ResultsOrder { + enum orders { + // Protobuf pattern, not used. + UNKNOWN = 0; - // No ordering enforcement. - UNORDERED = 1; + // No ordering enforcement. + UNORDERED = 1; - // Order results by ascending value order. - ASCENDING = 2; + // Order results by ascending value order. + ASCENDING = 2; - // Order results by descending value order. - DESCENDING = 3; -} \ No newline at end of file + // Order results by descending value order. + DESCENDING = 3; + } +} diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java index 5effd82893a..a2db6d760a5 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java @@ -22,7 +22,7 @@ import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETCATALOGS; import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETPREPAREDSTATEMENT; import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETSCHEMAS; -import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETSQLCAPABILITIES; +import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETSQLINFO; import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETTABLES; import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETTABLETYPES; @@ -181,7 +181,7 @@ public abstract Runnable acceptPutPreparedStatementQuery(CommandPreparedStatemen @Override public void doAction(CallContext context, Action action, StreamListener listener) { - if (action.getType().equals(FLIGHT_SQL_GETSQLCAPABILITIES.getType())) { + if (action.getType().equals(FLIGHT_SQL_GETSQLINFO.getType())) { getSqlCapabilities(context, listener); } else if (action.getType().equals(FLIGHT_SQL_GETCATALOGS.getType())) { diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java index 9e77699f4c4..7bd977de0df 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java @@ -43,43 +43,43 @@ public final class FlightSQLUtils { private static final boolean IS_SIGNED_FALSE = false; private static final boolean IS_SIGNED_TRUE = true; - public static final ActionType FLIGHT_SQL_GETSQLCAPABILITIES = new ActionType("GetSQLCapabilities", + public static final ActionType FLIGHT_SQL_GETSQLINFO = new ActionType("GetSQLINFO", "Retrieves details of SQL capabilities of the Flight server. \n" + "Request Message: N/A\n" + - "Response Message: SQLCapabilitiesResult"); + "Response Message: ActionGetSQLInfoResult"); public static final ActionType FLIGHT_SQL_GETCATALOGS = new ActionType("GetCatalogs", "Retrieves a list of all catalogs available on the server. \n" + - "Request Message: GetCatalogsRequest\n" + - "Response Message: GetCatalogsResult"); + "Request Message: ActionGetCatalogsRequest\n" + + "Response Message: ActionGetCatalogsResult"); public static final ActionType FLIGHT_SQL_GETSCHEMAS = new ActionType("GetSchemas", "Retrieves a list of schemas available on the server. \n" + - "Request Message: GetSchemasRequest\n" + - "Response Message: GetSchemasResult"); + "Request Message: ActionGetSchemasRequest\n" + + "Response Message: ActionGetSchemasResult"); public static final ActionType FLIGHT_SQL_GETTABLES = new ActionType("GetTables", "Retrieves a list of tables available on the server. \n" + - "Request Message: GetTablesRequest\n" + - "Response Message: GetTablesResult"); + "Request Message: ActionGetTablesRequest\n" + + "Response Message: ActionGetTablesResult"); public static final ActionType FLIGHT_SQL_GETTABLETYPES = new ActionType("GetTableTypes", "Retrieves a list of table types available on the server. \n" + "Request Message: N/A\n" + - "Response Message: GetTableTypesResult"); + "Response Message: ActionGetTableTypesResult"); public static final ActionType FLIGHT_SQL_GETPREPAREDSTATEMENT = new ActionType("GetPreparedStatement", "Creates a reusable prepared statement resource on the server. \n" + - "Request Message: ActionRequestGetPreparedStatement\n" + - "Response Message: ActionResponseGetPreparedStatement"); + "Request Message: ActionGetPreparedStatementRequest\n" + + "Response Message: ActionGetPreparedStatementResult"); public static final ActionType FLIGHT_SQL_CLOSEPREPAREDSTATEMENT = new ActionType("ClosePreparedStatement", "Closes a reusable prepared statement resource on the server. \n" + - "Request Message: ActionRequestClosePreparedStatement\n" + + "Request Message: ActionClosePreparedStatementRequest\n" + "Response Message: N/A"); public static final List FLIGHT_SQL_ACTIONS = ImmutableList.of( - FLIGHT_SQL_GETSQLCAPABILITIES, + FLIGHT_SQL_GETSQLINFO, FLIGHT_SQL_GETCATALOGS, FLIGHT_SQL_GETSCHEMAS, FLIGHT_SQL_GETTABLES, From c3575e46008ad4b17cb200dacc68a81e4ec86f71 Mon Sep 17 00:00:00 2001 From: Ryan Nicholson Date: Tue, 1 Sep 2020 17:45:49 -0700 Subject: [PATCH 05/16] Update FlightSQL.proto Update new message to snake_case. --- format/FlightSQL.proto | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index 41cb7e19228..92688e1d73a 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -43,12 +43,12 @@ message ActionGetSQLInfoResult { */ message TGetSQLInfoValue { oneof value { - string stringValue = 1; - int32 integerValue = 2; - int32 integerBitmask = 3; - int32 integerFlag = 4; - bytes binaryValue = 5; - int64 longValue = 6; + string string_value = 1; + int32 integer_value = 2; + int32 integer_bitmask = 3; + int32 integer_flag = 4; + bytes binary_value = 5; + int64 long_value = 6; } } From 0c91122a6a574f82819a6833e1fe5b9c6659a8fd Mon Sep 17 00:00:00 2001 From: Ryan Nicholson Date: Tue, 1 Sep 2020 18:12:15 -0700 Subject: [PATCH 06/16] Update FlightSQL.proto Update comments to reflect current field names. --- format/FlightSQL.proto | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index 92688e1d73a..8158d440324 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -175,11 +175,11 @@ message ActionGetPreparedStatementResult { // Opaque handle for the prepared statement on the server. bytes prepared_statement_handle = 1; - // If a result set generating query was provided, datasetSchema contains the + // If a result set generating query was provided, dataset_schema contains the // schema of the dataset as described in Schema.fbs::Schema. bytes dataset_schema = 2; - // If the query provided contained parameters, parameterSchema contains the + // If the query provided contained parameters, parameter_schema contains the // Schema of the expected parameters as described in Schema.fbs::Schema. bytes parameter_schema = 3; } From fb6b8224ebe3e704060f00e9fa159a48f05be85d Mon Sep 17 00:00:00 2001 From: Ryan Nicholson Date: Wed, 2 Sep 2020 08:32:31 -0700 Subject: [PATCH 07/16] Rename "getSQLCapabilities" to "getSQLInfo" in FlightSQLProducer --- .../org/apache/arrow/flight/sql/FlightSQLProducer.java | 8 ++++---- .../org/apache/arrow/flight/sql/FlightSQLExample.java | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java index a2db6d760a5..ce7ca7265f3 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java @@ -182,7 +182,7 @@ public abstract Runnable acceptPutPreparedStatementQuery(CommandPreparedStatemen public void doAction(CallContext context, Action action, StreamListener listener) { if (action.getType().equals(FLIGHT_SQL_GETSQLINFO.getType())) { - getSqlCapabilities(context, listener); + getSqlInfo(context, listener); } else if (action.getType().equals(FLIGHT_SQL_GETCATALOGS.getType())) { final ActionGetCatalogsRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), @@ -215,13 +215,13 @@ public void doAction(CallContext context, Action action, StreamListener } /** - * Returns the SQL Capabilities of the server by returning a - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSQLCapabilitiesResult} in a {@link Result}. + * Returns the SQL Info of the server by returning a + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSQLInfoResult} in a {@link Result}. * * @param context Per-call context. * @param listener A stream of responses. */ - public abstract void getSqlCapabilities(CallContext context, StreamListener listener); + public abstract void getSqlInfo(CallContext context, StreamListener listener); /** * Returns the available catalogs by returning a stream of diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java index 99b1a9835c3..f7326465b8c 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java @@ -539,7 +539,7 @@ public void doExchange(CallContext context, FlightStream reader, ServerStreamLis } @Override - public void getSqlCapabilities(CallContext context, StreamListener listener) { + public void getSqlInfo(CallContext context, StreamListener listener) { // TODO - build example implementation throw Status.UNIMPLEMENTED.asRuntimeException(); } From 311dec32aa8ccb9abae190265349c1cd16452dec Mon Sep 17 00:00:00 2001 From: tifflhl Date: Thu, 21 Jan 2021 18:15:59 -0800 Subject: [PATCH 08/16] Address code review comments (1) - Address code review comments from https://github.com/ryannicholson/arrow/pull/2 --- format/FlightSQL.proto | 81 ++++- .../arrow/flight/sql/FlightSQLProducer.java | 306 ++++++++++-------- 2 files changed, 245 insertions(+), 142 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index 8158d440324..9a8c72a9c7a 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -22,15 +22,60 @@ option java_package = "org.apache.arrow.flight.sql.impl"; package arrow.flight.protocol.sql; /* - * Wrap the result of a "GetSQLInfo" action. + * Wrapper for the result of a "GetSQLInfo" action. */ message ActionGetSQLInfoResult { /* - * Values are based on Apache Hive's Thrift Service and - * ODBC's CLIGetInfo() function. Possible types include: - * - CLI_IDENTIFIER_QUOTE_CHAR - * - CLI_ORDER_BY_COLUMNS_IN_SELECT - * - TODO add more info types. + * Values are modelled after ODBC's SQLGetInfo() function. This information is intended to provide + * Flight SQL clients with basic, SQL syntax and SQL functions related information. + * More information types can be added in future releases. + * E.g. more SQL syntax support types, scalar functions support, type conversion support etc. + * + * Initially, Flight SQL will support the following information types: + * - Server Information + * - + * + * 1. Server Information: Provides basic information about the Flight SQL Server. + * + * The name of the Flight SQL Server. + * FLIGHT_SQL_SERVER_NAME + * + * The native version of the Flight SQL Server. + * FLIGHT_SQL_SERVER_VERSION + * + * The Arrow version of the Flight SQL Server. + * FLIGHT_SQL_SERVER_ARROW_VERSION + * + * Indicates whether the Flight SQL Server is read only. + * FLIGHT_SQL_SERVER_READ_ONLY + * + * Indicates whether the Flight SQL Server supports both read and write. + * FLIGHT_SQL_SERVER_READ_WRITE + * + * 2. SQL Syntax Information: provides information about SQL syntax supported by the Flight SQL Server. + * + * Indicates whether the Flight SQL Server supports CREATE and DROP of catalogs. + * In a SQL environment, a catalog is a collection of schemas. + * SQL_DDL_CATALOG + * + * Indicates whether the Flight SQL Server supports CREATE and DROP of schemas. + * In a SQL environment, a catalog is a collection of tables, views, indexes etc. + * SQL_DDL_SCHEMA + * + * Indicates whether the Flight SQL Server supports CREATE and DROP of tables. + * In a SQL environment, a table is a collection of rows of information. Each row of information + * may have one or more columns of data. + * SQL_DDL_TABLE + * + * Indicates the case sensitivity of catalog, table and schema names. + * SQL_IDENTIFIER_CASE + * + * Indicates the supported character(s) used to surround a delimited identifier. + * SQL_IDENTIFIER_QUOTE_CHAR + * + * Indicates case sensitivity of quoted identifiers. + * SQL_QUOTED_IDENTIFIER_CASE + * */ map flight_sql_info = 1; @@ -41,7 +86,7 @@ message ActionGetSQLInfoResult { /* * Wrapper for values returned in ActionGetSQLInfoResult. */ -message TGetSQLInfoValue { +message GetSQLInfoValue { oneof value { string string_value = 1; int32 integer_value = 2; @@ -78,7 +123,7 @@ message ActionGetCatalogsResult { */ message ActionGetSchemasRequest { /* - * Specifies the order of result values with prescendence: + * Specifies the order of result values with precedence: * - catalog * - schema */ @@ -86,9 +131,11 @@ message ActionGetSchemasRequest { /* * Specifies the Catalog to search for schemas. + * If omitted, then schemas for all catalogs are searched. */ string catalog = 2; + // TODO: Clarify what kind of filter pattern - regex? // Specifies a filter pattern for schemas to search for. string schema_filter_pattern = 3; } @@ -119,9 +166,11 @@ message ActionGetTablesRequest { // Specifies the Catalog to search for schemas. string catalog = 2; + // TODO: Clarify what kind of filter pattern - regex? // Specifies a filter pattern for schemas to search for. string schema_filter_pattern = 3; + // TODO: Clarify what kind of filter pattern - regex? // Specifies a filter pattern for tables to search for. string table_name_filter_pattern = 4; @@ -142,8 +191,8 @@ message ActionGetTablesResult { string table_type = 4; /* - * Schema of the dataset as described in Schema.fbs::Schema, - * Null if includeSchema on request is false. + * Schema of the dataset as described in Schema.fbs::Schema, it is serialized as an IPC message. + * Null if include_schema on request is false. */ bytes arrow_metadata = 5; } @@ -152,6 +201,9 @@ message ActionGetTablesResult { * Wrap the result of a "GetTableTypes" action. */ message ActionGetTableTypesResult { + /* + * Indicates the type of the table. E.g. table (regular data table) , view, system table etc. + */ string table_type = 1; } @@ -159,11 +211,10 @@ message ActionGetTableTypesResult { /* * Request message for the "GetPreparedStatement" action on a - * Flight SQL enabled backend. - * Requests a list of tables available in the server. + * Flight SQL enabled backend. */ message ActionGetPreparedStatementRequest { - // The SQL syntax. + // The valid SQL string to get a prepared statement for. string query = 1; } @@ -176,7 +227,7 @@ message ActionGetPreparedStatementResult { bytes prepared_statement_handle = 1; // If a result set generating query was provided, dataset_schema contains the - // schema of the dataset as described in Schema.fbs::Schema. + // schema of the dataset as described in Schema.fbs::Schema, it is serialized as an IPC message. bytes dataset_schema = 2; // If the query provided contained parameters, parameter_schema contains the @@ -211,6 +262,8 @@ message CommandStatementQuery { /* * Represents an instance of executing a prepared statement. Used in the * command member of FlightDescriptor for the following RPC calls: + * TODO: Is the idea that a Put with parameter values would execute multiple bound versions of the prepared statement + * TODO: (one for each row)? Seems like that will work ok for Insert statements, but what about other kinds of prepared statements? * - DoPut: bind parameter values. * - GetFlightInfo: execute the prepared statement instance. */ diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java index ce7ca7265f3..3802295b41d 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java @@ -55,7 +55,15 @@ * API to Implement an Arrow Flight SQL producer. */ public abstract class FlightSQLProducer implements FlightProducer, AutoCloseable { - + /** + * Depending on the provided command, method either: + * 1. Return information about a SQL query, or + * 2. Return information about a prepared statement. In this case, parameters binding is allowed. + * + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return information about the given SQL query, or the given prepared statement. + */ @Override public FlightInfo getFlightInfo(CallContext context, FlightDescriptor descriptor) { final Any command = FlightSQLUtils.parseOrThrow(descriptor.getCommand()); @@ -73,27 +81,12 @@ public FlightInfo getFlightInfo(CallContext context, FlightDescriptor descriptor } /** - * Get information about a particular SQL query based data stream. - * - * @param command The sql command to generate the data stream. - * @param context Per-call context. - * @param descriptor The descriptor identifying the data stream. - * @return Metadata about the stream. - */ - public abstract FlightInfo getFlightInfoStatement(CommandStatementQuery command, FlightDescriptor descriptor, - CallContext context); - - /** - * Get information about a particular prepared statement data stream. + * Returns the schema of the result produced by the SQL query. * - * @param command The prepared statement to generate the data stream. - * @param context Per-call context. + * @param context Per-call context. * @param descriptor The descriptor identifying the data stream. - * @return Metadata about the stream. + * @return the result set schema. */ - public abstract FlightInfo getFlightInfoPreparedStatement(CommandPreparedStatementQuery command, - FlightDescriptor descriptor, CallContext context); - @Override public SchemaResult getSchema(CallContext context, FlightDescriptor descriptor) { final Any command = FlightSQLUtils.parseOrThrow(descriptor.getCommand()); @@ -107,16 +100,49 @@ public SchemaResult getSchema(CallContext context, FlightDescriptor descriptor) } /** - * Get schema about a particular SQL query based data stream. + * Depending on the provided command, method either: + * 1. Return data for a stream produced by executing the provided SQL query, or + * 2. Return data for a prepared statement. In this case, parameters binding is allowed. * - * @param command The sql command to generate the data stream. - * @param context Per-call context. - * @param descriptor The descriptor identifying the data stream. - * @return Schema for the stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. */ - public abstract SchemaResult getSchemaStatement(CommandStatementQuery command, FlightDescriptor descriptor, - CallContext context); + @Override + public void getStream(CallContext context, Ticket ticket, ServerStreamListener listener) { + final Any command; + try { + command = Any.parseFrom(ticket.getBytes()); + } catch (InvalidProtocolBufferException e) { + listener.error(e); + return; + } + + if (command.is(CommandStatementQuery.class)) { + getStreamStatement(FlightSQLUtils.unpackOrThrow(command, CommandStatementQuery.class), + context, ticket, listener); + + } else if (command.is(CommandPreparedStatementQuery.class)) { + getStreamPreparedStatement(FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), + context, ticket, listener); + } + + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + + /** + * Depending on the provided command, method either: + * 1. Execute provided SQL query as an update statement, or + * 2. Execute provided update SQL query prepared statement. In this case, parameters binding + * is allowed, or + * 3. Binds parameters to the provided prepared statement. + * + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The data stream listener for update result acknowledgement. + * @return a Runnable to process the stream. + */ @Override public Runnable acceptPut(CallContext context, FlightStream flightStream, StreamListener ackStream) { final Any command = FlightSQLUtils.parseOrThrow(flightStream.getDescriptor().getCommand()); @@ -141,43 +167,24 @@ public Runnable acceptPut(CallContext context, FlightStream flightStream, Stream } /** - * Accept uploaded data for a particular SQL query based data stream. PutResults must be in the form of a - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.DoPutUpdateResult}. - * - * @param command The sql command to generate the data stream. - * @param context Per-call context. - * @param flightStream The data stream being uploaded. - * @param ackStream The result data stream. - * @return A runnable to process the stream. - */ - public abstract Runnable acceptPutStatement(CommandStatementUpdate command, CallContext context, - FlightStream flightStream, StreamListener ackStream); - - /** - * Accept uploaded data for a particular prepared statement data stream. PutResults must be in the form of a - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.DoPutUpdateResult}. + * Lists all available Flight SQL actions. * - * @param command The prepared statement to generate the data stream. - * @param context Per-call context. - * @param flightStream The data stream being uploaded. - * @param ackStream The result data stream. - * @return A runnable to process the stream. + * @param context Per-call context. + * @param listener An interface for sending data back to the client. */ - public abstract Runnable acceptPutPreparedStatementUpdate(CommandPreparedStatementUpdate command, - CallContext context, FlightStream flightStream, StreamListener ackStream); + @Override + public void listActions(CallContext context, StreamListener listener) { + FLIGHT_SQL_ACTIONS.forEach(action -> listener.onNext(action)); + listener.onCompleted(); + } /** - * Accept uploaded parameter values for a particular prepared statement query. + * Performs the requested Flight SQL action. * - * @param command The prepared statement the parameter values will bind to. - * @param context Per-call context. - * @param flightStream The data stream being uploaded. - * @param ackStream The result data stream. - * @return A runnable to process the stream. + * @param context Per-call context. + * @param action Client-supplied parameters. + * @param listener A stream of responses. */ - public abstract Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery command, - CallContext context, FlightStream flightStream, StreamListener ackStream); - @Override public void doAction(CallContext context, Action action, StreamListener listener) { @@ -215,125 +222,168 @@ public void doAction(CallContext context, Action action, StreamListener } /** - * Returns the SQL Info of the server by returning a - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSQLInfoResult} in a {@link Result}. + * Creates a prepared statement on the server and returns a handle and metadata for in a + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult} object in a {@link Result} + * object. * + * @param request The sql command to generate the prepared statement. * @param context Per-call context. * @param listener A stream of responses. */ - public abstract void getSqlInfo(CallContext context, StreamListener listener); + public abstract void getPreparedStatement(ActionGetPreparedStatementRequest request, CallContext context, + StreamListener listener); /** - * Returns the available catalogs by returning a stream of - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetCatalogsResult} objects in {@link Result} objects. + * Closes a prepared statement on the server. No result is expected. * - * @param request request filter parameters. + * @param request The sql command to generate the prepared statement. * @param context Per-call context. * @param listener A stream of responses. */ - public abstract void getCatalogs(ActionGetCatalogsRequest request, CallContext context, - StreamListener listener); + public abstract void closePreparedStatement(ActionClosePreparedStatementRequest request, CallContext context, + StreamListener listener); /** - * Returns the available schemas by returning a stream of - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSchemasResult} objects in {@link Result} objects. + * Gets information about a particular SQL query based data stream. * - * @param request request filter parameters. + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoStatement(CommandStatementQuery command, FlightDescriptor descriptor, + CallContext context); + + /** + * Gets information about a particular prepared statement data stream. + * + * @param command The prepared statement to generate the data stream. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoPreparedStatement(CommandPreparedStatementQuery command, + FlightDescriptor descriptor, CallContext context); + + + /** + * Gets schema about a particular SQL query based data stream. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Schema for the stream. + */ + public abstract SchemaResult getSchemaStatement(CommandStatementQuery command, FlightDescriptor descriptor, + CallContext context); + + /** + * Returns data for a SQL query based data stream. + * + * @param command The sql command to generate the data stream. * @param context Per-call context. - * @param listener A stream of responses. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. */ - public abstract void getSchemas(ActionGetSchemasRequest request, CallContext context, - StreamListener listener); + public abstract void getStreamStatement(CommandStatementQuery command, CallContext context, Ticket ticket, + ServerStreamListener listener); /** - * Returns the available table types by returning a stream of - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTableTypesResult} objects in {@link Result} objects. + * Returns data for a particular prepared statement query instance. * + * @param command The prepared statement to generate the data stream. * @param context Per-call context. - * @param listener A stream of responses. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. */ - public abstract void getTableTypes(CallContext context, StreamListener listener); + public abstract void getStreamPreparedStatement(CommandPreparedStatementQuery command, CallContext context, + Ticket ticket, ServerStreamListener listener); /** - * Returns the available tables by returning a stream of - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult} objects in {@link Result} objects. + * Accepts uploaded data for a particular SQL query based data stream. PutResults must be in the form of a + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.DoPutUpdateResult}. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The result data stream. + * @return A runnable to process the stream. + */ + public abstract Runnable acceptPutStatement(CommandStatementUpdate command, CallContext context, + FlightStream flightStream, StreamListener ackStream); + + /** + * Accepts uploaded data for a particular prepared statement data stream. PutResults must be in the form of a + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.DoPutUpdateResult}. + * + * @param command The prepared statement to generate the data stream. + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The result data stream. + * @return A runnable to process the stream. + */ + public abstract Runnable acceptPutPreparedStatementUpdate(CommandPreparedStatementUpdate command, + CallContext context, FlightStream flightStream, StreamListener ackStream); + + /** + * Accepts uploaded parameter values for a particular prepared statement query. + * + * @param command The prepared statement the parameter values will bind to. + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The result data stream. + * @return A runnable to process the stream. + */ + public abstract Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery command, + CallContext context, FlightStream flightStream, StreamListener ackStream); + + /** + * Returns the SQL Info of the server by returning a + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSQLInfoResult} in a {@link Result}. * - * @param request request filter parameters. * @param context Per-call context. * @param listener A stream of responses. */ - public abstract void getTables(ActionGetTablesRequest request, CallContext context, StreamListener listener); + public abstract void getSqlInfo(CallContext context, StreamListener listener); /** - * Creates a prepared statement on the server and returns a handle and metadata for in a - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult} object in a {@link Result} - * object. + * Returns the available catalogs by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetCatalogsResult} objects in {@link Result} objects. * - * @param request The sql command to generate the prepared statement. + * @param request request filter parameters. * @param context Per-call context. * @param listener A stream of responses. */ - public abstract void getPreparedStatement(ActionGetPreparedStatementRequest request, CallContext context, + public abstract void getCatalogs(ActionGetCatalogsRequest request, CallContext context, StreamListener listener); /** - * Closes a prepared statement on the server. No result is expected. + * Returns the available schemas by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSchemasResult} objects in {@link Result} objects. * - * @param request The sql command to generate the prepared statement. + * @param request request filter parameters. * @param context Per-call context. * @param listener A stream of responses. */ - public abstract void closePreparedStatement(ActionClosePreparedStatementRequest request, CallContext context, + public abstract void getSchemas(ActionGetSchemasRequest request, CallContext context, StreamListener listener); - @Override - public void listActions(CallContext context, StreamListener listener) { - FLIGHT_SQL_ACTIONS.forEach(action -> listener.onNext(action)); - listener.onCompleted(); - } - - @Override - public void getStream(CallContext context, Ticket ticket, ServerStreamListener listener) { - final Any command; - - try { - command = Any.parseFrom(ticket.getBytes()); - } catch (InvalidProtocolBufferException e) { - listener.error(e); - return; - } - - if (command.is(CommandStatementQuery.class)) { - getStreamStatement(FlightSQLUtils.unpackOrThrow(command, CommandStatementQuery.class), - context, ticket, listener); - - } else if (command.is(CommandPreparedStatementQuery.class)) { - getStreamPreparedStatement(FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), - context, ticket, listener); - } - - throw Status.INVALID_ARGUMENT.asRuntimeException(); - } - /** - * Return data for a SQL query based data stream. + * Returns the available tables by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult} objects in {@link Result} objects. * - * @param command The sql command to generate the data stream. + * @param request request filter parameters. * @param context Per-call context. - * @param ticket The application-defined ticket identifying this stream. - * @param listener An interface for sending data back to the client. + * @param listener A stream of responses. */ - public abstract void getStreamStatement(CommandStatementQuery command, CallContext context, Ticket ticket, - ServerStreamListener listener); + public abstract void getTables(ActionGetTablesRequest request, CallContext context, StreamListener listener); /** - * Return data for a particular prepared statement query instance. + * Returns the available table types by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTableTypesResult} objects in {@link Result} objects. * - * @param command The prepared statement to generate the data stream. * @param context Per-call context. - * @param ticket The application-defined ticket identifying this stream. - * @param listener An interface for sending data back to the client. + * @param listener A stream of responses. */ - public abstract void getStreamPreparedStatement(CommandPreparedStatementQuery command, CallContext context, - Ticket ticket, ServerStreamListener listener); + public abstract void getTableTypes(CallContext context, StreamListener listener); } From a7d1658025bf9947696d34553bbc07dbdbc80a79 Mon Sep 17 00:00:00 2001 From: tifflhl Date: Fri, 29 Jan 2021 16:58:16 -0800 Subject: [PATCH 09/16] Address code review comments (2) --- format/FlightSQL.proto | 29 +++++++++++++++++++++-------- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index 9a8c72a9c7a..b02e18c3c4b 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -77,10 +77,10 @@ message ActionGetSQLInfoResult { * SQL_QUOTED_IDENTIFIER_CASE * */ - map flight_sql_info = 1; + map flight_sql_info = 1; // Implementation specific custom SQLInfo values. - map custom_info = 2; + map custom_info = 2; } /* @@ -135,8 +135,13 @@ message ActionGetSchemasRequest { */ string catalog = 2; - // TODO: Clarify what kind of filter pattern - regex? - // Specifies a filter pattern for schemas to search for. + /* + * Specifies a filter pattern for schemas to search for. + * When no schema_filter_pattern is provided, the pattern will not be used to narrow the search. + * In the pattern string, two special characters can be used to denote matching rules: + * - "%" means to match any substring with 0 or more characters. + * - "_" means to match any one character. + */ string schema_filter_pattern = 3; } @@ -166,12 +171,20 @@ message ActionGetTablesRequest { // Specifies the Catalog to search for schemas. string catalog = 2; - // TODO: Clarify what kind of filter pattern - regex? - // Specifies a filter pattern for schemas to search for. + /* Specifies a filter pattern for schemas to search for. + * When no schema_filter_pattern is provided, the pattern will not be used to narrow the search. + * In the pattern string, two special characters can be used to denote matching rules: + * - "%" means to match any substring with 0 or more characters. + * - "_" means to match any one character. + */ string schema_filter_pattern = 3; - // TODO: Clarify what kind of filter pattern - regex? - // Specifies a filter pattern for tables to search for. + /* Specifies a filter pattern for tables to search for. + * When no schema_filter_pattern is provided, the pattern will not be used to narrow the search. + * In the pattern string, two special characters can be used to denote matching rules: + * - "%" means to match any substring with 0 or more characters. + * - "_" means to match any one character. + */ string table_name_filter_pattern = 4; // Specifies a filter of table types which must match. From 95c4d7168646824829d3f35361f38beeecd42dbf Mon Sep 17 00:00:00 2001 From: Kyle Porter Date: Tue, 29 Jun 2021 11:57:14 -0700 Subject: [PATCH 10/16] Update Flight proto to address CR comments. --- format/FlightSQL.proto | 222 +++++++++++++++++------------------------ 1 file changed, 90 insertions(+), 132 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index b02e18c3c4b..f2b3987e89a 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -22,18 +22,30 @@ option java_package = "org.apache.arrow.flight.sql.impl"; package arrow.flight.protocol.sql; /* - * Wrapper for the result of a "GetSQLInfo" action. + * Represents a metadata request. Used in the command member of FlightDescriptor + * for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the query. + * + * The returned schema will be: + * < + * info_name: utf8, + * value: dense_union + * > + * where there is one row per requested piece of metadata information. */ -message ActionGetSQLInfoResult { +message CommandGetSqlInfo { /* * Values are modelled after ODBC's SQLGetInfo() function. This information is intended to provide * Flight SQL clients with basic, SQL syntax and SQL functions related information. * More information types can be added in future releases. * E.g. more SQL syntax support types, scalar functions support, type conversion support etc. * + * // TODO: Flesh out the available set of metadata below. + * * Initially, Flight SQL will support the following information types: * - Server Information - * - + * - Syntax Information * * 1. Server Information: Provides basic information about the Flight SQL Server. * @@ -49,9 +61,6 @@ message ActionGetSQLInfoResult { * Indicates whether the Flight SQL Server is read only. * FLIGHT_SQL_SERVER_READ_ONLY * - * Indicates whether the Flight SQL Server supports both read and write. - * FLIGHT_SQL_SERVER_READ_WRITE - * * 2. SQL Syntax Information: provides information about SQL syntax supported by the Flight SQL Server. * * Indicates whether the Flight SQL Server supports CREATE and DROP of catalogs. @@ -77,63 +86,46 @@ message ActionGetSQLInfoResult { * SQL_QUOTED_IDENTIFIER_CASE * */ - map flight_sql_info = 1; - - // Implementation specific custom SQLInfo values. - map custom_info = 2; -} + repeated string info = 1; -/* - * Wrapper for values returned in ActionGetSQLInfoResult. - */ -message GetSQLInfoValue { - oneof value { - string string_value = 1; - int32 integer_value = 2; - int32 integer_bitmask = 3; - int32 integer_flag = 4; - bytes binary_value = 5; - int64 long_value = 6; - } -} - -/* - * Request message for the "GetCatalogs" action on a - * Flight SQL enabled backend. - * Requests a list of catalogs available in the server. - */ -message ActionGetCatalogsRequest { - /* - * Specifies the order of result values. - */ - ResultsOrder order = 1; + // Implementation specific custom SqlInfo values. + repeated string custom_info = 2; } /* - * Wrap the result of a "GetCatalogs" action. + Represents a request to retrieve the list of catalogs on a Flight SQL enabled backend. + * Used in the command member of FlightDescriptor for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the query. + * + * The returned schema will be: + * < + * catalog_name: utf8 + * > + * The returned data should be ordered by catalog_name. */ -message ActionGetCatalogsResult { - repeated string catalog = 1; +message CommandGetCatalogs { } /* - * Request message for the "GetSchemas" action on a - * Flight SQL enabled backend. - * Requests a list of schemas available in the server. + * Represents a request to retrieve the list of schemas on a Flight SQL enabled backend. + * Used in the command member of FlightDescriptor for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the query. + * + * The returned schema will be: + * < + * catalog_name: utf8, + * schema_name: utf8 + * > + * The returned data should be ordered by catalog_name, then schema_name. */ -message ActionGetSchemasRequest { - /* - * Specifies the order of result values with precedence: - * - catalog - * - schema - */ - ResultsOrder order = 1; - +message CommandGetSchemas { /* * Specifies the Catalog to search for schemas. - * If omitted, then schemas for all catalogs are searched. + * If omitted, then all catalogs are searched. */ - string catalog = 2; + string catalog = 1; /* * Specifies a filter pattern for schemas to search for. @@ -142,100 +134,84 @@ message ActionGetSchemasRequest { * - "%" means to match any substring with 0 or more characters. * - "_" means to match any one character. */ - string schema_filter_pattern = 3; -} - -/* - * Wrap the result of a "GetSchemas" action. - */ -message ActionGetSchemasResult { - string catalog = 1; - string schema = 2; + string schema_filter_pattern = 2; } /* - * Request message for the "GetTables" action on a - * Flight SQL enabled backend. - * Requests a list of tables available in the server. + * Represents a request to retrieve the list of tables, and optionally their schemas, on a Flight SQL enabled backend. + * Used in the command member of FlightDescriptor for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the query. + * + * The returned schema will be: + * < + * catalog_name: utf8, + * schema_name: utf8, + * table_name: utf8, + * table_type: utf8, + * table_schema: bytes + * > + * The returned data should be ordered by catalog_name, schema_name, table_name, then table_type. */ message ActionGetTablesRequest { /* - * Specifies the order of result values with prescendence: - * - catalog - * - schema - * - table_type - * - table + * Specifies the Catalog to search for schemas. + * If omitted, then all catalogs are searched. */ - ResultsOrder order = 1; - - // Specifies the Catalog to search for schemas. - string catalog = 2; + string catalog = 1; /* Specifies a filter pattern for schemas to search for. - * When no schema_filter_pattern is provided, the pattern will not be used to narrow the search. + * When no schema_filter_pattern is provided, all schemas matching other filters are searched. * In the pattern string, two special characters can be used to denote matching rules: * - "%" means to match any substring with 0 or more characters. * - "_" means to match any one character. */ - string schema_filter_pattern = 3; + string schema_filter_pattern = 2; /* Specifies a filter pattern for tables to search for. - * When no schema_filter_pattern is provided, the pattern will not be used to narrow the search. + * When no table_name_filter_pattern is provided, all tables matching other filters are searched. * In the pattern string, two special characters can be used to denote matching rules: * - "%" means to match any substring with 0 or more characters. * - "_" means to match any one character. */ - string table_name_filter_pattern = 4; + string table_name_filter_pattern = 3; // Specifies a filter of table types which must match. - repeated string table_types = 5; + repeated string table_types = 4; // Specifies if the schema should be returned for found tables. - bool include_schema = 6; -} - -/* - * Wrap the result of a "GetTables" action. - */ -message ActionGetTablesResult { - string catalog = 1; - string schema = 2; - string table = 3; - string table_type = 4; - - /* - * Schema of the dataset as described in Schema.fbs::Schema, it is serialized as an IPC message. - * Null if include_schema on request is false. - */ - bytes arrow_metadata = 5; + bool include_schema = 5; } /* - * Wrap the result of a "GetTableTypes" action. + * Represents a request to retrieve the list of table types on a Flight SQL enabled backend. + * Used in the command member of FlightDescriptor for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the query. + * + * The returned schema will be: + * < + * table_type: utf8 + * > + * The returned data should be ordered by table_type. */ -message ActionGetTableTypesResult { - /* - * Indicates the type of the table. E.g. table (regular data table) , view, system table etc. - */ - string table_type = 1; +message CommandGetTableTypes { } // SQL Execution Action Messages /* - * Request message for the "GetPreparedStatement" action on a - * Flight SQL enabled backend. + * Request message for the "GetPreparedStatement" action on a Flight SQL enabled backend. */ -message ActionGetPreparedStatementRequest { - // The valid SQL string to get a prepared statement for. +message ActionCreatePreparedStatementRequest { + // The valid SQL string to create a prepared statement for. string query = 1; } /* * Wrap the result of a "GetPreparedStatement" action. */ -message ActionGetPreparedStatementResult { - +message ActionCreatePreparedStatementResult { // Opaque handle for the prepared statement on the server. bytes prepared_statement_handle = 1; @@ -249,8 +225,7 @@ message ActionGetPreparedStatementResult { } /* - * Request message for the "ClosePreparedStatement" action on a - * Flight SQL enabled backend. + * Request message for the "ClosePreparedStatement" action on a Flight SQL enabled backend. * Closes server resources associated with the prepared statement handle. */ message ActionClosePreparedStatementRequest { @@ -273,24 +248,22 @@ message CommandStatementQuery { } /* - * Represents an instance of executing a prepared statement. Used in the - * command member of FlightDescriptor for the following RPC calls: - * TODO: Is the idea that a Put with parameter values would execute multiple bound versions of the prepared statement - * TODO: (one for each row)? Seems like that will work ok for Insert statements, but what about other kinds of prepared statements? + * Represents an instance of executing a prepared statement. Used in the command member of FlightDescriptor for + * the following RPC calls: * - DoPut: bind parameter values. * - GetFlightInfo: execute the prepared statement instance. */ message CommandPreparedStatementQuery { // Unique identifier for the instance of the prepared statement to execute. - bytes client_execution_handle = 1; + bytes client_execution_handle = 1; + // Opaque handle for the prepared statement on the server. bytes prepared_statement_handle = 2; } /* * Represents a SQL update query. Used in the command member of FlightDescriptor - * for the the RPC call DoPut to cause the server to execute the included - * SQL update. + * for the the RPC call DoPut to cause the server to execute the included SQL update. */ message CommandStatementUpdate { // The SQL syntax. @@ -304,7 +277,8 @@ message CommandStatementUpdate { */ message CommandPreparedStatementUpdate { // Unique identifier for the instance of the prepared statement to execute. - bytes client_execution_handle = 1; + bytes client_execution_handle = 1; + // Opaque handle for the prepared statement on the server. bytes prepared_statement_handle = 2; } @@ -319,19 +293,3 @@ message DoPutUpdateResult { // an unknown updated record count. int64 record_count = 1; } - -message ResultsOrder { - enum orders { - // Protobuf pattern, not used. - UNKNOWN = 0; - - // No ordering enforcement. - UNORDERED = 1; - - // Order results by ascending value order. - ASCENDING = 2; - - // Order results by descending value order. - DESCENDING = 3; - } -} From b1bd8956370e966b37749aa1e8ce56c314773d05 Mon Sep 17 00:00:00 2001 From: Kyle Porter Date: Tue, 29 Jun 2021 21:31:19 -0700 Subject: [PATCH 11/16] Address additional CR comments. --- format/FlightSQL.proto | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index f2b3987e89a..cad639bd8b8 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -55,7 +55,7 @@ message CommandGetSqlInfo { * The native version of the Flight SQL Server. * FLIGHT_SQL_SERVER_VERSION * - * The Arrow version of the Flight SQL Server. + * The Arrow format version of the Flight SQL Server. * FLIGHT_SQL_SERVER_ARROW_VERSION * * Indicates whether the Flight SQL Server is read only. @@ -85,6 +85,7 @@ message CommandGetSqlInfo { * Indicates case sensitivity of quoted identifiers. * SQL_QUOTED_IDENTIFIER_CASE * + * If omitted, then all metadata will be retrieved. */ repeated string info = 1; @@ -153,7 +154,7 @@ message CommandGetSchemas { * > * The returned data should be ordered by catalog_name, schema_name, table_name, then table_type. */ -message ActionGetTablesRequest { +message CommandGetTables { /* * Specifies the Catalog to search for schemas. * If omitted, then all catalogs are searched. @@ -250,7 +251,7 @@ message CommandStatementQuery { /* * Represents an instance of executing a prepared statement. Used in the command member of FlightDescriptor for * the following RPC calls: - * - DoPut: bind parameter values. + * - DoPut: bind parameter values. All of the bound parameter sets will be executed as a single atomic execution. * - GetFlightInfo: execute the prepared statement instance. */ message CommandPreparedStatementQuery { From 76aeb04951c1d5ce544eea428495ab8a15c7baba Mon Sep 17 00:00:00 2001 From: Kyle Porter Date: Wed, 30 Jun 2021 09:29:22 -0700 Subject: [PATCH 12/16] Small update to collapse custom metadata into existing field. Specify a prefix to identify the custom metadata. --- format/FlightSQL.proto | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/format/FlightSQL.proto b/format/FlightSQL.proto index cad639bd8b8..672b0230802 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSQL.proto @@ -86,11 +86,11 @@ message CommandGetSqlInfo { * SQL_QUOTED_IDENTIFIER_CASE * * If omitted, then all metadata will be retrieved. + * Flight SQL Servers may choose to include additional metadata above and beyond the specified set, however they must + * at least return the specified set. If additional metadata is included, the metadata names should be prefixed + * with "CUSTOM_" to identify them as such. */ repeated string info = 1; - - // Implementation specific custom SqlInfo values. - repeated string custom_info = 2; } /* From 46603c8baee25c41666c5c53b14adecff9d07861 Mon Sep 17 00:00:00 2001 From: Kyle Porter Date: Mon, 5 Jul 2021 15:25:33 -0700 Subject: [PATCH 13/16] Update FlightSqlProducer to conform to new design. Rename files for SQL -> Sql. Correct compilation errors in client code, but design needs to be updated. Tests do not yet compile. --- format/{FlightSQL.proto => FlightSql.proto} | 10 +- java/flight/flight-sql/pom.xml | 13 +- .../arrow/flight/sql/FlightSQLProducer.java | 389 ------------ ...ntUtils.java => FlightSqlClientUtils.java} | 61 +- .../arrow/flight/sql/FlightSqlProducer.java | 564 ++++++++++++++++++ ...lightSQLUtils.java => FlightSqlUtils.java} | 41 +- .../arrow/flight/sql/FlightSQLExample.java | 4 +- 7 files changed, 603 insertions(+), 479 deletions(-) rename format/{FlightSQL.proto => FlightSql.proto} (97%) delete mode 100644 java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java rename java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/{FlightSQLClientUtils.java => FlightSqlClientUtils.java} (73%) create mode 100644 java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java rename java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/{FlightSQLUtils.java => FlightSqlUtils.java} (74%) diff --git a/format/FlightSQL.proto b/format/FlightSql.proto similarity index 97% rename from format/FlightSQL.proto rename to format/FlightSql.proto index 672b0230802..fdf48562d14 100644 --- a/format/FlightSQL.proto +++ b/format/FlightSql.proto @@ -25,7 +25,7 @@ package arrow.flight.protocol.sql; * Represents a metadata request. Used in the command member of FlightDescriptor * for the following RPC calls: * - GetSchema: return the schema of the query. - * - GetFlightInfo: execute the query. + * - GetFlightInfo: execute the metadata request. * * The returned schema will be: * < @@ -97,7 +97,7 @@ message CommandGetSqlInfo { Represents a request to retrieve the list of catalogs on a Flight SQL enabled backend. * Used in the command member of FlightDescriptor for the following RPC calls: * - GetSchema: return the schema of the query. - * - GetFlightInfo: execute the query. + * - GetFlightInfo: execute the catalog metadata request. * * The returned schema will be: * < @@ -112,7 +112,7 @@ message CommandGetCatalogs { * Represents a request to retrieve the list of schemas on a Flight SQL enabled backend. * Used in the command member of FlightDescriptor for the following RPC calls: * - GetSchema: return the schema of the query. - * - GetFlightInfo: execute the query. + * - GetFlightInfo: execute the catalog metadata request. * * The returned schema will be: * < @@ -142,7 +142,7 @@ message CommandGetSchemas { * Represents a request to retrieve the list of tables, and optionally their schemas, on a Flight SQL enabled backend. * Used in the command member of FlightDescriptor for the following RPC calls: * - GetSchema: return the schema of the query. - * - GetFlightInfo: execute the query. + * - GetFlightInfo: execute the catalog metadata request. * * The returned schema will be: * < @@ -188,7 +188,7 @@ message CommandGetTables { * Represents a request to retrieve the list of table types on a Flight SQL enabled backend. * Used in the command member of FlightDescriptor for the following RPC calls: * - GetSchema: return the schema of the query. - * - GetFlightInfo: execute the query. + * - GetFlightInfo: execute the catalog metadata request. * * The returned schema will be: * < diff --git a/java/flight/flight-sql/pom.xml b/java/flight/flight-sql/pom.xml index f7f5fda9096..c08b7acf066 100644 --- a/java/flight/flight-sql/pom.xml +++ b/java/flight/flight-sql/pom.xml @@ -10,13 +10,13 @@ OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> + 4.0.0 arrow-java-root org.apache.arrow - 2.0.0-SNAPSHOT + 4.0.0-SNAPSHOT ../../pom.xml - 4.0.0 flight-sql Arrow Flight SQL @@ -84,13 +84,6 @@ grpc-api ${dep.grpc.version} - - org.apache.arrow - flight-core - ${project.version} - tests - test - org.apache.arrow arrow-vector @@ -163,7 +156,7 @@ test ${basedir}/src/test/protobuf - ${project.build.directory}/generated-test-sources//protobuf + ${project.build.directory}/generated-test-sources/protobuf compile diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java deleted file mode 100644 index 3802295b41d..00000000000 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLProducer.java +++ /dev/null @@ -1,389 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.arrow.flight.sql; - -import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_ACTIONS; -import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_CLOSEPREPAREDSTATEMENT; -import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETCATALOGS; -import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETPREPAREDSTATEMENT; -import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETSCHEMAS; -import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETSQLINFO; -import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETTABLES; -import static org.apache.arrow.flight.sql.FlightSQLUtils.FLIGHT_SQL_GETTABLETYPES; - -import org.apache.arrow.flight.Action; -import org.apache.arrow.flight.ActionType; -import org.apache.arrow.flight.FlightDescriptor; -import org.apache.arrow.flight.FlightInfo; -import org.apache.arrow.flight.FlightProducer; -import org.apache.arrow.flight.FlightStream; -import org.apache.arrow.flight.PutResult; -import org.apache.arrow.flight.Result; -import org.apache.arrow.flight.SchemaResult; -import org.apache.arrow.flight.Ticket; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionClosePreparedStatementRequest; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetCatalogsRequest; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementRequest; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSchemasRequest; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesRequest; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementQuery; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementUpdate; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandStatementQuery; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandStatementUpdate; - -import com.google.protobuf.Any; -import com.google.protobuf.InvalidProtocolBufferException; - -import io.grpc.Status; - -/** - * API to Implement an Arrow Flight SQL producer. - */ -public abstract class FlightSQLProducer implements FlightProducer, AutoCloseable { - /** - * Depending on the provided command, method either: - * 1. Return information about a SQL query, or - * 2. Return information about a prepared statement. In this case, parameters binding is allowed. - * - * @param context Per-call context. - * @param descriptor The descriptor identifying the data stream. - * @return information about the given SQL query, or the given prepared statement. - */ - @Override - public FlightInfo getFlightInfo(CallContext context, FlightDescriptor descriptor) { - final Any command = FlightSQLUtils.parseOrThrow(descriptor.getCommand()); - - if (command.is(CommandStatementQuery.class)) { - return getFlightInfoStatement(FlightSQLUtils.unpackOrThrow(command, CommandStatementQuery.class), descriptor, - context); - - } else if (command.is(CommandPreparedStatementQuery.class)) { - return getFlightInfoPreparedStatement( - FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), descriptor, context); - } - - throw Status.INVALID_ARGUMENT.asRuntimeException(); - } - - /** - * Returns the schema of the result produced by the SQL query. - * - * @param context Per-call context. - * @param descriptor The descriptor identifying the data stream. - * @return the result set schema. - */ - @Override - public SchemaResult getSchema(CallContext context, FlightDescriptor descriptor) { - final Any command = FlightSQLUtils.parseOrThrow(descriptor.getCommand()); - - if (command.is(CommandStatementQuery.class)) { - return getSchemaStatement(FlightSQLUtils.unpackOrThrow(command, CommandStatementQuery.class), descriptor, - context); - } - - throw Status.INVALID_ARGUMENT.asRuntimeException(); - } - - /** - * Depending on the provided command, method either: - * 1. Return data for a stream produced by executing the provided SQL query, or - * 2. Return data for a prepared statement. In this case, parameters binding is allowed. - * - * @param context Per-call context. - * @param ticket The application-defined ticket identifying this stream. - * @param listener An interface for sending data back to the client. - */ - @Override - public void getStream(CallContext context, Ticket ticket, ServerStreamListener listener) { - final Any command; - - try { - command = Any.parseFrom(ticket.getBytes()); - } catch (InvalidProtocolBufferException e) { - listener.error(e); - return; - } - - if (command.is(CommandStatementQuery.class)) { - getStreamStatement(FlightSQLUtils.unpackOrThrow(command, CommandStatementQuery.class), - context, ticket, listener); - - } else if (command.is(CommandPreparedStatementQuery.class)) { - getStreamPreparedStatement(FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), - context, ticket, listener); - } - - throw Status.INVALID_ARGUMENT.asRuntimeException(); - } - - /** - * Depending on the provided command, method either: - * 1. Execute provided SQL query as an update statement, or - * 2. Execute provided update SQL query prepared statement. In this case, parameters binding - * is allowed, or - * 3. Binds parameters to the provided prepared statement. - * - * @param context Per-call context. - * @param flightStream The data stream being uploaded. - * @param ackStream The data stream listener for update result acknowledgement. - * @return a Runnable to process the stream. - */ - @Override - public Runnable acceptPut(CallContext context, FlightStream flightStream, StreamListener ackStream) { - final Any command = FlightSQLUtils.parseOrThrow(flightStream.getDescriptor().getCommand()); - - if (command.is(CommandStatementUpdate.class)) { - return acceptPutStatement( - FlightSQLUtils.unpackOrThrow(command, CommandStatementUpdate.class), - context, flightStream, ackStream); - - } else if (command.is(CommandPreparedStatementUpdate.class)) { - return acceptPutPreparedStatementUpdate( - FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementUpdate.class), - context, flightStream, ackStream); - - } else if (command.is(CommandPreparedStatementQuery.class)) { - return acceptPutPreparedStatementQuery( - FlightSQLUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), - context, flightStream, ackStream); - } - - throw Status.INVALID_ARGUMENT.asRuntimeException(); - } - - /** - * Lists all available Flight SQL actions. - * - * @param context Per-call context. - * @param listener An interface for sending data back to the client. - */ - @Override - public void listActions(CallContext context, StreamListener listener) { - FLIGHT_SQL_ACTIONS.forEach(action -> listener.onNext(action)); - listener.onCompleted(); - } - - /** - * Performs the requested Flight SQL action. - * - * @param context Per-call context. - * @param action Client-supplied parameters. - * @param listener A stream of responses. - */ - @Override - public void doAction(CallContext context, Action action, StreamListener listener) { - - if (action.getType().equals(FLIGHT_SQL_GETSQLINFO.getType())) { - getSqlInfo(context, listener); - - } else if (action.getType().equals(FLIGHT_SQL_GETCATALOGS.getType())) { - final ActionGetCatalogsRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), - ActionGetCatalogsRequest.class); - getCatalogs(request, context, listener); - - } else if (action.getType().equals(FLIGHT_SQL_GETSCHEMAS.getType())) { - final ActionGetSchemasRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), - ActionGetSchemasRequest.class); - getSchemas(request, context, listener); - - } else if (action.getType().equals(FLIGHT_SQL_GETTABLES.getType())) { - final ActionGetTablesRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), - ActionGetTablesRequest.class); - getTables(request, context, listener); - - } else if (action.getType().equals(FLIGHT_SQL_GETTABLETYPES.getType())) { - getTableTypes(context, listener); - - } else if (action.getType().equals(FLIGHT_SQL_GETPREPAREDSTATEMENT.getType())) { - final ActionGetPreparedStatementRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), - ActionGetPreparedStatementRequest.class); - getPreparedStatement(request, context, listener); - - } else if (action.getType().equals(FLIGHT_SQL_CLOSEPREPAREDSTATEMENT.getType())) { - final ActionClosePreparedStatementRequest request = FlightSQLUtils.unpackAndParseOrThrow(action.getBody(), - ActionClosePreparedStatementRequest.class); - closePreparedStatement(request, context, listener); - } - } - - /** - * Creates a prepared statement on the server and returns a handle and metadata for in a - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult} object in a {@link Result} - * object. - * - * @param request The sql command to generate the prepared statement. - * @param context Per-call context. - * @param listener A stream of responses. - */ - public abstract void getPreparedStatement(ActionGetPreparedStatementRequest request, CallContext context, - StreamListener listener); - - /** - * Closes a prepared statement on the server. No result is expected. - * - * @param request The sql command to generate the prepared statement. - * @param context Per-call context. - * @param listener A stream of responses. - */ - public abstract void closePreparedStatement(ActionClosePreparedStatementRequest request, CallContext context, - StreamListener listener); - - /** - * Gets information about a particular SQL query based data stream. - * - * @param command The sql command to generate the data stream. - * @param context Per-call context. - * @param descriptor The descriptor identifying the data stream. - * @return Metadata about the stream. - */ - public abstract FlightInfo getFlightInfoStatement(CommandStatementQuery command, FlightDescriptor descriptor, - CallContext context); - - /** - * Gets information about a particular prepared statement data stream. - * - * @param command The prepared statement to generate the data stream. - * @param context Per-call context. - * @param descriptor The descriptor identifying the data stream. - * @return Metadata about the stream. - */ - public abstract FlightInfo getFlightInfoPreparedStatement(CommandPreparedStatementQuery command, - FlightDescriptor descriptor, CallContext context); - - - /** - * Gets schema about a particular SQL query based data stream. - * - * @param command The sql command to generate the data stream. - * @param context Per-call context. - * @param descriptor The descriptor identifying the data stream. - * @return Schema for the stream. - */ - public abstract SchemaResult getSchemaStatement(CommandStatementQuery command, FlightDescriptor descriptor, - CallContext context); - - /** - * Returns data for a SQL query based data stream. - * - * @param command The sql command to generate the data stream. - * @param context Per-call context. - * @param ticket The application-defined ticket identifying this stream. - * @param listener An interface for sending data back to the client. - */ - public abstract void getStreamStatement(CommandStatementQuery command, CallContext context, Ticket ticket, - ServerStreamListener listener); - - /** - * Returns data for a particular prepared statement query instance. - * - * @param command The prepared statement to generate the data stream. - * @param context Per-call context. - * @param ticket The application-defined ticket identifying this stream. - * @param listener An interface for sending data back to the client. - */ - public abstract void getStreamPreparedStatement(CommandPreparedStatementQuery command, CallContext context, - Ticket ticket, ServerStreamListener listener); - - /** - * Accepts uploaded data for a particular SQL query based data stream. PutResults must be in the form of a - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.DoPutUpdateResult}. - * - * @param command The sql command to generate the data stream. - * @param context Per-call context. - * @param flightStream The data stream being uploaded. - * @param ackStream The result data stream. - * @return A runnable to process the stream. - */ - public abstract Runnable acceptPutStatement(CommandStatementUpdate command, CallContext context, - FlightStream flightStream, StreamListener ackStream); - - /** - * Accepts uploaded data for a particular prepared statement data stream. PutResults must be in the form of a - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.DoPutUpdateResult}. - * - * @param command The prepared statement to generate the data stream. - * @param context Per-call context. - * @param flightStream The data stream being uploaded. - * @param ackStream The result data stream. - * @return A runnable to process the stream. - */ - public abstract Runnable acceptPutPreparedStatementUpdate(CommandPreparedStatementUpdate command, - CallContext context, FlightStream flightStream, StreamListener ackStream); - - /** - * Accepts uploaded parameter values for a particular prepared statement query. - * - * @param command The prepared statement the parameter values will bind to. - * @param context Per-call context. - * @param flightStream The data stream being uploaded. - * @param ackStream The result data stream. - * @return A runnable to process the stream. - */ - public abstract Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery command, - CallContext context, FlightStream flightStream, StreamListener ackStream); - - /** - * Returns the SQL Info of the server by returning a - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSQLInfoResult} in a {@link Result}. - * - * @param context Per-call context. - * @param listener A stream of responses. - */ - public abstract void getSqlInfo(CallContext context, StreamListener listener); - - /** - * Returns the available catalogs by returning a stream of - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetCatalogsResult} objects in {@link Result} objects. - * - * @param request request filter parameters. - * @param context Per-call context. - * @param listener A stream of responses. - */ - public abstract void getCatalogs(ActionGetCatalogsRequest request, CallContext context, - StreamListener listener); - - /** - * Returns the available schemas by returning a stream of - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetSchemasResult} objects in {@link Result} objects. - * - * @param request request filter parameters. - * @param context Per-call context. - * @param listener A stream of responses. - */ - public abstract void getSchemas(ActionGetSchemasRequest request, CallContext context, - StreamListener listener); - - /** - * Returns the available tables by returning a stream of - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult} objects in {@link Result} objects. - * - * @param request request filter parameters. - * @param context Per-call context. - * @param listener A stream of responses. - */ - public abstract void getTables(ActionGetTablesRequest request, CallContext context, StreamListener listener); - - /** - * Returns the available table types by returning a stream of - * {@link org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTableTypesResult} objects in {@link Result} objects. - * - * @param context Per-call context. - * @param listener A stream of responses. - */ - public abstract void getTableTypes(CallContext context, StreamListener listener); -} diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLClientUtils.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClientUtils.java similarity index 73% rename from java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLClientUtils.java rename to java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClientUtils.java index 3a462e106c2..f93c242312e 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLClientUtils.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClientUtils.java @@ -20,7 +20,6 @@ import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -29,11 +28,9 @@ import org.apache.arrow.flight.FlightDescriptor; import org.apache.arrow.flight.FlightInfo; import org.apache.arrow.flight.Result; -import org.apache.arrow.flight.sql.impl.FlightSQL; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesRequest; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSql; +import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery; import org.apache.arrow.vector.types.pojo.Schema; import com.google.protobuf.Any; @@ -44,7 +41,7 @@ /** * Client side utilities to work with Flight SQL semantics. */ -public final class FlightSQLClientUtils { +public final class FlightSqlClientUtils { /** * Helper method to request a list of tables from a Flight SQL enabled endpoint. @@ -55,42 +52,32 @@ public final class FlightSQLClientUtils { * @param tableFilterPattern The table filter pattern. * @param tableTypes The table types to include. * @param includeSchema True to include the schema upon return, false to not include the schema. - * @return A list of tables matching the criteria. + * @return a FlightInfo object representing the stream(s) to fetch. */ - public static List getTables(FlightClient client, String catalog, String schemaFilterPattern, + public static FlightInfo getTables(FlightClient client, String catalog, String schemaFilterPattern, String tableFilterPattern, List tableTypes, boolean includeSchema) { - final ActionGetTablesRequest.Builder requestBuilder = ActionGetTablesRequest - .newBuilder() - .setIncludeSchema(includeSchema); + final FlightSql.CommandGetTables.Builder builder = FlightSql.CommandGetTables.newBuilder(); if (catalog != null) { - requestBuilder.setCatalog(catalog); + builder.setCatalog(catalog); } if (schemaFilterPattern != null) { - requestBuilder.setSchemaFilterPattern(schemaFilterPattern); + builder.setSchemaFilterPattern(schemaFilterPattern); } if (tableFilterPattern != null) { - requestBuilder.setTableNameFilterPattern(tableFilterPattern); + builder.setTableNameFilterPattern(tableFilterPattern); } if (tableTypes != null) { - requestBuilder.addAllTableTypes(tableTypes); + builder.addAllTableTypes(tableTypes); } + builder.setIncludeSchema(includeSchema); - final Iterator results = client.doAction(new Action( - "GetTables", Any.pack(requestBuilder.build()).toByteArray())); - - final List getTablesResults = new ArrayList<>(); - results.forEachRemaining(result -> { - ActionGetTablesResult actual = FlightSQLUtils.unpackAndParseOrThrow(result.getBody(), - ActionGetTablesResult.class); - getTablesResults.add(actual); - }); - - return getTablesResults; + final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); + return client.getInfo(descriptor); } /** @@ -100,16 +87,16 @@ public static List getTables(FlightClient client, String * @param query The query to prepare. * @return Metadata and handles to the prepared statement which exists on the server. */ - public static FlightSQLPreparedStatement getPreparedStatement(FlightClient client, String query) { - return new FlightSQLPreparedStatement(client, query); + public static FlightSqlPreparedStatement getPreparedStatement(FlightClient client, String query) { + return new FlightSqlPreparedStatement(client, query); } /** * Helper class to encapsulate Flight SQL prepared statement logic. */ - public static class FlightSQLPreparedStatement implements Closeable { + public static class FlightSqlPreparedStatement implements Closeable { private final FlightClient client; - private final ActionGetPreparedStatementResult preparedStatementResult; + private final ActionCreatePreparedStatementResult preparedStatementResult; private long invocationCount; private boolean isClosed; private Schema resultSetSchema = null; @@ -118,22 +105,22 @@ public static class FlightSQLPreparedStatement implements Closeable { /** * Constructor. * - * @param client The client. FlightSQLPreparedStatement does not maintain this resource. + * @param client The client. FlightSqlPreparedStatement does not maintain this resource. * @param sql The query. */ - public FlightSQLPreparedStatement(FlightClient client, String sql) { + public FlightSqlPreparedStatement(FlightClient client, String sql) { this.client = client; final Iterator preparedStatementResults = client.doAction(new Action("GetPreparedStatement", - Any.pack(FlightSQL.ActionGetPreparedStatementRequest + Any.pack(FlightSql.ActionCreatePreparedStatementRequest .newBuilder() .setQuery(sql) .build()) .toByteArray())); - preparedStatementResult = FlightSQLUtils.unpackAndParseOrThrow( + preparedStatementResult = FlightSqlUtils.unpackAndParseOrThrow( preparedStatementResults.next().getBody(), - ActionGetPreparedStatementResult.class); + ActionCreatePreparedStatementResult.class); invocationCount = 0; isClosed = false; @@ -198,7 +185,7 @@ public long executeUpdate() { public void close() { isClosed = true; final Iterator closePreparedStatementResults = client.doAction(new Action("ClosePreparedStatement", - Any.pack(FlightSQL.ActionClosePreparedStatementRequest + Any.pack(FlightSql.ActionClosePreparedStatementRequest .newBuilder() .setPreparedStatementHandleBytes(preparedStatementResult.getPreparedStatementHandle()) .build()) diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java new file mode 100644 index 00000000000..e9abd878070 --- /dev/null +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java @@ -0,0 +1,564 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.sql; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.arrow.flatbuf.Type; +import org.apache.arrow.flight.Action; +import org.apache.arrow.flight.ActionType; +import org.apache.arrow.flight.FlightDescriptor; +import org.apache.arrow.flight.FlightInfo; +import org.apache.arrow.flight.FlightProducer; +import org.apache.arrow.flight.FlightStream; +import org.apache.arrow.flight.PutResult; +import org.apache.arrow.flight.Result; +import org.apache.arrow.flight.SchemaResult; +import org.apache.arrow.flight.Ticket; +import org.apache.arrow.flight.sql.impl.FlightSql.ActionClosePreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCatalogs; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSchemas; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSqlInfo; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTableTypes; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTables; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementUpdate; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementUpdate; +import org.apache.arrow.vector.types.UnionMode; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; + +import com.google.protobuf.Any; +import com.google.protobuf.InvalidProtocolBufferException; + +import io.grpc.Status; + +/** + * API to Implement an Arrow Flight SQL producer. + */ +public abstract class FlightSqlProducer implements FlightProducer, AutoCloseable { + /** + * Depending on the provided command, method either: + * 1. Return information about a SQL query, or + * 2. Return information about a prepared statement. In this case, parameters binding is allowed. + * + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return information about the given SQL query, or the given prepared statement. + */ + @Override + public FlightInfo getFlightInfo(CallContext context, FlightDescriptor descriptor) { + final Any command = FlightSqlUtils.parseOrThrow(descriptor.getCommand()); + + if (command.is(CommandStatementQuery.class)) { + return getFlightInfoStatement( + FlightSqlUtils.unpackOrThrow(command, CommandStatementQuery.class), context, descriptor); + } else if (command.is(CommandPreparedStatementQuery.class)) { + return getFlightInfoPreparedStatement( + FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), context, descriptor); + } else if (command.is(CommandGetCatalogs.class)) { + return getFlightInfoCatalogs( + FlightSqlUtils.unpackOrThrow(command, CommandGetCatalogs.class), context, descriptor); + } else if (command.is(CommandGetSchemas.class)) { + return getFlightInfoSchemas( + FlightSqlUtils.unpackOrThrow(command, CommandGetSchemas.class), context, descriptor); + } else if (command.is(CommandGetTables.class)) { + return getFlightInfoTables( + FlightSqlUtils.unpackOrThrow(command, CommandGetTables.class), context, descriptor); + } else if (command.is(CommandGetTableTypes.class)) { + return getFlightInfoTableTypes(context, descriptor); + } else if (command.is(CommandGetSqlInfo.class)) { + return getFlightInfoSqlInfo( + FlightSqlUtils.unpackOrThrow(command, CommandGetSqlInfo.class), context, descriptor); + } + + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + + /** + * Returns the schema of the result produced by the SQL query. + * + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return the result set schema. + */ + @Override + public SchemaResult getSchema(CallContext context, FlightDescriptor descriptor) { + final Any command = FlightSqlUtils.parseOrThrow(descriptor.getCommand()); + + if (command.is(CommandStatementQuery.class)) { + return getSchemaStatement( + FlightSqlUtils.unpackOrThrow(command, CommandStatementQuery.class), context, descriptor); + } else if (command.is(CommandGetCatalogs.class)) { + return getSchemaCatalogs(); + } else if (command.is(CommandGetSchemas.class)) { + return getSchemaSchemas(); + } else if (command.is(CommandGetTables.class)) { + return getSchemaTables(); + } else if (command.is(CommandGetTableTypes.class)) { + return getSchemaTableTypes(); + } else if (command.is(CommandGetSqlInfo.class)) { + return getSchemaSqlInfo(); + } + + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + + /** + * Depending on the provided command, method either: + * 1. Return data for a stream produced by executing the provided SQL query, or + * 2. Return data for a prepared statement. In this case, parameters binding is allowed. + * + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + @Override + public void getStream(CallContext context, Ticket ticket, ServerStreamListener listener) { + final Any command; + + try { + command = Any.parseFrom(ticket.getBytes()); + } catch (InvalidProtocolBufferException e) { + listener.error(e); + return; + } + + if (command.is(CommandStatementQuery.class)) { + getStreamStatement( + FlightSqlUtils.unpackOrThrow(command, CommandStatementQuery.class), context, ticket, listener); + } else if (command.is(CommandPreparedStatementQuery.class)) { + getStreamPreparedStatement( + FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), context, ticket, listener); + } else if (command.is(CommandGetCatalogs.class)) { + getStreamCatalogs(context, ticket, listener); + } else if (command.is(CommandGetSchemas.class)) { + getStreamSchemas(FlightSqlUtils.unpackOrThrow(command, CommandGetSchemas.class), context, ticket, listener); + } else if (command.is(CommandGetTables.class)) { + getStreamTables(FlightSqlUtils.unpackOrThrow(command, CommandGetTables.class), context, ticket, listener); + } else if (command.is(CommandGetTableTypes.class)) { + getStreamTableTypes(context, ticket, listener); + } else if (command.is(CommandGetSqlInfo.class)) { + getStreamSqlInfo(FlightSqlUtils.unpackOrThrow(command, CommandGetSqlInfo.class), context, ticket, listener); + } else { + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + } + + /** + * Depending on the provided command, method either: + * 1. Execute provided SQL query as an update statement, or + * 2. Execute provided update SQL query prepared statement. In this case, parameters binding + * is allowed, or + * 3. Binds parameters to the provided prepared statement. + * + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The data stream listener for update result acknowledgement. + * @return a Runnable to process the stream. + */ + @Override + public Runnable acceptPut(CallContext context, FlightStream flightStream, StreamListener ackStream) { + final Any command = FlightSqlUtils.parseOrThrow(flightStream.getDescriptor().getCommand()); + + if (command.is(CommandStatementUpdate.class)) { + return acceptPutStatement( + FlightSqlUtils.unpackOrThrow(command, CommandStatementUpdate.class), + context, flightStream, ackStream); + + } else if (command.is(CommandPreparedStatementUpdate.class)) { + return acceptPutPreparedStatementUpdate( + FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementUpdate.class), + context, flightStream, ackStream); + + } else if (command.is(CommandPreparedStatementQuery.class)) { + return acceptPutPreparedStatementQuery( + FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), + context, flightStream, ackStream); + } + + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + + /** + * Lists all available Flight SQL actions. + * + * @param context Per-call context. + * @param listener An interface for sending data back to the client. + */ + @Override + public void listActions(CallContext context, StreamListener listener) { + FlightSqlUtils.FLIGHT_SQL_ACTIONS.forEach(listener::onNext); + listener.onCompleted(); + } + + /** + * Performs the requested Flight SQL action. + * + * @param context Per-call context. + * @param action Client-supplied parameters. + * @param listener A stream of responses. + */ + @Override + public void doAction(CallContext context, Action action, StreamListener listener) { + if (action.getType().equals(FlightSqlUtils.FLIGHT_SQL_CREATEPREPAREDSTATEMENT.getType())) { + final ActionCreatePreparedStatementRequest request = FlightSqlUtils.unpackAndParseOrThrow(action.getBody(), + ActionCreatePreparedStatementRequest.class); + createPreparedStatement(request, context, listener); + } else if (action.getType().equals(FlightSqlUtils.FLIGHT_SQL_CLOSEPREPAREDSTATEMENT.getType())) { + final ActionClosePreparedStatementRequest request = FlightSqlUtils.unpackAndParseOrThrow(action.getBody(), + ActionClosePreparedStatementRequest.class); + closePreparedStatement(request, context, listener); + } + } + + /** + * Creates a prepared statement on the server and returns a handle and metadata for in a + * {@link org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult} object in a {@link Result} + * object. + * + * @param request The sql command to generate the prepared statement. + * @param context Per-call context. + * @param listener A stream of responses. + */ + public abstract void createPreparedStatement(ActionCreatePreparedStatementRequest request, CallContext context, + StreamListener listener); + + /** + * Closes a prepared statement on the server. No result is expected. + * + * @param request The sql command to generate the prepared statement. + * @param context Per-call context. + * @param listener A stream of responses. + */ + public abstract void closePreparedStatement(ActionClosePreparedStatementRequest request, CallContext context, + StreamListener listener); + + /** + * Gets information about a particular SQL query based data stream. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoStatement(CommandStatementQuery command, CallContext context, + FlightDescriptor descriptor); + + /** + * Gets information about a particular prepared statement data stream. + * + * @param command The prepared statement to generate the data stream. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoPreparedStatement(CommandPreparedStatementQuery command, + CallContext context, FlightDescriptor descriptor); + + /** + * Gets schema about a particular SQL query based data stream. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Schema for the stream. + */ + public abstract SchemaResult getSchemaStatement(CommandStatementQuery command, CallContext context, + FlightDescriptor descriptor); + + /** + * Returns data for a SQL query based data stream. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamStatement(CommandStatementQuery command, CallContext context, Ticket ticket, + ServerStreamListener listener); + + /** + * Returns data for a particular prepared statement query instance. + * + * @param command The prepared statement to generate the data stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamPreparedStatement(CommandPreparedStatementQuery command, CallContext context, + Ticket ticket, ServerStreamListener listener); + + /** + * Accepts uploaded data for a particular SQL query based data stream. PutResults must be in the form of a + * {@link org.apache.arrow.flight.sql.impl.FlightSql.DoPutUpdateResult}. + * + * @param command The sql command to generate the data stream. + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The result data stream. + * @return A runnable to process the stream. + */ + public abstract Runnable acceptPutStatement(CommandStatementUpdate command, CallContext context, + FlightStream flightStream, StreamListener ackStream); + + /** + * Accepts uploaded data for a particular prepared statement data stream. PutResults must be in the form of a + * {@link org.apache.arrow.flight.sql.impl.FlightSql.DoPutUpdateResult}. + * + * @param command The prepared statement to generate the data stream. + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The result data stream. + * @return A runnable to process the stream. + */ + public abstract Runnable acceptPutPreparedStatementUpdate(CommandPreparedStatementUpdate command, + CallContext context, FlightStream flightStream, StreamListener ackStream); + + /** + * Accepts uploaded parameter values for a particular prepared statement query. + * + * @param command The prepared statement the parameter values will bind to. + * @param context Per-call context. + * @param flightStream The data stream being uploaded. + * @param ackStream The result data stream. + * @return A runnable to process the stream. + */ + public abstract Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery command, + CallContext context, FlightStream flightStream, StreamListener ackStream); + + /** + * Returns the SQL Info of the server by returning a + * {@link org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSqlInfo} in a {@link Result}. + * + * @param request request filter parameters. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoSqlInfo(CommandGetSqlInfo request, CallContext context, + FlightDescriptor descriptor); + + /** + * Gets schema about the get SQL info data stream. + * + * @return Schema for the stream. + */ + public SchemaResult getSchemaSqlInfo() { + final List fields = new ArrayList<>(); + + fields.add(new Field( + "info_name", + new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), + null)); + fields.add(new Field( + "value", + new FieldType(false, + new ArrowType.Union(UnionMode.Dense, new int[] {Type.Utf8, Type.Int}), /*dictionary=*/null), + null)); + + return new SchemaResult(new Schema(fields)); + } + + /** + * Returns data for SQL info based data stream. + * + * @param command The command to generate the data stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamSqlInfo(CommandGetSqlInfo command, CallContext context, Ticket ticket, + ServerStreamListener listener); + + /** + * Returns the available catalogs by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCatalogs} objects in {@link Result} objects. + * + * @param request request filter parameters. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoCatalogs(CommandGetCatalogs request, CallContext context, + FlightDescriptor descriptor); + + /** + * Gets schema about the get catalogs data stream. + * + * @return Schema for the stream. + */ + public SchemaResult getSchemaCatalogs() { + final List fields = new ArrayList<>(); + + fields.add(new Field( + "catalog_name", + new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), + null)); + + return new SchemaResult(new Schema(fields)); + } + + /** + * Returns data for catalogs based data stream. + * + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamCatalogs(CallContext context, Ticket ticket, + ServerStreamListener listener); + + /** + * Returns the available schemas by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSchemas} objects in {@link Result} objects. + * + * @param request request filter parameters. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoSchemas(CommandGetSchemas request, CallContext context, + FlightDescriptor descriptor); + + /** + * Gets schema about the get schemas data stream. + * + * @return Schema for the stream. + */ + public SchemaResult getSchemaSchemas() { + final List fields = new ArrayList<>(); + + fields.add(new Field( + "catalog_name", + new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), + null)); + fields.add(new Field( + "schema_name", + new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), + null)); + + return new SchemaResult(new Schema(fields)); + } + + /** + * Returns data for schemas based data stream. + * + * @param command The command to generate the data stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamSchemas(CommandGetSchemas command, CallContext context, Ticket ticket, + ServerStreamListener listener); + + /** + * Returns the available tables by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTables} objects in {@link Result} objects. + * + * @param request request filter parameters. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoTables(CommandGetTables request, CallContext context, + FlightDescriptor descriptor); + + /** + * Gets schema about the get tables data stream. + * + * @return Schema for the stream. + */ + public SchemaResult getSchemaTables() { + final List fields = new ArrayList<>(); + + fields.add(new Field( + "catalog_name", + new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), + null)); + fields.add(new Field( + "schema_name", + new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), + null)); + fields.add(new Field( + "table_name", + new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), + null)); + fields.add(new Field( + "table_type", + new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), + null)); + fields.add(new Field( + "table_schema", + new FieldType(false, ArrowType.Binary.INSTANCE, /*dictionary=*/null), + null)); + + return new SchemaResult(new Schema(fields)); + } + + /** + * Returns data for tables based data stream. + * + * @param command The command to generate the data stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamTables(CommandGetTables command, CallContext context, Ticket ticket, + ServerStreamListener listener); + + /** + * Returns the available table types by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTableTypes} objects in {@link Result} objects. + * + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoTableTypes(CallContext context, FlightDescriptor descriptor); + + /** + * Gets schema about the get table types data stream. + * + * @return Schema for the stream. + */ + public SchemaResult getSchemaTableTypes() { + final List fields = new ArrayList<>(); + + fields.add(new Field( + "table_type", + new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), + null)); + + return new SchemaResult(new Schema(fields)); + } + + /** + * Returns data for table types based data stream. + * + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamTableTypes(CallContext context, Ticket ticket, ServerStreamListener listener); +} diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java similarity index 74% rename from java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java rename to java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java index 7bd977de0df..f390845c296 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSQLUtils.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java @@ -34,7 +34,7 @@ /** * Utilities to work with Flight SQL semantics. */ -public final class FlightSQLUtils { +public final class FlightSqlUtils { private static final int BIT_WIDTH8 = 8; private static final int BIT_WIDTH_16 = 16; @@ -43,35 +43,10 @@ public final class FlightSQLUtils { private static final boolean IS_SIGNED_FALSE = false; private static final boolean IS_SIGNED_TRUE = true; - public static final ActionType FLIGHT_SQL_GETSQLINFO = new ActionType("GetSQLINFO", - "Retrieves details of SQL capabilities of the Flight server. \n" + - "Request Message: N/A\n" + - "Response Message: ActionGetSQLInfoResult"); - - public static final ActionType FLIGHT_SQL_GETCATALOGS = new ActionType("GetCatalogs", - "Retrieves a list of all catalogs available on the server. \n" + - "Request Message: ActionGetCatalogsRequest\n" + - "Response Message: ActionGetCatalogsResult"); - - public static final ActionType FLIGHT_SQL_GETSCHEMAS = new ActionType("GetSchemas", - "Retrieves a list of schemas available on the server. \n" + - "Request Message: ActionGetSchemasRequest\n" + - "Response Message: ActionGetSchemasResult"); - - public static final ActionType FLIGHT_SQL_GETTABLES = new ActionType("GetTables", - "Retrieves a list of tables available on the server. \n" + - "Request Message: ActionGetTablesRequest\n" + - "Response Message: ActionGetTablesResult"); - - public static final ActionType FLIGHT_SQL_GETTABLETYPES = new ActionType("GetTableTypes", - "Retrieves a list of table types available on the server. \n" + - "Request Message: N/A\n" + - "Response Message: ActionGetTableTypesResult"); - - public static final ActionType FLIGHT_SQL_GETPREPAREDSTATEMENT = new ActionType("GetPreparedStatement", + public static final ActionType FLIGHT_SQL_CREATEPREPAREDSTATEMENT = new ActionType("CreatePreparedStatement", "Creates a reusable prepared statement resource on the server. \n" + - "Request Message: ActionGetPreparedStatementRequest\n" + - "Response Message: ActionGetPreparedStatementResult"); + "Request Message: ActionCreatePreparedStatementRequest\n" + + "Response Message: ActionCreatePreparedStatementResult"); public static final ActionType FLIGHT_SQL_CLOSEPREPAREDSTATEMENT = new ActionType("ClosePreparedStatement", "Closes a reusable prepared statement resource on the server. \n" + @@ -79,12 +54,7 @@ public final class FlightSQLUtils { "Response Message: N/A"); public static final List FLIGHT_SQL_ACTIONS = ImmutableList.of( - FLIGHT_SQL_GETSQLINFO, - FLIGHT_SQL_GETCATALOGS, - FLIGHT_SQL_GETSCHEMAS, - FLIGHT_SQL_GETTABLES, - FLIGHT_SQL_GETTABLETYPES, - FLIGHT_SQL_GETPREPAREDSTATEMENT, + FLIGHT_SQL_CREATEPREPAREDSTATEMENT, FLIGHT_SQL_CLOSEPREPAREDSTATEMENT ); @@ -155,7 +125,6 @@ public static ArrowType getArrowTypeFromJDBCType(int jdbcDataType, int precision case Types.TIMESTAMP_WITH_TIMEZONE: default: return ArrowType.Utf8.INSTANCE; - // throw new UnsupportedOperationException(); } } diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java index f7326465b8c..e7c45bcc982 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java @@ -17,7 +17,7 @@ package org.apache.arrow.flight.sql; -import static org.apache.arrow.flight.sql.FlightSQLUtils.getArrowTypeFromJDBCType; +import static org.apache.arrow.flight.sql.FlightSqlUtils.getArrowTypeFromJDBCType; import java.io.File; import java.io.IOException; @@ -259,7 +259,7 @@ private Schema buildSchema(String catalog, String schema, String table) throws S final int precision = columns.getInt("DECIMAL_DIGITS"); final int scale = columns.getInt("COLUMN_SIZE"); - final ArrowType arrowType = FlightSQLUtils.getArrowTypeFromJDBCType(jdbcDataType, precision, scale); + final ArrowType arrowType = FlightSqlUtils.getArrowTypeFromJDBCType(jdbcDataType, precision, scale); final FieldType fieldType = new FieldType(arrowIsNullable, arrowType, /*dictionary=*/null); fields.add(new Field(columnName, fieldType, null)); From 9481f44edc06be9df68a0d99fac7d85de88c8ae7 Mon Sep 17 00:00:00 2001 From: Kyle Porter Date: Mon, 5 Jul 2021 17:12:44 -0700 Subject: [PATCH 14/16] Correct the dense_union type for schema return of SQL info. Correct some additional SQL -> Sql file renames. Reduce the test compilation problems (still more to do). --- .../arrow/flight/sql/FlightSqlProducer.java | 65 ++++++------------- ...{TestFlightSQL.java => TestFlightSql.java} | 36 +++++----- ...tSQLExample.java => FlightSqlExample.java} | 30 ++++----- .../flight/sql/PreparedStatementCacheKey.java | 4 +- ...QLExample.proto => flightSqlExample.proto} | 0 5 files changed, 54 insertions(+), 81 deletions(-) rename java/flight/flight-sql/src/test/java/org/apache/arrow/flight/{TestFlightSQL.java => TestFlightSql.java} (89%) rename java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/{FlightSQLExample.java => FlightSqlExample.java} (95%) rename java/flight/flight-sql/src/test/protobuf/{flightSQLExample.proto => flightSqlExample.proto} (100%) diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java index e9abd878070..5e4f76f1a81 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.List; -import org.apache.arrow.flatbuf.Type; import org.apache.arrow.flight.Action; import org.apache.arrow.flight.ActionType; import org.apache.arrow.flight.FlightDescriptor; @@ -42,6 +41,7 @@ import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementUpdate; import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementQuery; import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementUpdate; +import org.apache.arrow.vector.types.Types; import org.apache.arrow.vector.types.UnionMode; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; @@ -367,15 +367,19 @@ public abstract FlightInfo getFlightInfoSqlInfo(CommandGetSqlInfo request, CallC public SchemaResult getSchemaSqlInfo() { final List fields = new ArrayList<>(); - fields.add(new Field( - "info_name", - new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), - null)); + fields.add(new Field("info_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + + // dense_union + final List children = new ArrayList<>(); + children.add(new Field("string_value", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + children.add(new Field("int_value", FieldType.nullable(Types.MinorType.INT.getType()), null)); + children.add(new Field("bigint_value", FieldType.nullable(Types.MinorType.BIGINT.getType()), null)); + children.add(new Field("int32_bitmask", FieldType.nullable(Types.MinorType.INT.getType()), null)); + fields.add(new Field( "value", - new FieldType(false, - new ArrowType.Union(UnionMode.Dense, new int[] {Type.Utf8, Type.Int}), /*dictionary=*/null), - null)); + new FieldType(false, new ArrowType.Union(UnionMode.Dense, new int[] {0, 1, 2, 3}), /*dictionary=*/null), + children)); return new SchemaResult(new Schema(fields)); } @@ -411,10 +415,7 @@ public abstract FlightInfo getFlightInfoCatalogs(CommandGetCatalogs request, Cal public SchemaResult getSchemaCatalogs() { final List fields = new ArrayList<>(); - fields.add(new Field( - "catalog_name", - new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), - null)); + fields.add(new Field("catalog_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); return new SchemaResult(new Schema(fields)); } @@ -449,14 +450,8 @@ public abstract FlightInfo getFlightInfoSchemas(CommandGetSchemas request, CallC public SchemaResult getSchemaSchemas() { final List fields = new ArrayList<>(); - fields.add(new Field( - "catalog_name", - new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), - null)); - fields.add(new Field( - "schema_name", - new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), - null)); + fields.add(new Field("catalog_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("schema_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); return new SchemaResult(new Schema(fields)); } @@ -492,26 +487,11 @@ public abstract FlightInfo getFlightInfoTables(CommandGetTables request, CallCon public SchemaResult getSchemaTables() { final List fields = new ArrayList<>(); - fields.add(new Field( - "catalog_name", - new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), - null)); - fields.add(new Field( - "schema_name", - new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), - null)); - fields.add(new Field( - "table_name", - new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), - null)); - fields.add(new Field( - "table_type", - new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), - null)); - fields.add(new Field( - "table_schema", - new FieldType(false, ArrowType.Binary.INSTANCE, /*dictionary=*/null), - null)); + fields.add(new Field("catalog_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("schema_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("table_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("table_type", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("table_schema", FieldType.nullable(Types.MinorType.VARBINARY.getType()), null)); return new SchemaResult(new Schema(fields)); } @@ -545,10 +525,7 @@ public abstract void getStreamTables(CommandGetTables command, CallContext conte public SchemaResult getSchemaTableTypes() { final List fields = new ArrayList<>(); - fields.add(new Field( - "table_type", - new FieldType(false, ArrowType.Utf8.INSTANCE, /*dictionary=*/null), - null)); + fields.add(new Field("table_type", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); return new SchemaResult(new Schema(fields)); } diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java similarity index 89% rename from java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java rename to java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java index fd393472acf..7767bd9e907 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSQL.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java @@ -17,8 +17,7 @@ package org.apache.arrow.flight; -import static org.apache.arrow.flight.sql.FlightSQLClientUtils.getPreparedStatement; -import static org.apache.arrow.flight.sql.FlightSQLClientUtils.getTables; +import static org.apache.arrow.flight.sql.FlightSqlClientUtils.getPreparedStatement; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -26,18 +25,15 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; import java.util.List; -import org.apache.arrow.flight.sql.FlightSQLClientUtils; -import org.apache.arrow.flight.sql.FlightSQLExample; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionClosePreparedStatementRequest; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementRequest; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesRequest; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementQuery; +import org.apache.arrow.flight.sql.FlightSqlClientUtils; +import org.apache.arrow.flight.sql.FlightSqlExample; +import org.apache.arrow.flight.sql.impl.FlightSql.ActionClosePreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementRequest; +import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery; import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.memory.util.ArrowBufPointer; @@ -61,7 +57,7 @@ /** * Test direct usage of Flight SQL workflows. */ -public class TestFlightSQL { +public class TestFlightSql { private static BufferAllocator allocator; private static FlightServer server; @@ -80,7 +76,7 @@ public static void setUp() throws Exception { allocator = new RootAllocator(Integer.MAX_VALUE); final Location serverLocation = Location.forGrpcInsecure(FlightTestUtil.LOCALHOST, 0); - server = FlightServer.builder(allocator, serverLocation, new FlightSQLExample(serverLocation)).build(); + server = FlightServer.builder(allocator, serverLocation, new FlightSqlExample(serverLocation)).build(); server.start(); final Location clientLocation = Location.forGrpcInsecure(FlightTestUtil.LOCALHOST, server.getPort()); @@ -92,6 +88,7 @@ public static void tearDown() throws Exception { AutoCloseables.close(client, server, allocator); } + /* @Test public void testGetTables() throws Exception { // Arrange @@ -119,7 +116,7 @@ public void testGetTables() throws Exception { } @Test - public void testGetTablesWithFlightSQLClientUtils() throws Exception { + public void testGetTablesWithFlightSqlClientUtils() throws Exception { // Arrange final ActionGetTablesResult expected = ActionGetTablesResult.newBuilder() .setSchema("APP") @@ -136,19 +133,20 @@ public void testGetTablesWithFlightSQLClientUtils() throws Exception { assertEquals(1, results.size()); assertEquals(expected, results.get(0)); } + */ @Test public void testSimplePrepStmt() throws Exception { final Iterator preparedStatementResults = client.doAction(new Action("GetPreparedStatement", - Any.pack(ActionGetPreparedStatementRequest + Any.pack(ActionCreatePreparedStatementRequest .newBuilder() .setQuery("Select * from intTable") .build()) .toByteArray())); assertTrue(preparedStatementResults.hasNext()); - final ActionGetPreparedStatementResult preparedStatementResult = - Any.parseFrom(preparedStatementResults.next().getBody()).unpack(ActionGetPreparedStatementResult.class); + final ActionCreatePreparedStatementResult preparedStatementResult = + Any.parseFrom(preparedStatementResults.next().getBody()).unpack(ActionCreatePreparedStatementResult.class); assertFalse(preparedStatementResults.hasNext()); final Schema actualSchema = Schema.deserialize(preparedStatementResult.getDatasetSchema().asReadOnlyByteBuffer()); @@ -210,8 +208,8 @@ public void testSimplePrepStmt() throws Exception { } @Test - public void testSimplePrepStmtWithFlightSQLClientUtils() throws Exception { - final FlightSQLClientUtils.FlightSQLPreparedStatement preparedStatement = + public void testSimplePrepStmtWithFlightSqlClientUtils() throws Exception { + final FlightSqlClientUtils.FlightSqlPreparedStatement preparedStatement = getPreparedStatement(client, "Select * from intTable"); final Schema actualSchema = preparedStatement.getResultSetSchema(); diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSqlExample.java similarity index 95% rename from java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java rename to java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSqlExample.java index e7c45bcc982..b6247714bdd 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSQLExample.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSqlExample.java @@ -52,13 +52,11 @@ import org.apache.arrow.flight.Result; import org.apache.arrow.flight.SchemaResult; import org.apache.arrow.flight.Ticket; -import org.apache.arrow.flight.sql.impl.FlightSQL; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetPreparedStatementResult; -import org.apache.arrow.flight.sql.impl.FlightSQL.ActionGetTablesResult; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementQuery; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandPreparedStatementUpdate; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandStatementQuery; -import org.apache.arrow.flight.sql.impl.FlightSQL.CommandStatementUpdate; +import org.apache.arrow.flight.sql.impl.FlightSql; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementUpdate; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementQuery; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandStatementUpdate; import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.util.AutoCloseables; @@ -93,15 +91,15 @@ import io.grpc.Status; /** - * Proof of concept {@link FlightSQLProducer} implementation showing an Apache Derby backed Flight SQL server capable + * Proof of concept {@link FlightSqlProducer} implementation showing an Apache Derby backed Flight SQL server capable * of the following workflows: * - returning a list of tables from the action "GetTables". * - creation of a prepared statement from the action "GetPreparedStatement". * - execution of a prepared statement by using a {@link CommandPreparedStatementQuery} with getFlightInfo and * getStream. */ -public class FlightSQLExample extends FlightSQLProducer implements AutoCloseable { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FlightSQLExample.class); +public class FlightSqlExample extends FlightSqlProducer implements AutoCloseable { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FlightSqlExample.class); private static final int BATCH_ROW_SIZE = 1000; @@ -111,7 +109,7 @@ public class FlightSQLExample extends FlightSQLProducer implements AutoCloseable private final LoadingCache commandExecutePreparedStatementLoadingCache; private final LoadingCache preparedStatementLoadingCache; - public FlightSQLExample(Location location) { + public FlightSqlExample(Location location) { removeDerbyDatabaseIfExists(); populateDerbyDatabase(); @@ -142,7 +140,7 @@ public FlightSQLExample(Location location) { } @Override - public void getTables(FlightSQL.ActionGetTablesRequest request, CallContext context, + public void getTables(FlightSql.ActionGetTablesRequest request, CallContext context, StreamListener listener) { try { final String catalog = (request.getCatalog().isEmpty() ? null : request.getCatalog()); @@ -195,7 +193,7 @@ private Result getTableResult(final ResultSet tables, boolean includeSchema) thr } @Override - public void getPreparedStatement(FlightSQL.ActionGetPreparedStatementRequest request, CallContext context, + public void getPreparedStatement(FlightSql.ActionGetPreparedStatementRequest request, CallContext context, StreamListener listener) { final PreparedStatementCacheKey handle = new PreparedStatementCacheKey( UUID.randomUUID().toString(), request.getQuery()); @@ -341,7 +339,7 @@ private int readBatch(ResultSet resultSet, ResultSetMetaData resultSetMetaData, @Override - public void closePreparedStatement(FlightSQL.ActionClosePreparedStatementRequest request, CallContext context, + public void closePreparedStatement(FlightSql.ActionClosePreparedStatementRequest request, CallContext context, StreamListener listener) { try { preparedStatementLoadingCache.invalidate( @@ -545,14 +543,14 @@ public void getSqlInfo(CallContext context, StreamListener listener) { } @Override - public void getCatalogs(FlightSQL.ActionGetCatalogsRequest request, CallContext context, + public void getCatalogs(FlightSql.ActionGetCatalogsRequest request, CallContext context, StreamListener listener) { // TODO - build example implementation throw Status.UNIMPLEMENTED.asRuntimeException(); } @Override - public void getSchemas(FlightSQL.ActionGetSchemasRequest request, CallContext context, + public void getSchemas(FlightSql.ActionGetSchemasRequest request, CallContext context, StreamListener listener) { // TODO - build example implementation throw Status.UNIMPLEMENTED.asRuntimeException(); diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementCacheKey.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementCacheKey.java index 9c56e3162d2..cc8db427b55 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementCacheKey.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/PreparedStatementCacheKey.java @@ -19,7 +19,7 @@ import java.util.Objects; -import org.apache.arrow.flight.sql.impl.FlightSQLExample.PreparedStatementHandle; +import org.apache.arrow.flight.sql.impl.FlightSqlExample.PreparedStatementHandle; import org.apache.arrow.util.Preconditions; import com.google.protobuf.Any; @@ -45,7 +45,7 @@ String getSql() { } ByteString toProtocol() { - return Any.pack(org.apache.arrow.flight.sql.impl.FlightSQLExample.PreparedStatementHandle + return Any.pack(org.apache.arrow.flight.sql.impl.FlightSqlExample.PreparedStatementHandle .newBuilder() .setSql(getSql()) .setUuid(getUuid()) diff --git a/java/flight/flight-sql/src/test/protobuf/flightSQLExample.proto b/java/flight/flight-sql/src/test/protobuf/flightSqlExample.proto similarity index 100% rename from java/flight/flight-sql/src/test/protobuf/flightSQLExample.proto rename to java/flight/flight-sql/src/test/protobuf/flightSqlExample.proto From aced0c2c83c6893bb6311cb4857ca3d4b7915756 Mon Sep 17 00:00:00 2001 From: Kyle Porter Date: Tue, 6 Jul 2021 16:28:54 -0700 Subject: [PATCH 15/16] Additional CR changes. Note - FlightSqlExample is not functional and needs to be updated. --- .../arrow/adapter/jdbc/JdbcToArrowConfig.java | 17 +- .../arrow/flight/sql/FlightSqlClient.java | 323 ++++++++++++++++++ .../flight/sql/FlightSqlClientUtils.java | 206 ----------- .../arrow/flight/sql/FlightSqlProducer.java | 20 +- .../arrow/flight/sql/FlightSqlUtils.java | 83 ----- .../apache/arrow/flight/TestFlightSql.java | 205 +++-------- .../arrow/flight/sql/FlightSqlExample.java | 87 ++++- 7 files changed, 470 insertions(+), 471 deletions(-) create mode 100644 java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java delete mode 100644 java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClientUtils.java diff --git a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java index 9a587e7e44b..b1a41dc4cd8 100644 --- a/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java +++ b/java/adapter/jdbc/src/main/java/org/apache/arrow/adapter/jdbc/JdbcToArrowConfig.java @@ -127,6 +127,8 @@ public final class JdbcToArrowConfig { *

  • TIMESTAMP --> ArrowType.Timestamp(TimeUnit.MILLISECOND, calendar timezone)
  • *
  • CLOB --> ArrowType.Utf8
  • *
  • BLOB --> ArrowType.Binary
  • + *
  • ARRAY --> ArrowType.List
  • + *
  • STRUCT --> ArrowType.Struct
  • *
  • NULL --> ArrowType.Null
  • * */ @@ -149,13 +151,6 @@ public final class JdbcToArrowConfig { // set up type converter this.jdbcToArrowTypeConverter = jdbcToArrowTypeConverter != null ? jdbcToArrowTypeConverter : fieldInfo -> { - final String timezone; - if (calendar != null) { - timezone = calendar.getTimeZone().getID(); - } else { - timezone = null; - } - switch (fieldInfo.getJdbcType()) { case Types.BOOLEAN: case Types.BIT: @@ -191,6 +186,12 @@ public final class JdbcToArrowConfig { case Types.TIME: return new ArrowType.Time(TimeUnit.MILLISECOND, 32); case Types.TIMESTAMP: + final String timezone; + if (calendar != null) { + timezone = calendar.getTimeZone().getID(); + } else { + timezone = null; + } return new ArrowType.Timestamp(TimeUnit.MILLISECOND, timezone); case Types.BINARY: case Types.VARBINARY: @@ -201,6 +202,8 @@ public final class JdbcToArrowConfig { return new ArrowType.List(); case Types.NULL: return new ArrowType.Null(); + case Types.STRUCT: + return new ArrowType.Struct(); default: // no-op, shouldn't get here return null; diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java new file mode 100644 index 00000000000..3ce88b48dd3 --- /dev/null +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.arrow.flight.sql; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.arrow.flight.Action; +import org.apache.arrow.flight.CallOption; +import org.apache.arrow.flight.FlightClient; +import org.apache.arrow.flight.FlightDescriptor; +import org.apache.arrow.flight.FlightInfo; +import org.apache.arrow.flight.FlightStream; +import org.apache.arrow.flight.Result; +import org.apache.arrow.flight.SchemaResult; +import org.apache.arrow.flight.Ticket; +import org.apache.arrow.flight.sql.impl.FlightSql; +import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery; +import org.apache.arrow.vector.types.pojo.Schema; + +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; + +import io.grpc.Status; + +/** + * Flight client with Flight SQL semantics. + */ +public class FlightSqlClient { + private FlightClient client; + + public FlightSqlClient(FlightClient client) { + this.client = client; + } + + /** + * Execute a query on the server. + * + * @param query The query to execute. + * @return a FlightInfo object representing the stream(s) to fetch. + */ + public FlightInfo execute(String query) { + final FlightSql.CommandStatementQuery.Builder builder = FlightSql.CommandStatementQuery.newBuilder(); + builder.setQuery(query); + final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); + return client.getInfo(descriptor); + } + + /** + * Execute an update query on the server. + * + * @param query The query to execute. + * @return a FlightInfo object representing the stream(s) to fetch. + */ + public long executeUpdate(String query) { + final FlightSql.CommandStatementUpdate.Builder builder = FlightSql.CommandStatementUpdate.newBuilder(); + builder.setQuery(query); + return 0; // TODO + } + + /** + * Request a list of catalogs. + * + * @return a FlightInfo object representing the stream(s) to fetch. + */ + public FlightInfo getCatalogs() { + final FlightSql.CommandGetCatalogs.Builder builder = FlightSql.CommandGetCatalogs.newBuilder(); + final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); + return client.getInfo(descriptor); + } + + /** + * Request a list of schemas. + * + * @param catalog The catalog. + * @param schemaFilterPattern The schema filter pattern. + * @return a FlightInfo object representing the stream(s) to fetch. + */ + public FlightInfo getSchemas(String catalog, String schemaFilterPattern) { + final FlightSql.CommandGetSchemas.Builder builder = FlightSql.CommandGetSchemas.newBuilder(); + + if (catalog != null) { + builder.setCatalog(catalog); + } + + if (schemaFilterPattern != null) { + builder.setSchemaFilterPattern(schemaFilterPattern); + } + + final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); + return client.getInfo(descriptor); + } + + /** + * Get schema for a stream. + * @param descriptor The descriptor for the stream. + * @param options RPC-layer hints for this call. + */ + public SchemaResult getSchema(FlightDescriptor descriptor, CallOption... options) { + return this.client.getSchema(descriptor, options); + } + + /** + * Retrieve a stream from the server. + * @param ticket The ticket granting access to the data stream. + * @param options RPC-layer hints for this call. + */ + public FlightStream getStream(Ticket ticket, CallOption... options) { + return this.client.getStream(ticket, options); + } + + /** + * Request a set of Flight SQL metadata. + * + * @param info The set of metadata to retrieve. None to retrieve all metadata. + * @return a FlightInfo object representing the stream(s) to fetch. + */ + public FlightInfo getSqlInfo(String... info) { + final FlightSql.CommandGetSqlInfo.Builder builder = FlightSql.CommandGetSqlInfo.newBuilder(); + + if (info != null && 0 != info.length) { + builder.addAllInfo(Arrays.asList(info)); + } + + final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); + return client.getInfo(descriptor); + } + + /** + * Request a list of tables. + * + * @param catalog The catalog. + * @param schemaFilterPattern The schema filter pattern. + * @param tableFilterPattern The table filter pattern. + * @param tableTypes The table types to include. + * @param includeSchema True to include the schema upon return, false to not include the schema. + * @return a FlightInfo object representing the stream(s) to fetch. + */ + public FlightInfo getTables(String catalog, String schemaFilterPattern, + String tableFilterPattern, List tableTypes, boolean includeSchema) { + + final FlightSql.CommandGetTables.Builder builder = FlightSql.CommandGetTables.newBuilder(); + + if (catalog != null) { + builder.setCatalog(catalog); + } + + if (schemaFilterPattern != null) { + builder.setSchemaFilterPattern(schemaFilterPattern); + } + + if (tableFilterPattern != null) { + builder.setTableNameFilterPattern(tableFilterPattern); + } + + if (tableTypes != null) { + builder.addAllTableTypes(tableTypes); + } + builder.setIncludeSchema(includeSchema); + + final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); + return client.getInfo(descriptor); + } + + /** + * Request a list of table types. + * + * @return a FlightInfo object representing the stream(s) to fetch. + */ + public FlightInfo getTableTypes() { + final FlightSql.CommandGetTableTypes.Builder builder = FlightSql.CommandGetTableTypes.newBuilder(); + final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); + return client.getInfo(descriptor); + } + + /** + * Create a prepared statement on the server. + * + * @param query The query to prepare. + * @return The representation of the prepared statement which exists on the server. + */ + public PreparedStatement prepare(String query) { + return new PreparedStatement(client, query); + } + + /** + * Helper class to encapsulate Flight SQL prepared statement logic. + */ + public static class PreparedStatement implements Closeable { + private final FlightClient client; + private final ActionCreatePreparedStatementResult preparedStatementResult; + private AtomicLong invocationCount; + private boolean isClosed; + private Schema resultSetSchema = null; + private Schema parameterSchema = null; + + /** + * Constructor. + * + * @param client The client. FlightSqlPreparedStatement does not maintain this resource. + * @param sql The query. + */ + public PreparedStatement(FlightClient client, String sql) { + this.client = client; + + final Iterator preparedStatementResults = client.doAction(new Action( + FlightSqlUtils.FLIGHT_SQL_CREATEPREPAREDSTATEMENT.getType(), + Any.pack(FlightSql.ActionCreatePreparedStatementRequest + .newBuilder() + .setQuery(sql) + .build()) + .toByteArray())); + + preparedStatementResult = FlightSqlUtils.unpackAndParseOrThrow( + preparedStatementResults.next().getBody(), + ActionCreatePreparedStatementResult.class); + + invocationCount = new AtomicLong(0); + isClosed = false; + } + + /** + * Returns the Schema of the resultset. + * + * @return the Schema of the resultset. + */ + public Schema getResultSetSchema() { + if (resultSetSchema == null && preparedStatementResult.getDatasetSchema() != null) { + resultSetSchema = Schema.deserialize(preparedStatementResult.getDatasetSchema().asReadOnlyByteBuffer()); + } + return resultSetSchema; + } + + /** + * Returns the Schema of the parameters. + * + * @return the Schema of the parameters. + */ + public Schema getParameterSchema() { + if (parameterSchema == null && preparedStatementResult.getParameterSchema() != null) { + parameterSchema = Schema.deserialize(preparedStatementResult.getParameterSchema().asReadOnlyByteBuffer()); + } + return parameterSchema; + } + + /** + * Executes the prepared statement query on the server. + * + * @return a FlightInfo object representing the stream(s) to fetch. + * @throws IOException if the PreparedStatement is closed. + */ + public FlightInfo execute() throws IOException { + if (isClosed) { + throw new IllegalStateException("Prepared statement has already been closed on the server."); + } + + final FlightDescriptor descriptor = FlightDescriptor + .command(Any.pack(CommandPreparedStatementQuery.newBuilder() + .setClientExecutionHandle( + ByteString.copyFrom(ByteBuffer.allocate(Long.BYTES).putLong(invocationCount.getAndIncrement()))) + .setPreparedStatementHandle(preparedStatementResult.getPreparedStatementHandle()) + .build()) + .toByteArray()); + + return client.getInfo(descriptor); + } + + /** + * Executes the prepared statement update on the server. + * + * @return the number of rows updated. + */ + public long executeUpdate() { + throw Status.UNIMPLEMENTED.asRuntimeException(); + } + + // TODO: Set parameter values + + @Override + public void close() { + isClosed = true; + final Iterator closePreparedStatementResults = client.doAction(new Action( + FlightSqlUtils.FLIGHT_SQL_CLOSEPREPAREDSTATEMENT.getType(), + Any.pack(FlightSql.ActionClosePreparedStatementRequest + .newBuilder() + .setPreparedStatementHandleBytes(preparedStatementResult.getPreparedStatementHandle()) + .build()) + .toByteArray())); + closePreparedStatementResults.forEachRemaining(result -> { + }); + } + + /** + * Returns if the prepared statement is already closed. + * + * @return true if the prepared statement is already closed. + */ + public boolean isClosed() { + return isClosed; + } + } +} diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClientUtils.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClientUtils.java deleted file mode 100644 index f93c242312e..00000000000 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClientUtils.java +++ /dev/null @@ -1,206 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.arrow.flight.sql; - -import java.io.Closeable; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Iterator; -import java.util.List; - -import org.apache.arrow.flight.Action; -import org.apache.arrow.flight.FlightClient; -import org.apache.arrow.flight.FlightDescriptor; -import org.apache.arrow.flight.FlightInfo; -import org.apache.arrow.flight.Result; -import org.apache.arrow.flight.sql.impl.FlightSql; -import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult; -import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery; -import org.apache.arrow.vector.types.pojo.Schema; - -import com.google.protobuf.Any; -import com.google.protobuf.ByteString; - -import io.grpc.Status; - -/** - * Client side utilities to work with Flight SQL semantics. - */ -public final class FlightSqlClientUtils { - - /** - * Helper method to request a list of tables from a Flight SQL enabled endpoint. - * - * @param client The Flight Client. - * @param catalog The catalog. - * @param schemaFilterPattern The schema filter pattern. - * @param tableFilterPattern The table filter pattern. - * @param tableTypes The table types to include. - * @param includeSchema True to include the schema upon return, false to not include the schema. - * @return a FlightInfo object representing the stream(s) to fetch. - */ - public static FlightInfo getTables(FlightClient client, String catalog, String schemaFilterPattern, - String tableFilterPattern, List tableTypes, boolean includeSchema) { - - final FlightSql.CommandGetTables.Builder builder = FlightSql.CommandGetTables.newBuilder(); - - if (catalog != null) { - builder.setCatalog(catalog); - } - - if (schemaFilterPattern != null) { - builder.setSchemaFilterPattern(schemaFilterPattern); - } - - if (tableFilterPattern != null) { - builder.setTableNameFilterPattern(tableFilterPattern); - } - - if (tableTypes != null) { - builder.addAllTableTypes(tableTypes); - } - builder.setIncludeSchema(includeSchema); - - final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); - return client.getInfo(descriptor); - } - - /** - * Helper method to create a prepared statement on the server. - * - * @param client The Flight Client. - * @param query The query to prepare. - * @return Metadata and handles to the prepared statement which exists on the server. - */ - public static FlightSqlPreparedStatement getPreparedStatement(FlightClient client, String query) { - return new FlightSqlPreparedStatement(client, query); - } - - /** - * Helper class to encapsulate Flight SQL prepared statement logic. - */ - public static class FlightSqlPreparedStatement implements Closeable { - private final FlightClient client; - private final ActionCreatePreparedStatementResult preparedStatementResult; - private long invocationCount; - private boolean isClosed; - private Schema resultSetSchema = null; - private Schema parameterSchema = null; - - /** - * Constructor. - * - * @param client The client. FlightSqlPreparedStatement does not maintain this resource. - * @param sql The query. - */ - public FlightSqlPreparedStatement(FlightClient client, String sql) { - this.client = client; - - final Iterator preparedStatementResults = client.doAction(new Action("GetPreparedStatement", - Any.pack(FlightSql.ActionCreatePreparedStatementRequest - .newBuilder() - .setQuery(sql) - .build()) - .toByteArray())); - - preparedStatementResult = FlightSqlUtils.unpackAndParseOrThrow( - preparedStatementResults.next().getBody(), - ActionCreatePreparedStatementResult.class); - - invocationCount = 0; - isClosed = false; - } - - /** - * Returns the Schema of the resultset. - * - * @return the Schema of the resultset. - */ - public Schema getResultSetSchema() { - if (resultSetSchema == null && preparedStatementResult.getDatasetSchema() != null) { - resultSetSchema = Schema.deserialize(preparedStatementResult.getDatasetSchema().asReadOnlyByteBuffer()); - } - return resultSetSchema; - } - - /** - * Returns the Schema of the parameters. - * - * @return the Schema of the parameters. - */ - public Schema getParameterSchema() { - if (parameterSchema == null && preparedStatementResult.getParameterSchema() != null) { - parameterSchema = Schema.deserialize(preparedStatementResult.getParameterSchema().asReadOnlyByteBuffer()); - } - return parameterSchema; - } - - /** - * Executes the prepared statement query on the server. - * - * @return a FlightInfo object representing the stream(s) to fetch. - * @throws IOException if the PreparedStatement is closed. - */ - public FlightInfo executeQuery() throws IOException { - if (isClosed) { - throw new IOException("Prepared statement has already been closed on the server."); - } - - final FlightDescriptor descriptor = FlightDescriptor - .command(Any.pack(CommandPreparedStatementQuery.newBuilder() - .setClientExecutionHandle( - ByteString.copyFrom(ByteBuffer.allocate(Long.BYTES).putLong(invocationCount++))) - .setPreparedStatementHandle(preparedStatementResult.getPreparedStatementHandle()) - .build()) - .toByteArray()); - - return client.getInfo(descriptor); - } - - /** - * Executes the prepared statement update on the server. - * - * @return the number of rows updated. - */ - public long executeUpdate() { - throw Status.UNIMPLEMENTED.asRuntimeException(); - } - - @Override - public void close() { - isClosed = true; - final Iterator closePreparedStatementResults = client.doAction(new Action("ClosePreparedStatement", - Any.pack(FlightSql.ActionClosePreparedStatementRequest - .newBuilder() - .setPreparedStatementHandleBytes(preparedStatementResult.getPreparedStatementHandle()) - .build()) - .toByteArray())); - closePreparedStatementResults.forEachRemaining(result -> { - }); - } - - /** - * Returns if the prepared statement is already closed. - * - * @return true if the prepared statement is already closed. - */ - public boolean isClosed() { - return isClosed; - } - } -} diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java index 5e4f76f1a81..96ecd785516 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java @@ -183,18 +183,16 @@ public Runnable acceptPut(CallContext context, FlightStream flightStream, Stream if (command.is(CommandStatementUpdate.class)) { return acceptPutStatement( - FlightSqlUtils.unpackOrThrow(command, CommandStatementUpdate.class), - context, flightStream, ackStream); - + FlightSqlUtils.unpackOrThrow(command, CommandStatementUpdate.class), + context, flightStream, ackStream); } else if (command.is(CommandPreparedStatementUpdate.class)) { return acceptPutPreparedStatementUpdate( - FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementUpdate.class), - context, flightStream, ackStream); - + FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementUpdate.class), + context, flightStream, ackStream); } else if (command.is(CommandPreparedStatementQuery.class)) { return acceptPutPreparedStatementQuery( - FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), - context, flightStream, ackStream); + FlightSqlUtils.unpackOrThrow(command, CommandPreparedStatementQuery.class), + context, flightStream, ackStream); } throw Status.INVALID_ARGUMENT.asRuntimeException(); @@ -223,13 +221,15 @@ public void listActions(CallContext context, StreamListener listener public void doAction(CallContext context, Action action, StreamListener listener) { if (action.getType().equals(FlightSqlUtils.FLIGHT_SQL_CREATEPREPAREDSTATEMENT.getType())) { final ActionCreatePreparedStatementRequest request = FlightSqlUtils.unpackAndParseOrThrow(action.getBody(), - ActionCreatePreparedStatementRequest.class); + ActionCreatePreparedStatementRequest.class); createPreparedStatement(request, context, listener); } else if (action.getType().equals(FlightSqlUtils.FLIGHT_SQL_CLOSEPREPAREDSTATEMENT.getType())) { final ActionClosePreparedStatementRequest request = FlightSqlUtils.unpackAndParseOrThrow(action.getBody(), - ActionClosePreparedStatementRequest.class); + ActionClosePreparedStatementRequest.class); closePreparedStatement(request, context, listener); } + + throw Status.INVALID_ARGUMENT.asRuntimeException(); } /** diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java index f390845c296..9bafca17d8b 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlUtils.java @@ -17,14 +17,9 @@ package org.apache.arrow.flight.sql; -import java.sql.Types; import java.util.List; import org.apache.arrow.flight.ActionType; -import org.apache.arrow.vector.types.DateUnit; -import org.apache.arrow.vector.types.FloatingPointPrecision; -import org.apache.arrow.vector.types.TimeUnit; -import org.apache.arrow.vector.types.pojo.ArrowType; import com.google.common.collect.ImmutableList; import com.google.protobuf.Any; @@ -35,14 +30,6 @@ * Utilities to work with Flight SQL semantics. */ public final class FlightSqlUtils { - - private static final int BIT_WIDTH8 = 8; - private static final int BIT_WIDTH_16 = 16; - private static final int BIT_WIDTH_32 = 32; - private static final int BIT_WIDTH_64 = 64; - private static final boolean IS_SIGNED_FALSE = false; - private static final boolean IS_SIGNED_TRUE = true; - public static final ActionType FLIGHT_SQL_CREATEPREPAREDSTATEMENT = new ActionType("CreatePreparedStatement", "Creates a reusable prepared statement resource on the server. \n" + "Request Message: ActionCreatePreparedStatementRequest\n" + @@ -58,76 +45,6 @@ public final class FlightSqlUtils { FLIGHT_SQL_CLOSEPREPAREDSTATEMENT ); - /** - * Converts {@link java.sql.Types} values returned from JDBC Apis to Arrow types. - * - * @param jdbcDataType {@link java.sql.Types} value. - * @param precision Precision of the type. - * @param scale Scale of the type. - * @return The Arrow equivalent type. - */ - public static ArrowType getArrowTypeFromJDBCType(int jdbcDataType, int precision, int scale) { - - switch (jdbcDataType) { - case Types.BIT: - case Types.BOOLEAN: - return ArrowType.Bool.INSTANCE; - case Types.TINYINT: - return new ArrowType.Int(BIT_WIDTH8, IS_SIGNED_TRUE); - case Types.SMALLINT: - return new ArrowType.Int(BIT_WIDTH_16, IS_SIGNED_TRUE); - case Types.INTEGER: - return new ArrowType.Int(BIT_WIDTH_32, IS_SIGNED_TRUE); - case Types.BIGINT: - return new ArrowType.Int(BIT_WIDTH_64, IS_SIGNED_TRUE); - case Types.FLOAT: - case Types.REAL: - return new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE); - case Types.DOUBLE: - return new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE); - case Types.NUMERIC: - case Types.DECIMAL: - return new ArrowType.Decimal(precision, scale); - case Types.DATE: - return new ArrowType.Date(DateUnit.DAY); - case Types.TIME: - return new ArrowType.Time(TimeUnit.MILLISECOND, BIT_WIDTH_32); - case Types.TIMESTAMP: - return new ArrowType.Timestamp(TimeUnit.MILLISECOND, null); - case Types.BINARY: - case Types.VARBINARY: - case Types.LONGVARBINARY: - return ArrowType.Binary.INSTANCE; - case Types.NULL: - return ArrowType.Null.INSTANCE; - - case Types.CHAR: - case Types.VARCHAR: - case Types.LONGVARCHAR: - case Types.CLOB: - case Types.NCHAR: - case Types.NVARCHAR: - case Types.LONGNVARCHAR: - case Types.NCLOB: - - case Types.OTHER: - case Types.JAVA_OBJECT: - case Types.DISTINCT: - case Types.STRUCT: - case Types.ARRAY: - case Types.BLOB: - case Types.REF: - case Types.DATALINK: - case Types.ROWID: - case Types.SQLXML: - case Types.REF_CURSOR: - case Types.TIME_WITH_TIMEZONE: - case Types.TIMESTAMP_WITH_TIMEZONE: - default: - return ArrowType.Utf8.INSTANCE; - } - } - /** * Helper to parse {@link com.google.protobuf.Any} objects to the specific protobuf object. * diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java index 7767bd9e907..69300ceab89 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/TestFlightSql.java @@ -17,42 +17,29 @@ package org.apache.arrow.flight; -import static org.apache.arrow.flight.sql.FlightSqlClientUtils.getPreparedStatement; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.nio.charset.StandardCharsets; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.Iterator; import java.util.List; -import org.apache.arrow.flight.sql.FlightSqlClientUtils; +import org.apache.arrow.flight.sql.FlightSqlClient; import org.apache.arrow.flight.sql.FlightSqlExample; -import org.apache.arrow.flight.sql.impl.FlightSql.ActionClosePreparedStatementRequest; -import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementRequest; -import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementResult; -import org.apache.arrow.flight.sql.impl.FlightSql.CommandPreparedStatementQuery; import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.memory.RootAllocator; -import org.apache.arrow.memory.util.ArrowBufPointer; import org.apache.arrow.util.AutoCloseables; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.Types; import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.FieldType; import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.arrow.vector.util.ElementAddressableVectorIterator; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; - -import com.google.protobuf.Any; -import com.google.protobuf.ByteString; +import org.junit.jupiter.api.Assertions; /** * Test direct usage of Flight SQL workflows. @@ -62,14 +49,11 @@ public class TestFlightSql { private static FlightServer server; private static FlightClient client; + private static FlightSqlClient sqlClient; protected static final Schema SCHEMA_INT_TABLE = new Schema(Arrays.asList( - new Field("KEYNAME", new - FieldType(true, ArrowType.Utf8.INSTANCE, null), - null), - new Field("VALUE", - new FieldType(true, new ArrowType.Int(32, true), null), - null))); + new Field("KEYNAME", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null), + new Field("VALUE", FieldType.nullable(Types.MinorType.INT.getType()), null))); @BeforeClass public static void setUp() throws Exception { @@ -81,6 +65,7 @@ public static void setUp() throws Exception { final Location clientLocation = Location.forGrpcInsecure(FlightTestUtil.LOCALHOST, server.getPort()); client = FlightClient.builder(allocator, clientLocation).build(); + sqlClient = new FlightSqlClient(client); } @AfterClass @@ -88,173 +73,73 @@ public static void tearDown() throws Exception { AutoCloseables.close(client, server, allocator); } - /* @Test public void testGetTables() throws Exception { - // Arrange - final ActionGetTablesResult expected = ActionGetTablesResult.newBuilder() - .setSchema("APP") - .setTable("INTTABLE") - .setTableType("TABLE") - .setArrowMetadata(ByteString.copyFrom(SCHEMA_INT_TABLE.toByteArray())) - .build(); - - // Act - final Iterator results = client.doAction(new Action("GetTables", - Any.pack(ActionGetTablesRequest - .newBuilder() - .addTableTypes("TABLE") - .setIncludeSchema(true) - .build()) - .toByteArray())); - - // Assert - while (results.hasNext()) { - ActionGetTablesResult actual = Any.parseFrom(results.next().getBody()).unpack(ActionGetTablesResult.class); - assertEquals(expected, actual); + final FlightInfo info = sqlClient.getTables(null, null, null, null, false); + try (final FlightStream stream = sqlClient.getStream(info.getEndpoints().get(0).getTicket())) { + final List> results = getResults(stream); + Assertions.assertEquals(1, results.size()); + Assertions.assertEquals( + Arrays.asList(null, "APP", "INTTABLE", "TABLE", SCHEMA_INT_TABLE.toJson()), results.get(0)); } } - @Test - public void testGetTablesWithFlightSqlClientUtils() throws Exception { - // Arrange - final ActionGetTablesResult expected = ActionGetTablesResult.newBuilder() - .setSchema("APP") - .setTable("INTTABLE") - .setTableType("TABLE") - .setArrowMetadata(ByteString.copyFrom(SCHEMA_INT_TABLE.toByteArray())) - .build(); - - // Act - final List results = getTables(client, null, null, null, - Collections.singletonList("TABLE"), true); - - // Assert - assertEquals(1, results.size()); - assertEquals(expected, results.get(0)); - } - */ - @Test public void testSimplePrepStmt() throws Exception { - final Iterator preparedStatementResults = client.doAction(new Action("GetPreparedStatement", - Any.pack(ActionCreatePreparedStatementRequest - .newBuilder() - .setQuery("Select * from intTable") - .build()) - .toByteArray())); - - assertTrue(preparedStatementResults.hasNext()); - final ActionCreatePreparedStatementResult preparedStatementResult = - Any.parseFrom(preparedStatementResults.next().getBody()).unpack(ActionCreatePreparedStatementResult.class); - assertFalse(preparedStatementResults.hasNext()); - - final Schema actualSchema = Schema.deserialize(preparedStatementResult.getDatasetSchema().asReadOnlyByteBuffer()); - assertEquals(SCHEMA_INT_TABLE, actualSchema); - - final FlightDescriptor descriptor = FlightDescriptor - .command(Any.pack(CommandPreparedStatementQuery.newBuilder() - .setClientExecutionHandle(ByteString.copyFrom(new byte[]{1, 2, 3, 4})) - .setPreparedStatementHandle(preparedStatementResult.getPreparedStatementHandle()) - .build()) - .toByteArray()); + final FlightSqlClient.PreparedStatement preparedStatement = sqlClient.prepare("Select * from intTable"); - final FlightInfo info = client.getInfo(descriptor); - assertEquals(SCHEMA_INT_TABLE, info.getSchema()); - - final FlightStream stream = client.getStream(info.getEndpoints().get(0).getTicket()); - assertEquals(SCHEMA_INT_TABLE, stream.getSchema()); - - List actualStringResults = new ArrayList<>(); - List actualIntResults = new ArrayList<>(); - while (stream.next()) { - final VectorSchemaRoot root = stream.getRoot(); - final long rowCount = root.getRowCount(); - - for (Field field : root.getSchema().getFields()) { - final FieldVector fieldVector = root.getVector(field.getName()); - - if (fieldVector instanceof VarCharVector) { + final Schema actualSchema = preparedStatement.getResultSetSchema(); + Assertions.assertEquals(SCHEMA_INT_TABLE, actualSchema); - final ElementAddressableVectorIterator it = - new ElementAddressableVectorIterator<>((VarCharVector) fieldVector); + final FlightInfo info = preparedStatement.execute(); + Assertions.assertEquals(SCHEMA_INT_TABLE, info.getSchema()); - for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { - final ArrowBufPointer pt = it.next(); - final byte[] bytes = new byte[(int) pt.getLength()]; - pt.getBuf().getBytes(pt.getOffset(), bytes); + try (final FlightStream stream = sqlClient.getStream(info.getEndpoints().get(0).getTicket())) { + Assertions.assertEquals(SCHEMA_INT_TABLE, stream.getSchema()); - actualStringResults.add(new String(bytes, StandardCharsets.UTF_8)); - } - } else if (fieldVector instanceof IntVector) { - for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { - actualIntResults.add(((IntVector) fieldVector).get(rowIndex)); - } - } - } + final List> results = getResults(stream); + Assertions.assertEquals(3, results.size()); + Assertions.assertEquals(Arrays.asList("one", "1"), results.get(0)); + Assertions.assertEquals(Arrays.asList("zero", "0"), results.get(1)); + Assertions.assertEquals(Arrays.asList("negative one", "-1"), results.get(2)); } - stream.getRoot().clear(); - - assertEquals(Arrays.asList("one", "zero", "negative one"), actualStringResults); - assertEquals(Arrays.asList(1, 0, -1), actualIntResults); - final Iterator closePreparedStatementResults = client.doAction(new Action("ClosePreparedStatement", - Any.pack(ActionClosePreparedStatementRequest - .newBuilder() - .setPreparedStatementHandleBytes(preparedStatementResult.getPreparedStatementHandle()) - .build()) - .toByteArray())); - assertFalse(closePreparedStatementResults.hasNext()); + AutoCloseables.close(preparedStatement); + Assertions.assertTrue(preparedStatement.isClosed()); } - @Test - public void testSimplePrepStmtWithFlightSqlClientUtils() throws Exception { - final FlightSqlClientUtils.FlightSqlPreparedStatement preparedStatement = - getPreparedStatement(client, "Select * from intTable"); - - final Schema actualSchema = preparedStatement.getResultSetSchema(); - assertEquals(SCHEMA_INT_TABLE, actualSchema); - - final FlightInfo info = preparedStatement.executeQuery(); - assertEquals(SCHEMA_INT_TABLE, info.getSchema()); - - final FlightStream stream = client.getStream(info.getEndpoints().get(0).getTicket()); - assertEquals(SCHEMA_INT_TABLE, stream.getSchema()); - - List actualStringResults = new ArrayList<>(); - List actualIntResults = new ArrayList<>(); + List> getResults(FlightStream stream) { + final List> results = new ArrayList<>(); while (stream.next()) { final VectorSchemaRoot root = stream.getRoot(); final long rowCount = root.getRowCount(); + for (int i = 0; i < rowCount; ++i) { + results.add(new ArrayList<>()); + } for (Field field : root.getSchema().getFields()) { final FieldVector fieldVector = root.getVector(field.getName()); if (fieldVector instanceof VarCharVector) { - - final ElementAddressableVectorIterator it = - new ElementAddressableVectorIterator<>((VarCharVector) fieldVector); - + final VarCharVector varcharVector = (VarCharVector) fieldVector; for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { - final ArrowBufPointer pt = it.next(); - final byte[] bytes = new byte[(int) pt.getLength()]; - pt.getBuf().getBytes(pt.getOffset(), bytes); - - actualStringResults.add(new String(bytes, StandardCharsets.UTF_8)); + results.get(rowIndex).add(varcharVector.getObject(rowIndex).toString()); } } else if (fieldVector instanceof IntVector) { for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { - actualIntResults.add(((IntVector) fieldVector).get(rowIndex)); + results.get(rowIndex).add(String.valueOf(((IntVector) fieldVector).get(rowIndex))); + } + } else if (fieldVector instanceof VarBinaryVector) { + final VarBinaryVector varbinaryVector = (VarBinaryVector) fieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + results.get(rowIndex).add(Schema.deserialize(ByteBuffer.wrap(varbinaryVector.get(rowIndex))).toJson()); } + } else { + throw new UnsupportedOperationException("Not yet implemented"); } } } stream.getRoot().clear(); - - assertEquals(Arrays.asList("one", "zero", "negative one"), actualStringResults); - assertEquals(Arrays.asList(1, 0, -1), actualIntResults); - - AutoCloseables.close(preparedStatement); - assertTrue(preparedStatement.isClosed()); + return results; } } diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSqlExample.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSqlExample.java index b6247714bdd..bb0d727db61 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSqlExample.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSqlExample.java @@ -17,8 +17,6 @@ package org.apache.arrow.flight.sql; -import static org.apache.arrow.flight.sql.FlightSqlUtils.getArrowTypeFromJDBCType; - import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -32,6 +30,7 @@ import java.sql.ResultSet; import java.sql.ResultSetMetaData; import java.sql.SQLException; +import java.sql.Types; import java.util.ArrayList; import java.util.Comparator; import java.util.List; @@ -66,6 +65,9 @@ import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.arrow.vector.types.DateUnit; +import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.TimeUnit; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.FieldType; @@ -101,6 +103,12 @@ public class FlightSqlExample extends FlightSqlProducer implements AutoCloseable { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FlightSqlExample.class); + private static final int BIT_WIDTH_8 = 8; + private static final int BIT_WIDTH_16 = 16; + private static final int BIT_WIDTH_32 = 32; + private static final int BIT_WIDTH_64 = 64; + private static final boolean IS_SIGNED_TRUE = true; + private static final int BATCH_ROW_SIZE = 1000; private final Location location; @@ -214,7 +222,7 @@ public void getPreparedStatement(FlightSql.ActionGetPreparedStatementRequest req .build()) .toByteArray())); - } catch (ExecutionException | SQLException e) { + } catch (Throwable e) { listener.onError(e); } finally { listener.onCompleted(); @@ -232,7 +240,7 @@ public FlightInfo getFlightInfoPreparedStatement(CommandPreparedStatementQuery c .of(new FlightEndpoint(new Ticket(Any.pack(command).toByteArray()), location)); return new FlightInfo(schema, descriptor, endpoints, -1, -1); - } catch (ExecutionException | SQLException e) { + } catch (Throwable e) { logger.error("There was a problem executing the prepared statement", e); throw new FlightRuntimeException(new CallStatus(FlightStatusCode.INTERNAL, e, e.getMessage(), null)); } @@ -294,7 +302,7 @@ public void getStreamPreparedStatement(CommandPreparedStatementQuery command, Ca listener.putNext(); } } - } catch (ExecutionException | SQLException e) { + } catch (Throwable e) { listener.error(e); } finally { listener.completed(); @@ -596,4 +604,73 @@ public void getStreamStatement(CommandStatementQuery command, CallContext contex throw Status.UNIMPLEMENTED.asRuntimeException(); } + + /** + * Converts {@link java.sql.Types} values returned from JDBC Apis to Arrow types. + * + * @param jdbcDataType {@link java.sql.Types} value. + * @param precision Precision of the type. + * @param scale Scale of the type. + * @return The Arrow equivalent type. + */ + static ArrowType getArrowTypeFromJDBCType(int jdbcDataType, int precision, int scale) { + switch (jdbcDataType) { + case Types.BIT: + case Types.BOOLEAN: + return ArrowType.Bool.INSTANCE; + case Types.TINYINT: + return new ArrowType.Int(BIT_WIDTH_8, IS_SIGNED_TRUE); + case Types.SMALLINT: + return new ArrowType.Int(BIT_WIDTH_16, IS_SIGNED_TRUE); + case Types.INTEGER: + return new ArrowType.Int(BIT_WIDTH_32, IS_SIGNED_TRUE); + case Types.BIGINT: + return new ArrowType.Int(BIT_WIDTH_64, IS_SIGNED_TRUE); + case Types.FLOAT: + case Types.REAL: + return new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE); + case Types.DOUBLE: + return new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE); + case Types.NUMERIC: + case Types.DECIMAL: + return new ArrowType.Decimal(precision, scale); + case Types.DATE: + return new ArrowType.Date(DateUnit.DAY); + case Types.TIME: + return new ArrowType.Time(TimeUnit.MILLISECOND, BIT_WIDTH_32); + case Types.TIMESTAMP: + return new ArrowType.Timestamp(TimeUnit.MILLISECOND, null); + case Types.BINARY: + case Types.VARBINARY: + case Types.LONGVARBINARY: + return ArrowType.Binary.INSTANCE; + case Types.NULL: + return ArrowType.Null.INSTANCE; + + case Types.CHAR: + case Types.VARCHAR: + case Types.LONGVARCHAR: + case Types.CLOB: + case Types.NCHAR: + case Types.NVARCHAR: + case Types.LONGNVARCHAR: + case Types.NCLOB: + + case Types.OTHER: + case Types.JAVA_OBJECT: + case Types.DISTINCT: + case Types.STRUCT: + case Types.ARRAY: + case Types.BLOB: + case Types.REF: + case Types.DATALINK: + case Types.ROWID: + case Types.SQLXML: + case Types.REF_CURSOR: + case Types.TIME_WITH_TIMEZONE: + case Types.TIMESTAMP_WITH_TIMEZONE: + default: + return ArrowType.Utf8.INSTANCE; + } + } } From 869ad450970a3ca389563caa402ef0db11db056c Mon Sep 17 00:00:00 2001 From: Kyle Porter Date: Wed, 7 Jul 2021 10:06:34 -0700 Subject: [PATCH 16/16] Add support for primary and foreign keys. --- format/FlightSql.proto | 82 +++++++++++++- .../arrow/flight/sql/FlightSqlClient.java | 74 +++++++++++- .../arrow/flight/sql/FlightSqlProducer.java | 105 ++++++++++++++++++ 3 files changed, 256 insertions(+), 5 deletions(-) diff --git a/format/FlightSql.proto b/format/FlightSql.proto index fdf48562d14..4032ff0548a 100644 --- a/format/FlightSql.proto +++ b/format/FlightSql.proto @@ -94,7 +94,7 @@ message CommandGetSqlInfo { } /* - Represents a request to retrieve the list of catalogs on a Flight SQL enabled backend. + * Represents a request to retrieve the list of catalogs on a Flight SQL enabled backend. * Used in the command member of FlightDescriptor for the following RPC calls: * - GetSchema: return the schema of the query. * - GetFlightInfo: execute the catalog metadata request. @@ -156,12 +156,13 @@ message CommandGetSchemas { */ message CommandGetTables { /* - * Specifies the Catalog to search for schemas. + * Specifies the Catalog to search for the tables. * If omitted, then all catalogs are searched. */ string catalog = 1; - /* Specifies a filter pattern for schemas to search for. + /* + * Specifies a filter pattern for schemas to search for. * When no schema_filter_pattern is provided, all schemas matching other filters are searched. * In the pattern string, two special characters can be used to denote matching rules: * - "%" means to match any substring with 0 or more characters. @@ -169,7 +170,8 @@ message CommandGetTables { */ string schema_filter_pattern = 2; - /* Specifies a filter pattern for tables to search for. + /* + * Specifies a filter pattern for tables to search for. * When no table_name_filter_pattern is provided, all tables matching other filters are searched. * In the pattern string, two special characters can be used to denote matching rules: * - "%" means to match any substring with 0 or more characters. @@ -199,6 +201,78 @@ message CommandGetTables { message CommandGetTableTypes { } +/* + * Represents a request to retrieve the primary keys of a table on a Flight SQL enabled backend. + * Used in the command member of FlightDescriptor for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned schema will be: + * < + * catalog_name: utf8, + * schema_name: utf8, + * table_name: utf8, + * column_name: utf8, + * key_sequence: int, + * key_name: utf8 + * > + * The returned data should be ordered by catalog_name, schema_name, table_name, key_name, then key_sequence. + */ +message CommandGetPrimaryKeys { + // Specifies the catalog to search for the table. + string catalog = 1; + + // Specifies the schema to search for the table. + string schema = 2; + + // Specifies the table to get the primary keys for. + string table = 3; +} + +/* + * Represents a request to retrieve the foreign keys of a table on a Flight SQL enabled backend. + * Used in the command member of FlightDescriptor for the following RPC calls: + * - GetSchema: return the schema of the query. + * - GetFlightInfo: execute the catalog metadata request. + * + * The returned schema will be: + * < + * pk_catalog_name: utf8, + * pk_schema_name: utf8, + * pk_table_name: utf8, + * pk_column_name: utf8, + * fk_catalog_name: utf8, + * fk_schema_name: utf8, + * fk_table_name: utf8, + * fk_column_name: utf8, + * key_sequence: int, + * fk_key_name: utf8, + * pk_key_name: utf8, + * update_rule: int, + * delete_rule: int + * > + * The returned data should be ordered by catalog_name, schema_name, table_name, key_name, then key_sequence. + */ +message CommandGetForeignKeys { + // Specifies the catalog to search for the primary key table. + string pk_catalog = 1; + + // Specifies the schema to search for the primary key table. + string pk_schema = 2; + + // Specifies the primary key table to get the foreign keys for. + string pk_table = 3; + + // Specifies the catalog to search for the foreign key table. + string fk_catalog = 4; + + // Specifies the schema to search for the foreign key table. + string fk_schema = 5; + + // Specifies the foreign key table to get the foreign keys for. + string fk_table = 6; +} + // SQL Execution Action Messages /* diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java index 3ce88b48dd3..de0485c9b34 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlClient.java @@ -159,7 +159,6 @@ public FlightInfo getSqlInfo(String... info) { */ public FlightInfo getTables(String catalog, String schemaFilterPattern, String tableFilterPattern, List tableTypes, boolean includeSchema) { - final FlightSql.CommandGetTables.Builder builder = FlightSql.CommandGetTables.newBuilder(); if (catalog != null) { @@ -183,6 +182,79 @@ public FlightInfo getTables(String catalog, String schemaFilterPattern, return client.getInfo(descriptor); } + /** + * Request the primary keys for a table. + * + * @param catalog The catalog. + * @param schema The schema. + * @param table The table. + * @return a FlightInfo object representing the stream(s) to fetch. + */ + public FlightInfo getPrimaryKeys(String catalog, String schema, String table) { + final FlightSql.CommandGetPrimaryKeys.Builder builder = FlightSql.CommandGetPrimaryKeys.newBuilder(); + + if (catalog != null) { + builder.setCatalog(catalog); + } + + if (schema != null) { + builder.setSchema(schema); + } + + builder.setTable(table); + final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); + return client.getInfo(descriptor); + } + + /** + * Request the foreign keys for a table. + * + * One of pkTable or fkTable must be specified, both cannot be null. + * + * @param pkCatalog The primary key table catalog. + * @param pkSchema The primary key table schema. + * @param pkTable The primary key table. + * @param fkCatalog The foreign key table catalog. + * @param fkSchema The foreign key table schema. + * @param fkTable The foreign key table. + * @return a FlightInfo object representing the stream(s) to fetch. + */ + public FlightInfo getForeignKeys(String pkCatalog, String pkSchema, String pkTable, + String fkCatalog, String fkSchema, String fkTable) { + if (null == pkTable && null == fkTable) { + throw Status.INVALID_ARGUMENT.asRuntimeException(); + } + + final FlightSql.CommandGetForeignKeys.Builder builder = FlightSql.CommandGetForeignKeys.newBuilder(); + + if (pkCatalog != null) { + builder.setPkCatalog(pkCatalog); + } + + if (pkSchema != null) { + builder.setPkSchema(pkSchema); + } + + if (pkTable != null) { + builder.setPkTable(pkTable); + } + + if (fkCatalog != null) { + builder.setFkCatalog(fkCatalog); + } + + if (fkSchema != null) { + builder.setFkSchema(fkSchema); + } + + if (fkTable != null) { + builder.setFkTable(fkTable); + } + + final FlightDescriptor descriptor = FlightDescriptor.command(Any.pack(builder.build()).toByteArray()); + return client.getInfo(descriptor); + } + /** * Request a list of table types. * diff --git a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java index 96ecd785516..ed5cdb4d2c2 100644 --- a/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java +++ b/java/flight/flight-sql/src/main/java/org/apache/arrow/flight/sql/FlightSqlProducer.java @@ -33,6 +33,8 @@ import org.apache.arrow.flight.sql.impl.FlightSql.ActionClosePreparedStatementRequest; import org.apache.arrow.flight.sql.impl.FlightSql.ActionCreatePreparedStatementRequest; import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetCatalogs; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetForeignKeys; +import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetPrimaryKeys; import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSchemas; import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetSqlInfo; import org.apache.arrow.flight.sql.impl.FlightSql.CommandGetTableTypes; @@ -90,6 +92,12 @@ public FlightInfo getFlightInfo(CallContext context, FlightDescriptor descriptor } else if (command.is(CommandGetSqlInfo.class)) { return getFlightInfoSqlInfo( FlightSqlUtils.unpackOrThrow(command, CommandGetSqlInfo.class), context, descriptor); + } else if (command.is(CommandPreparedStatementQuery.class)) { + return getFlightInfoPrimaryKeys( + FlightSqlUtils.unpackOrThrow(command, CommandGetPrimaryKeys.class), context, descriptor); + } else if (command.is(CommandGetForeignKeys.class)) { + return getFlightInfoForeignKeys( + FlightSqlUtils.unpackOrThrow(command, CommandGetForeignKeys.class), context, descriptor); } throw Status.INVALID_ARGUMENT.asRuntimeException(); @@ -119,6 +127,10 @@ public SchemaResult getSchema(CallContext context, FlightDescriptor descriptor) return getSchemaTableTypes(); } else if (command.is(CommandGetSqlInfo.class)) { return getSchemaSqlInfo(); + } else if (command.is(CommandGetPrimaryKeys.class)) { + return getSchemaPrimaryKeys(); + } else if (command.is(CommandGetForeignKeys.class)) { + return getSchemaForeignKeys(); } throw Status.INVALID_ARGUMENT.asRuntimeException(); @@ -160,6 +172,10 @@ public void getStream(CallContext context, Ticket ticket, ServerStreamListener l getStreamTableTypes(context, ticket, listener); } else if (command.is(CommandGetSqlInfo.class)) { getStreamSqlInfo(FlightSqlUtils.unpackOrThrow(command, CommandGetSqlInfo.class), context, ticket, listener); + } else if (command.is(CommandGetPrimaryKeys.class)) { + getStreamPrimaryKeys(FlightSqlUtils.unpackOrThrow(command, CommandGetSqlInfo.class), context, ticket, listener); + } else if (command.is(CommandGetForeignKeys.class)) { + getStreamForeignKeys(FlightSqlUtils.unpackOrThrow(command, CommandGetSqlInfo.class), context, ticket, listener); } else { throw Status.INVALID_ARGUMENT.asRuntimeException(); } @@ -538,4 +554,93 @@ public SchemaResult getSchemaTableTypes() { * @param listener An interface for sending data back to the client. */ public abstract void getStreamTableTypes(CallContext context, Ticket ticket, ServerStreamListener listener); + + /** + * Returns the available primary keys by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSql.CommandGetPrimaryKeys} objects in {@link Result} objects. + * + * @param request request filter parameters. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoPrimaryKeys(CommandGetPrimaryKeys request, CallContext context, + FlightDescriptor descriptor); + + /** + * Gets schema about the get primary keys data stream. + * + * @return Schema for the stream. + */ + public SchemaResult getSchemaPrimaryKeys() { + final List fields = new ArrayList<>(); + + fields.add(new Field("catalog_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("schema_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("table_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("column_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("key_sequence", FieldType.nullable(Types.MinorType.INT.getType()), null)); + fields.add(new Field("key_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + + return new SchemaResult(new Schema(fields)); + } + + /** + * Returns data for primary keys based data stream. + * + * @param command The command to generate the data stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamPrimaryKeys(CommandGetPrimaryKeys command, CallContext context, Ticket ticket, + ServerStreamListener listener); + + /** + * Returns the available primary keys by returning a stream of + * {@link org.apache.arrow.flight.sql.impl.FlightSql.CommandGetForeignKeys} objects in {@link Result} objects. + * + * @param request request filter parameters. + * @param context Per-call context. + * @param descriptor The descriptor identifying the data stream. + * @return Metadata about the stream. + */ + public abstract FlightInfo getFlightInfoForeignKeys(CommandGetForeignKeys request, CallContext context, + FlightDescriptor descriptor); + + /** + * Gets schema about the get foreign keys data stream. + * + * @return Schema for the stream. + */ + public SchemaResult getSchemaForeignKeys() { + final List fields = new ArrayList<>(); + + fields.add(new Field("pk_catalog_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("pk_schema_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("pk_table_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("pk_column_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("fk_catalog_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("fk_schema_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("fk_table_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("fk_column_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("key_sequence", FieldType.nullable(Types.MinorType.INT.getType()), null)); + fields.add(new Field("fk_key_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("pk_key_name", FieldType.nullable(Types.MinorType.VARCHAR.getType()), null)); + fields.add(new Field("update_rule", FieldType.nullable(Types.MinorType.INT.getType()), null)); + fields.add(new Field("delete_rule", FieldType.nullable(Types.MinorType.INT.getType()), null)); + + return new SchemaResult(new Schema(fields)); + } + + /** + * Returns data for foreign keys based data stream. + * + * @param command The command to generate the data stream. + * @param context Per-call context. + * @param ticket The application-defined ticket identifying this stream. + * @param listener An interface for sending data back to the client. + */ + public abstract void getStreamForeignKeys(CommandGetForeignKeys command, CallContext context, Ticket ticket, + ServerStreamListener listener); }