Skip to content

Commit b62d85b

Browse files
committed
add builder pattern
1 parent 303aa92 commit b62d85b

4 files changed

Lines changed: 136 additions & 42 deletions

File tree

flight/flight-sql-jdbc-core/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightMetaImpl.java

Lines changed: 14 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -102,15 +102,13 @@ ArrowFlightPreparedStatement createPreparedStatement(
102102
final int resultSetConcurrency,
103103
final int resultSetHoldability)
104104
throws SQLException {
105-
final StatementHandle handle = super.createStatement(connection.handle);
106-
return ArrowFlightPreparedStatement.createPrepared(
107-
(ArrowFlightConnection) connection,
108-
handle,
109-
// null,
110-
query,
111-
resultSetType,
112-
resultSetConcurrency,
113-
resultSetHoldability);
105+
return ArrowFlightPreparedStatement.builder((ArrowFlightConnection) connection)
106+
.withQuery(query)
107+
.withGeneratedHandle()
108+
.withResultSetType(resultSetType)
109+
.withResultSetConcurrency(resultSetConcurrency)
110+
.withResultSetHoldability(resultSetHoldability)
111+
.build();
114112
}
115113

116114
@Override
@@ -153,14 +151,14 @@ public ExecuteResult prepareAndExecute(
153151
&& !(statement instanceof ArrowFlightPreparedStatement)) {
154152
throw new IllegalStateException("Prepared statement not found: " + handle);
155153
}
154+
if (statement instanceof ArrowFlightPreparedStatement) {
155+
((ArrowFlightPreparedStatement) statement).closePreparedResources();
156+
}
156157
final ArrowFlightPreparedStatement preparedStatement =
157-
ArrowFlightPreparedStatement.createPrepared(
158-
(ArrowFlightConnection) connection,
159-
handle,
160-
query,
161-
statement.getResultSetType(),
162-
statement.getResultSetConcurrency(),
163-
statement.getResultSetHoldability());
158+
ArrowFlightPreparedStatement.builder((ArrowFlightConnection) connection)
159+
.withQuery(query)
160+
.withExistingStatement(statement)
161+
.build();
164162
return preparedStatement.prepareAndExecute(callback);
165163
} catch (SQLTimeoutException e) {
166164
// So far AvaticaStatement(executeInternal) only handles NoSuchStatement and

flight/flight-sql-jdbc-core/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightPreparedStatement.java

Lines changed: 82 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import org.apache.arrow.util.Preconditions;
2626
import org.apache.arrow.vector.types.pojo.Schema;
2727
import org.apache.calcite.avatica.AvaticaPreparedStatement;
28+
import org.apache.calcite.avatica.AvaticaStatement;
2829
import org.apache.calcite.avatica.Meta.ExecuteBatchResult;
2930
import org.apache.calcite.avatica.Meta.ExecuteResult;
3031
import org.apache.calcite.avatica.Meta.MetaResultSet;
@@ -44,39 +45,19 @@ private ArrowFlightPreparedStatement(
4445
final ArrowFlightConnection connection,
4546
final StatementHandle handle,
4647
final Signature signature,
48+
final ArrowFlightSqlClientHandler.PreparedStatement preparedStatement,
4749
final int resultSetType,
4850
final int resultSetConcurrency,
4951
final int resultSetHoldability)
5052
throws SQLException {
5153
super(connection, handle, signature, resultSetType, resultSetConcurrency, resultSetHoldability);
54+
this.preparedStatement = Preconditions.checkNotNull(preparedStatement);
55+
this.handle.signature = signature;
56+
setSignature(signature);
5257
}
5358

54-
static ArrowFlightPreparedStatement createPrepared(
55-
final ArrowFlightConnection connection,
56-
final StatementHandle statementHandle,
57-
final String query,
58-
final int resultSetType,
59-
final int resultSetConcurrency,
60-
final int resultSetHoldability)
61-
throws SQLException {
62-
final ArrowFlightSqlClientHandler.PreparedStatement preparedStatement =
63-
connection.getClientHandler().prepare(query);
64-
final Signature signature =
65-
ArrowFlightMetaImpl.buildSignature(
66-
query, preparedStatement.getDataSetSchema(), preparedStatement.getParameterSchema());
67-
statementHandle.signature = signature;
68-
69-
final ArrowFlightPreparedStatement statement =
70-
new ArrowFlightPreparedStatement(
71-
connection,
72-
statementHandle,
73-
signature,
74-
resultSetType,
75-
resultSetConcurrency,
76-
resultSetHoldability);
77-
statement.preparedStatement = Preconditions.checkNotNull(preparedStatement);
78-
statement.setSignature(signature);
79-
return statement;
59+
static Builder builder(final ArrowFlightConnection connection) {
60+
return new Builder(connection);
8061
}
8162

8263
@Override
@@ -173,4 +154,79 @@ private void ensurePrepared() {
173154
throw new IllegalStateException("PreparedStatement is already closed.");
174155
}
175156
}
157+
158+
static final class Builder {
159+
private final ArrowFlightConnection connection;
160+
private StatementHandle handle;
161+
private String query;
162+
private Integer resultSetType;
163+
private Integer resultSetConcurrency;
164+
private Integer resultSetHoldability;
165+
private boolean generateHandle;
166+
167+
private Builder(final ArrowFlightConnection connection) {
168+
this.connection = Preconditions.checkNotNull(connection);
169+
}
170+
171+
Builder withQuery(final String query) {
172+
this.query = Preconditions.checkNotNull(query);
173+
return this;
174+
}
175+
176+
Builder withGeneratedHandle() {
177+
this.generateHandle = true;
178+
this.handle = null;
179+
return this;
180+
}
181+
182+
Builder withExistingStatement(final AvaticaStatement statement) throws SQLException {
183+
Preconditions.checkNotNull(statement);
184+
this.generateHandle = false;
185+
this.handle = Preconditions.checkNotNull(statement.handle);
186+
this.resultSetType = statement.getResultSetType();
187+
this.resultSetConcurrency = statement.getResultSetConcurrency();
188+
this.resultSetHoldability = statement.getResultSetHoldability();
189+
return this;
190+
}
191+
192+
Builder withResultSetType(final int resultSetType) {
193+
this.resultSetType = resultSetType;
194+
return this;
195+
}
196+
197+
Builder withResultSetConcurrency(final int resultSetConcurrency) {
198+
this.resultSetConcurrency = resultSetConcurrency;
199+
return this;
200+
}
201+
202+
Builder withResultSetHoldability(final int resultSetHoldability) {
203+
this.resultSetHoldability = resultSetHoldability;
204+
return this;
205+
}
206+
207+
ArrowFlightPreparedStatement build() throws SQLException {
208+
Preconditions.checkNotNull(query);
209+
Preconditions.checkNotNull(resultSetType);
210+
Preconditions.checkNotNull(resultSetConcurrency);
211+
Preconditions.checkNotNull(resultSetHoldability);
212+
if (!generateHandle && handle == null) {
213+
throw new IllegalStateException("PreparedStatement builder requires a handle.");
214+
}
215+
216+
final ArrowFlightSqlClientHandler.PreparedStatement preparedStatement =
217+
connection.getClientHandler().prepare(query);
218+
final Signature signature =
219+
ArrowFlightMetaImpl.buildSignature(
220+
query, preparedStatement.getDataSetSchema(), preparedStatement.getParameterSchema());
221+
222+
return new ArrowFlightPreparedStatement(
223+
connection,
224+
generateHandle ? null : handle,
225+
signature,
226+
preparedStatement,
227+
resultSetType,
228+
resultSetConcurrency,
229+
resultSetHoldability);
230+
}
231+
}
176232
}

