jduo commented on a change in pull request #12254: URL: https://github.com/apache/arrow/pull/12254#discussion_r824968996
########## File path: java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java ########## @@ -0,0 +1,535 @@ +/* + * 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.driver.jdbc.client; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.arrow.driver.jdbc.client.utils.ClientAuthenticationUtils; +import org.apache.arrow.flight.CallOption; +import org.apache.arrow.flight.FlightClient; +import org.apache.arrow.flight.FlightClientMiddleware; +import org.apache.arrow.flight.FlightEndpoint; +import org.apache.arrow.flight.FlightInfo; +import org.apache.arrow.flight.FlightStream; +import org.apache.arrow.flight.Location; +import org.apache.arrow.flight.auth2.BearerCredentialWriter; +import org.apache.arrow.flight.auth2.ClientBearerHeaderHandler; +import org.apache.arrow.flight.auth2.ClientIncomingAuthHeaderMiddleware; +import org.apache.arrow.flight.client.ClientCookieMiddleware; +import org.apache.arrow.flight.grpc.CredentialCallOption; +import org.apache.arrow.flight.sql.FlightSqlClient; +import org.apache.arrow.flight.sql.impl.FlightSql.SqlInfo; +import org.apache.arrow.flight.sql.util.TableRef; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.util.AutoCloseables; +import org.apache.arrow.util.Preconditions; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.calcite.avatica.Meta.StatementType; + +/** + * A {@link FlightSqlClient} handler. + */ +public final class ArrowFlightSqlClientHandler implements AutoCloseable { + + private final FlightSqlClient sqlClient; + private final Set<CallOption> options = new HashSet<>(); + + ArrowFlightSqlClientHandler(final FlightSqlClient sqlClient, + final Collection<CallOption> options) { + this.options.addAll(options); + this.sqlClient = Preconditions.checkNotNull(sqlClient); + } + + /** + * Creates a new {@link ArrowFlightSqlClientHandler} from the provided {@code client} and {@code options}. + * + * @param client the {@link FlightClient} to manage under a {@link FlightSqlClient} wrapper. + * @param options the {@link CallOption}s to persist in between subsequent client calls. + * @return a new {@link ArrowFlightSqlClientHandler}. + */ + public static ArrowFlightSqlClientHandler createNewHandler(final FlightClient client, + final Collection<CallOption> options) { + return new ArrowFlightSqlClientHandler(new FlightSqlClient(client), options); + } + + /** + * Gets the {@link #options} for the subsequent calls from this handler. + * + * @return the {@link CallOption}s. + */ + private CallOption[] getOptions() { + return options.toArray(new CallOption[0]); + } + + /** + * Makes an RPC "getStream" request based on the provided {@link FlightInfo} + * object. Retrieves the result of the query previously prepared with "getInfo." + * + * @param flightInfo The {@link FlightInfo} instance from which to fetch results. + * @return a {@code FlightStream} of results. + */ + public List<FlightStream> getStreams(final FlightInfo flightInfo) { + return flightInfo.getEndpoints().stream() Review comment: This isn't correct. This will try to get every ticket using the current client. However each endpoint is a pair <Ticket, List<Location>>. What we actually need to do is for each endpoint, create a FlightClient at each location and call getStream() with the associated ticket. See: https://github.com/apache/arrow/blob/dc2e0b2e44fdaa3d5ad0bb358ff8ce9db3bc7416/java/flight/flight-integration-tests/src/main/java/org/apache/arrow/flight/integration/tests/IntegrationTestClient.java#L151 Note that if the list of locations is empty, it means the data is only available at the current location: Can optimize this to some extent if the current location is used once, use the current client, though that limits the driver to only running one Statement at a time. We should consider pooling FlightClients. I think Apache Pools is a good choice here, and use KeyedObjectPool to model the problem: https://commons.apache.org/proper/commons-pool/apidocs/org/apache/commons/pool2/KeyedObjectPool.html -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
