diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlExample.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlExample.java index bb7444856668c..0b755f7d241a7 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlExample.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlExample.java @@ -280,6 +280,9 @@ private static boolean removeDerbyDatabaseIfExists() { private static boolean populateDerbyDatabase() { try (final Connection connection = DriverManager.getConnection("jdbc:derby:target/derbyDB;create=true"); Statement statement = connection.createStatement()) { + + dropTable(statement, "intTable"); + dropTable(statement, "foreignTable"); statement.execute("CREATE TABLE foreignTable (" + "id INT not null primary key GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), " + "foreignName varchar(100), " + @@ -302,6 +305,18 @@ private static boolean populateDerbyDatabase() { return true; } + private static void dropTable(final Statement statement, final String tableName) throws SQLException { + try { + statement.execute("DROP TABLE " + tableName); + } catch (SQLException e) { + // sql error code for "object does not exist"; which is fine, we're trying to delete the table + // see https://db.apache.org/derby/docs/10.17/ref/rrefexcept71493.html + if (!"42Y55".equals(e.getSQLState())) { + throw e; + } + } + } + private static ArrowType getArrowTypeFromJdbcType(final int jdbcDataType, final int precision, final int scale) { try { return JdbcToArrowUtils.getArrowTypeFromJdbcType(new JdbcFieldInfo(jdbcDataType, precision, scale), diff --git a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlStatelessExample.java b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlStatelessExample.java index 02f679b3efaf4..8d1d901eb94eb 100644 --- a/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlStatelessExample.java +++ b/java/flight/flight-sql/src/test/java/org/apache/arrow/flight/sql/example/FlightSqlStatelessExample.java @@ -77,7 +77,8 @@ public Runnable acceptPutPreparedStatementQuery(CommandPreparedStatementQuery co return () -> { final String query = new String(command.getPreparedStatementHandle().toStringUtf8()); - try (PreparedStatement preparedStatement = createPreparedStatement(query)) { + try (Connection connection = dataSource.getConnection(); + PreparedStatement preparedStatement = createPreparedStatement(connection, query)) { while (flightStream.next()) { final VectorSchemaRoot root = flightStream.getRoot(); final JdbcParameterBinder binder = JdbcParameterBinder.builder(preparedStatement, root).bindAll().build(); @@ -133,7 +134,8 @@ public void getStreamPreparedStatement(final CommandPreparedStatementQuery comma deserializePOJO(handle); final String query = doPutPreparedStatementResultPOJO.getQuery(); - try (PreparedStatement statement = createPreparedStatement(query); + try (Connection connection = dataSource.getConnection(); + PreparedStatement statement = createPreparedStatement(connection, query); ArrowFileReader reader = new ArrowFileReader(new SeekableReadChannel( new ByteArrayReadableSeekableByteChannel( doPutPreparedStatementResultPOJO.getParameters())), rootAllocator)) { @@ -152,7 +154,8 @@ public void getStreamPreparedStatement(final CommandPreparedStatementQuery comma } catch (StreamCorruptedException e) { // Case where there are no parameters final String query = new String(command.getPreparedStatementHandle().toStringUtf8()); - try (PreparedStatement preparedStatement = createPreparedStatement(query)) { + try (Connection connection = dataSource.getConnection(); + PreparedStatement preparedStatement = createPreparedStatement(connection, query)) { executeQuery(preparedStatement, listener); } } @@ -200,7 +203,8 @@ public FlightInfo getFlightInfoPreparedStatement(final CommandPreparedStatementQ } catch (StreamCorruptedException e) { query = new String(command.getPreparedStatementHandle().toStringUtf8()); } - try (PreparedStatement statement = createPreparedStatement(query)) { + try (Connection connection = dataSource.getConnection(); + PreparedStatement statement = createPreparedStatement(connection, query)) { ResultSetMetaData metaData = statement.getMetaData(); return getFlightInfoForSchema(command, descriptor, jdbcToArrowSchema(metaData, DEFAULT_CALENDAR)); @@ -226,8 +230,7 @@ private byte[] serializePOJO(DoPutPreparedStatementResultPOJO doPutPreparedState } } - private PreparedStatement createPreparedStatement(String query) throws SQLException { - final Connection connection = dataSource.getConnection(); + private PreparedStatement createPreparedStatement(Connection connection, String query) throws SQLException { return connection.prepareStatement(query, ResultSet.TYPE_SCROLL_INSENSITIVE, ResultSet.CONCUR_READ_ONLY); } }