Skip to content

Commit 2fb7fd9

Browse files
authored
GH-34532: [Java][FlightSQL] Change JDBC to handle multi-endpoints (#38521)
### Rationale for this change The Flight SQL JDBC Driver currently doesn't fetch at multiple endpoints correctly when the data is not at the same location as the original connection. ### What changes are included in this PR? - Create new clients to connect to new locations in endpoints. - If no location is reported using the current connection. - Make ArrowFlightSqlClientHandler's builder's build() function to be idempodent. - Add functionality to clone ArrowFlightSqClientHandler's builder so that it can be used for temporary connections to locations returned by getFlightInfo(). - Add utility classes in unit tests for constructing a distributed Flight SQL Server ### Are these changes tested? Yes. ### Are there any user-facing changes? The behavior for when there are reported endpoints from getFlightInfo is now fixed. However if users relied on the previous behavior of just getting streams from the same node, and their server only ever reported the original node, they may observe more Flight client connections opening and closing than before (since new connections get spawned for each partition that has at least one Location now). * Closes: #34532 Authored-by: James Duong <james.duong@improving.com> Signed-off-by: David Li <li.davidm96@gmail.com>
1 parent cd6e635 commit 2fb7fd9

File tree

8 files changed

+439
-109
lines changed

8 files changed

+439
-109
lines changed

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

Lines changed: 42 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.arrow.driver.jdbc;
1919

20-
import static org.apache.arrow.driver.jdbc.utils.FlightStreamQueue.createNewQueue;
20+
import static org.apache.arrow.driver.jdbc.utils.FlightEndpointDataQueue.createNewQueue;
2121

2222
import java.sql.ResultSet;
2323
import java.sql.ResultSetMetaData;
@@ -26,7 +26,8 @@
2626
import java.util.TimeZone;
2727
import java.util.concurrent.TimeUnit;
2828

29-
import org.apache.arrow.driver.jdbc.utils.FlightStreamQueue;
29+
import org.apache.arrow.driver.jdbc.client.CloseableEndpointStreamPair;
30+
import org.apache.arrow.driver.jdbc.utils.FlightEndpointDataQueue;
3031
import org.apache.arrow.driver.jdbc.utils.VectorSchemaRootTransformer;
3132
import org.apache.arrow.flight.FlightInfo;
3233
import org.apache.arrow.flight.FlightStream;
@@ -47,8 +48,8 @@ public final class ArrowFlightJdbcFlightStreamResultSet
4748
extends ArrowFlightJdbcVectorSchemaRootResultSet {
4849

4950
private final ArrowFlightConnection connection;
50-
private FlightStream currentFlightStream;
51-
private FlightStreamQueue flightStreamQueue;
51+
private CloseableEndpointStreamPair currentEndpointData;
52+
private FlightEndpointDataQueue flightEndpointDataQueue;
5253

5354
private VectorSchemaRootTransformer transformer;
5455
private VectorSchemaRoot currentVectorSchemaRoot;
@@ -102,20 +103,20 @@ static ArrowFlightJdbcFlightStreamResultSet fromFlightInfo(
102103

103104
resultSet.transformer = transformer;
104105

105-
resultSet.execute(flightInfo);
106+
resultSet.populateData(flightInfo);
106107
return resultSet;
107108
}
108109

109110
private void loadNewQueue() {
110-
Optional.ofNullable(flightStreamQueue).ifPresent(AutoCloseables::closeNoChecked);
111-
flightStreamQueue = createNewQueue(connection.getExecutorService());
111+
Optional.ofNullable(flightEndpointDataQueue).ifPresent(AutoCloseables::closeNoChecked);
112+
flightEndpointDataQueue = createNewQueue(connection.getExecutorService());
112113
}
113114

114115
private void loadNewFlightStream() throws SQLException {
115-
if (currentFlightStream != null) {
116-
AutoCloseables.closeNoChecked(currentFlightStream);
116+
if (currentEndpointData != null) {
117+
AutoCloseables.closeNoChecked(currentEndpointData);
117118
}
118-
this.currentFlightStream = getNextFlightStream(true);
119+
this.currentEndpointData = getNextEndpointStream(true);
119120
}
120121

121122
@Override
@@ -124,24 +125,24 @@ protected AvaticaResultSet execute() throws SQLException {
124125

125126
if (flightInfo != null) {
126127
schema = flightInfo.getSchemaOptional().orElse(null);
127-
execute(flightInfo);
128+
populateData(flightInfo);
128129
}
129130
return this;
130131
}
131132

132-
private void execute(final FlightInfo flightInfo) throws SQLException {
133+
private void populateData(final FlightInfo flightInfo) throws SQLException {
133134
loadNewQueue();
134-
flightStreamQueue.enqueue(connection.getClientHandler().getStreams(flightInfo));
135+
flightEndpointDataQueue.enqueue(connection.getClientHandler().getStreams(flightInfo));
135136
loadNewFlightStream();
136137

137138
// Ownership of the root will be passed onto the cursor.
138-
if (currentFlightStream != null) {
139-
executeForCurrentFlightStream();
139+
if (currentEndpointData != null) {
140+
populateDataForCurrentFlightStream();
140141
}
141142
}
142143

143-
private void executeForCurrentFlightStream() throws SQLException {
144-
final VectorSchemaRoot originalRoot = currentFlightStream.getRoot();
144+
private void populateDataForCurrentFlightStream() throws SQLException {
145+
final VectorSchemaRoot originalRoot = currentEndpointData.getStream().getRoot();
145146

146147
if (transformer != null) {
147148
try {
@@ -154,9 +155,9 @@ private void executeForCurrentFlightStream() throws SQLException {
154155
}
155156

156157
if (schema != null) {
157-
execute(currentVectorSchemaRoot, schema);
158+
populateData(currentVectorSchemaRoot, schema);
158159
} else {
159-
execute(currentVectorSchemaRoot);
160+
populateData(currentVectorSchemaRoot);
160161
}
161162
}
162163

@@ -179,20 +180,20 @@ public boolean next() throws SQLException {
179180
return true;
180181
}
181182

182-
if (currentFlightStream != null) {
183-
currentFlightStream.getRoot().clear();
184-
if (currentFlightStream.next()) {
185-
executeForCurrentFlightStream();
183+
if (currentEndpointData != null) {
184+
currentEndpointData.getStream().getRoot().clear();
185+
if (currentEndpointData.getStream().next()) {
186+
populateDataForCurrentFlightStream();
186187
continue;
187188
}
188189

189-
flightStreamQueue.enqueue(currentFlightStream);
190+
flightEndpointDataQueue.enqueue(currentEndpointData);
190191
}
191192

192-
currentFlightStream = getNextFlightStream(false);
193+
currentEndpointData = getNextEndpointStream(false);
193194

194-
if (currentFlightStream != null) {
195-
executeForCurrentFlightStream();
195+
if (currentEndpointData != null) {
196+
populateDataForCurrentFlightStream();
196197
continue;
197198
}
198199

@@ -207,14 +208,14 @@ public boolean next() throws SQLException {
207208
@Override
208209
protected void cancel() {
209210
super.cancel();
210-
final FlightStream currentFlightStream = this.currentFlightStream;
211-
if (currentFlightStream != null) {
212-
currentFlightStream.cancel("Cancel", null);
211+
final CloseableEndpointStreamPair currentEndpoint = this.currentEndpointData;
212+
if (currentEndpoint != null) {
213+
currentEndpoint.getStream().cancel("Cancel", null);
213214
}
214215

215-
if (flightStreamQueue != null) {
216+
if (flightEndpointDataQueue != null) {
216217
try {
217-
flightStreamQueue.close();
218+
flightEndpointDataQueue.close();
218219
} catch (final Exception e) {
219220
throw new RuntimeException(e);
220221
}
@@ -224,27 +225,28 @@ protected void cancel() {
224225
@Override
225226
public synchronized void close() {
226227
try {
227-
if (flightStreamQueue != null) {
228+
if (flightEndpointDataQueue != null) {
228229
// flightStreamQueue should close currentFlightStream internally
229-
flightStreamQueue.close();
230-
} else if (currentFlightStream != null) {
230+
flightEndpointDataQueue.close();
231+
} else if (currentEndpointData != null) {
231232
// close is only called for currentFlightStream if there's no queue
232-
currentFlightStream.close();
233+
currentEndpointData.close();
233234
}
235+
234236
} catch (final Exception e) {
235237
throw new RuntimeException(e);
236238
} finally {
237239
super.close();
238240
}
239241
}
240242

241-
private FlightStream getNextFlightStream(final boolean isExecution) throws SQLException {
242-
if (isExecution) {
243+
private CloseableEndpointStreamPair getNextEndpointStream(final boolean canTimeout) throws SQLException {
244+
if (canTimeout) {
243245
final int statementTimeout = statement != null ? statement.getQueryTimeout() : 0;
244246
return statementTimeout != 0 ?
245-
flightStreamQueue.next(statementTimeout, TimeUnit.SECONDS) : flightStreamQueue.next();
247+
flightEndpointDataQueue.next(statementTimeout, TimeUnit.SECONDS) : flightEndpointDataQueue.next();
246248
} else {
247-
return flightStreamQueue.next();
249+
return flightEndpointDataQueue.next();
248250
}
249251
}
250252
}

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -83,7 +83,7 @@ public static ArrowFlightJdbcVectorSchemaRootResultSet fromVectorSchemaRoot(
8383
new ArrowFlightJdbcVectorSchemaRootResultSet(null, state, signature, resultSetMetaData,
8484
timeZone, null);
8585

86-
resultSet.execute(vectorSchemaRoot);
86+
resultSet.populateData(vectorSchemaRoot);
8787
return resultSet;
8888
}
8989

@@ -92,7 +92,7 @@ protected AvaticaResultSet execute() throws SQLException {
9292
throw new RuntimeException("Can only execute with execute(VectorSchemaRoot)");
9393
}
9494

95-
void execute(final VectorSchemaRoot vectorSchemaRoot) {
95+
void populateData(final VectorSchemaRoot vectorSchemaRoot) {
9696
final List<Field> fields = vectorSchemaRoot.getSchema().getFields();
9797
final List<ColumnMetaData> columns = ConvertUtils.convertArrowFieldsToColumnMetaDataList(fields);
9898
signature.columns.clear();
@@ -102,7 +102,7 @@ void execute(final VectorSchemaRoot vectorSchemaRoot) {
102102
execute2(new ArrowFlightJdbcCursor(vectorSchemaRoot), this.signature.columns);
103103
}
104104

105-
void execute(final VectorSchemaRoot vectorSchemaRoot, final Schema schema) {
105+
void populateData(final VectorSchemaRoot vectorSchemaRoot, final Schema schema) {
106106
final List<ColumnMetaData> columns = ConvertUtils.convertArrowFieldsToColumnMetaDataList(schema.getFields());
107107
signature.columns.clear();
108108
signature.columns.addAll(columns);

0 commit comments

Comments
 (0)