flight/flight-sql-jdbc-core/src/test/java/org/apache/arrow/driver/jdbc/ArrowFlightPreparedStatementTest.java

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,8 @@
2121
import static org.junit.jupiter.api.Assertions.assertAll;
2222
import static org.junit.jupiter.api.Assertions.assertEquals;
2323
import static org.junit.jupiter.api.Assertions.assertFalse;
24+
import static org.junit.jupiter.api.Assertions.assertNotNull;
25+
import static org.junit.jupiter.api.Assertions.assertSame;
2426
import static org.junit.jupiter.api.Assertions.assertTrue;
2527

2628
import java.nio.charset.StandardCharsets;
@@ -98,6 +100,27 @@ public void testSimpleQueryNoParameterBindingWithExecute() throws SQLException {
98100
}
99101
}
100102

103+
@Test
104+
public void testPrepareStatementRegistersCreatedStatementByGeneratedHandle() throws SQLException {
105+
final String query = CoreMockedSqlProducers.LEGACY_REGULAR_SQL_CMD;
106+
final ArrowFlightConnection flightConnection = (ArrowFlightConnection) connection;
107+
108+
try (final PreparedStatement preparedStatement = connection.prepareStatement(query)) {
109+
final ArrowFlightPreparedStatement arrowPreparedStatement =
110+
(ArrowFlightPreparedStatement) preparedStatement;
111+
112+
assertNotNull(
113+
flightConnection
114+
.getMeta()
115+
.getPreparedStatementInstanceOrNull(arrowPreparedStatement.handle));
116+
assertSame(
117+
arrowPreparedStatement,
118+
flightConnection
119+
.getMeta()
120+
.getPreparedStatementInstanceOrNull(arrowPreparedStatement.handle));
121+
}
122+
}
123+
101124
@Test
102125
public void testQueryWithParameterBinding() throws SQLException {
103126
final String query = "Fake query with parameters";

flight/flight-sql-jdbc-core/src/test/java/org/apache/arrow/driver/jdbc/ArrowFlightStatementExecuteTest.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,8 @@
2222
import static org.hamcrest.CoreMatchers.not;
2323
import static org.hamcrest.CoreMatchers.nullValue;
2424
import static org.hamcrest.MatcherAssert.assertThat;
25+
import static org.junit.jupiter.api.Assertions.assertNotNull;
26+
import static org.junit.jupiter.api.Assertions.assertSame;
2527

2628
import java.sql.Connection;
2729
import java.sql.ResultSet;
@@ -137,6 +139,21 @@ public void testExecuteShouldRunSelectQuery() throws SQLException {
137139
is(allOf(equalTo(statement.getLargeUpdateCount()), equalTo(-1L))));
138140
}
139141

142+
@Test
143+
public void testExecuteReplacesStatementMapEntryWithPreparedStatement() throws SQLException {
144+
final ArrowFlightStatement arrowStatement = (ArrowFlightStatement) statement;
145+
final ArrowFlightConnection arrowConnection = (ArrowFlightConnection) connection;
146+
147+
assertThat(statement.execute(SAMPLE_QUERY_CMD), is(true));
148+
149+
final ArrowFlightPreparedStatement preparedStatement =
150+
arrowConnection.getMeta().getPreparedStatementInstanceOrNull(arrowStatement.handle);
151+
152+
assertNotNull(preparedStatement);
153+
assertSame(preparedStatement, arrowConnection.statementMap.get(arrowStatement.handle.id));
154+
assertThat(preparedStatement.handle.id, is(equalTo(arrowStatement.handle.id)));
155+
}
156+
140157
@Test
141158
public void testExecuteShouldRunUpdateQueryForSmallUpdate() throws SQLException {
142159
assertThat(statement.execute(SAMPLE_UPDATE_QUERY), is(false)); // Means this is an UPDATE query.

0 commit comments

Comments
 (0)