kylepbit commented on a change in pull request #9368:
URL: https://github.com/apache/arrow/pull/9368#discussion_r664935069



##########
File path: 
java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/FlightSqlExample.java
##########
@@ -0,0 +1,599 @@
+/*
+ * 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.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<PoolableConnection> dataSource;
+
+  private final LoadingCache<CommandPreparedStatementQuery, ResultSet> 
commandExecutePreparedStatementLoadingCache;
+  private final LoadingCache<PreparedStatementCacheKey, 
PreparedStatementContext> 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<PoolableConnection> 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<Result> 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.setArrowMetadata(ByteString.copyFrom(pojoSchema.toByteArray()));
+    }
+
+    return new Result(Any.pack(builder.build()).toByteArray());
+  }
+
+  @Override
+  public void getPreparedStatement(FlightSql.ActionGetPreparedStatementRequest 
request, CallContext context,
+          StreamListener<Result> 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<FlightEndpoint> 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<Field> 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) {

Review comment:
       I changed it to be Throwable.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscr...@arrow.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to