You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/04/07 20:41:28 UTC

[GitHub] [arrow] jduo opened a new pull request, #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

jduo opened a new pull request, #12830:
URL: https://github.com/apache/arrow/pull/12830

   This implements a JDBC driver able to communicate to Flight SQL sources.
   
   So far this covers:
   
       Metadata retrieval by DatabaseMetadata, ResultSetMetadata, etc.
       Query execution by statements and prepared statements
   
   Yet to be done:
   
       Parameter binding on prepared statements
   
   


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] github-actions[bot] commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1092182046

   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/master/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on JIRA? https://issues.apache.org/jira/browse/ARROW
   
   Opening JIRAs ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1097101579

   Was the merge commit intended? It's made this PR quite a bit messier


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] avantgardnerio commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
avantgardnerio commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929346155


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   This works, but I'm interested in feedback about how to do it the correct way: https://github.com/rafael-telles/arrow/pull/42
   
   I'm also happy to help others get set up to run with a similar test environment as me if that helps.



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929337830


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   Yes, you got it. We clarified the spec, but haven't necessarily updated everything else



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] avantgardnerio commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
avantgardnerio commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929346155


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   This works, but I'm interested in feedback about how to do it the correct way: https://github.com/rafael-telles/arrow/pull/42



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] avantgardnerio commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
avantgardnerio commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929303757


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   I think this behavior is incorrect, and it is causing problems for DataFusion's implementation. It looks like this code re-uses the existing `sqlClient`, ignoring the value of `flightInfo.enpoints[x].locations[y].uri` (which may be different than the original URI the `sqlClient` connected to, in the case of DataFusion, the `sqlClient` would be connected to the scheduler at one URI, and the data would be held on one or more executor nodes at different URIs).
   
   I can attempt a PR if you agree with my hypothesis. I'm new to all this, so I could be way off base. And thank you for putting in the massive amount of work necessary to make this BTW!



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] avantgardnerio commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
avantgardnerio commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929312223


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   Oh, I think I understand now:
   
   > merged a clarification for this behavior
   
   The behavior has been clarified, but the code to which I linked has not been corrected according to the now clarified expected behavior?
   
   If that's the case, I'll work on a fix and PR this branch.



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] ZMZ91 commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
ZMZ91 commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1138044117

   Hi there,
   Doe this support auth token?


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1092858159

   BTW, it looks like the first hundred or so commits can probably be rebased out without affecting the PR (there are some minor merge conflicts to fix after that), since they look to be old Flight SQL commits that were already merged in some form.


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] ZMZ91 commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
ZMZ91 commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1141590628

   Hi, I've tried this code and found it supports auth token only when set the server auth handler as NoOpAuthHandler and add an auth header in middleware to pass the token at the same time, right? 


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1092205766

   Thanks @jduo!
   
   I see several ICLAs have been filed already, I'll move the list here:
   
   For the IP clearance: it looks like these are the current contributors. As far as I can see, nobody has an ICLA on file. Can each of you file an ICLA ([instructions](https://www.apache.org/licenses/contributor-agreements.html)) and confirm here?
   
   - [ ] Abner Eduardo Ferreira
   - [x] Gabriel Escobar 
   - [x] iurysalino
   - [ ] jayhomn-bitquill 
   - [x] Jose Almeida 
   - [ ] Juscelino Junior
   - [ ] Kyle Porter 
   - [x] Rafael Telles
   - [ ] Ryan Nicholson 
   - [ ] tifflhl
   - [ ] Vinicius Fraga


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929307671


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   This is the repo/branch being used for the IP clearance so I would say you're already looking at it!
   
   My plan is once the clearance goes through, this will get merged into a branch on apache/arrow (the PR merge target here, flight-sql-jdbc), then we'll merge master into the branch, then finally do a PR into master (and review the code and such).



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] avantgardnerio commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
avantgardnerio commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929306501


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   Glad to hear it's been fixed. Is there a current, long-running repo & branch we can track until this gets merged into arrow proper?



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] jduo commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
jduo commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1097105944

   > Was the merge commit intended? It's made this PR quite a bit messier
   
   We're looking into creating a new branch with a single commit -- there are many rebase conflicts at the moment.


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929304931


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   You are right, we actually merged a clarification for this behavior a little while ago. https://github.com/apache/arrow/pull/12636
   
   Note we're still finalizing the IP clearance of this PR (but I assume it will go through one way or another)



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] avantgardnerio commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
avantgardnerio commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929306501


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   Glad to hear it's been fixed. Is there a current, long-running rep & branch we can track until this gets merged into arrow proper?



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1114999692

   @jduo do you still plan to rebase things? Also, would you like the base branch to be updated since the proposal was merged?


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm merged pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm merged PR #12830:
URL: https://github.com/apache/arrow/pull/12830


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] github-actions[bot] commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1199414004

   https://issues.apache.org/jira/browse/ARROW-15452


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1092997594

   Oh, CI indicates there are files that need a license header:
   
   ```
   
   Error:  Failed to execute goal org.apache.rat:apache-rat-plugin:0.13:check (rat-checks) on project arrow-java-root: Too many files with unapproved license: 1 See RAT report in: /arrow/java/target/rat.txt -> [Help 1]
   ```


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r846161414


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcArray.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.sql.Array;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Map;
+
+import org.apache.arrow.driver.jdbc.accessor.impl.complex.AbstractArrowFlightJdbcListVectorAccessor;
+import org.apache.arrow.driver.jdbc.utils.SqlTypes;
+import org.apache.arrow.memory.util.LargeMemoryUtil;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.util.TransferPair;
+
+/**
+ * Implementation of {@link Array} using an underlying {@link FieldVector}.
+ *
+ * @see AbstractArrowFlightJdbcListVectorAccessor
+ */
+public class ArrowFlightJdbcArray implements Array {
+
+  private final FieldVector dataVector;
+  private final long startOffset;
+  private final long valuesCount;
+
+  /**
+   * Instantiate an {@link Array} backed up by given {@link FieldVector}, limited by a start offset and values count.
+   *
+   * @param dataVector  underlying FieldVector, containing the Array items.
+   * @param startOffset offset from FieldVector pointing to this Array's first value.
+   * @param valuesCount how many items this Array contains.
+   */
+  public ArrowFlightJdbcArray(FieldVector dataVector, long startOffset, long valuesCount) {
+    this.dataVector = dataVector;
+    this.startOffset = startOffset;
+    this.valuesCount = valuesCount;
+  }
+
+  @Override
+  public String getBaseTypeName() {
+    final ArrowType arrowType = this.dataVector.getField().getType();
+    return SqlTypes.getSqlTypeNameFromArrowType(arrowType);
+  }
+
+  @Override
+  public int getBaseType() {
+    final ArrowType arrowType = this.dataVector.getField().getType();
+    return SqlTypes.getSqlTypeIdFromArrowType(arrowType);
+  }
+
+  @Override
+  public Object getArray() throws SQLException {
+    return getArray(null);
+  }
+
+  @Override
+  public Object getArray(Map<String, Class<?>> map) throws SQLException {
+    if (map != null) {
+      throw new SQLFeatureNotSupportedException();
+    }
+
+    return getArrayNoBoundCheck(this.dataVector, this.startOffset, this.valuesCount);
+  }
+
+  @Override
+  public Object getArray(long index, int count) throws SQLException {
+    return getArray(index, count, null);
+  }
+
+  private void checkBoundaries(long index, int count) {
+    if (index < 0 || index + count > this.startOffset + this.valuesCount) {
+      throw new ArrayIndexOutOfBoundsException();
+    }
+  }
+
+  private static Object getArrayNoBoundCheck(ValueVector dataVector, long start, long count) {
+    Object[] result = new Object[LargeMemoryUtil.checkedCastToInt(count)];
+    for (int i = 0; i < count; i++) {
+      result[i] = dataVector.getObject(LargeMemoryUtil.checkedCastToInt(start + i));
+    }
+
+    return result;
+  }
+
+  @Override
+  public Object getArray(long index, int count, Map<String, Class<?>> map) throws SQLException {
+    if (map != null) {
+      throw new SQLFeatureNotSupportedException();
+    }
+
+    checkBoundaries(index, count);
+    return getArrayNoBoundCheck(this.dataVector,
+        LargeMemoryUtil.checkedCastToInt(this.startOffset + index), count);
+  }
+
+  @Override
+  public ResultSet getResultSet() throws SQLException {
+    return this.getResultSet(null);
+  }
+
+  @Override
+  public ResultSet getResultSet(Map<String, Class<?>> map) throws SQLException {
+    if (map != null) {
+      throw new SQLFeatureNotSupportedException();
+    }
+
+    return getResultSetNoBoundariesCheck(this.dataVector, this.startOffset, this.valuesCount);
+  }
+
+  @Override
+  public ResultSet getResultSet(long index, int count) throws SQLException {
+    return getResultSet(index, count, null);
+  }
+
+  private static ResultSet getResultSetNoBoundariesCheck(ValueVector dataVector, long start,
+                                                         long count)
+      throws SQLException {
+    TransferPair transferPair = dataVector.getTransferPair(dataVector.getAllocator());
+    transferPair.splitAndTransfer(LargeMemoryUtil.checkedCastToInt(start),
+        LargeMemoryUtil.checkedCastToInt(count));
+    FieldVector vectorSlice = (FieldVector) transferPair.getTo();
+
+    VectorSchemaRoot vectorSchemaRoot = VectorSchemaRoot.of(vectorSlice);
+    return ArrowFlightJdbcVectorSchemaRootResultSet.fromVectorSchemaRoot(vectorSchemaRoot);
+  }
+
+  @Override
+  public ResultSet getResultSet(long index, int count, Map<String, Class<?>> map)
+      throws SQLException {
+    if (map != null) {
+      throw new SQLFeatureNotSupportedException();
+    }
+
+    checkBoundaries(index, count);
+    return getResultSetNoBoundariesCheck(this.dataVector,
+        LargeMemoryUtil.checkedCastToInt(this.startOffset + index), count);
+  }
+
+  @Override
+  public void free() {
+

Review Comment:
   Might it be reasonable to increment/decrement the underlying vector's reference count here? Though I suppose I see this is meant to be scoped to a transaction and presumably shouldn't be kept alive past the result set itself.



##########
java/flight/flight-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/utils/RootAllocatorTestRule.java:
##########
@@ -0,0 +1,820 @@
+/*
+ * 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.utils;
+
+import java.math.BigDecimal;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.Decimal256Vector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FixedSizeBinaryVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.LargeVarBinaryVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.TimeMicroVector;
+import org.apache.arrow.vector.TimeMilliVector;
+import org.apache.arrow.vector.TimeNanoVector;
+import org.apache.arrow.vector.TimeSecVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.TimeStampMilliTZVector;
+import org.apache.arrow.vector.TimeStampMilliVector;
+import org.apache.arrow.vector.TimeStampNanoTZVector;
+import org.apache.arrow.vector.TimeStampNanoVector;
+import org.apache.arrow.vector.TimeStampSecTZVector;
+import org.apache.arrow.vector.TimeStampSecVector;
+import org.apache.arrow.vector.TinyIntVector;
+import org.apache.arrow.vector.UInt1Vector;
+import org.apache.arrow.vector.UInt2Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.UInt8Vector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.complex.FixedSizeListVector;
+import org.apache.arrow.vector.complex.LargeListVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.impl.UnionFixedSizeListWriter;
+import org.apache.arrow.vector.complex.impl.UnionLargeListWriter;
+import org.apache.arrow.vector.complex.impl.UnionListWriter;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+public class RootAllocatorTestRule implements TestRule, AutoCloseable {

Review Comment:
   nit, but since this exposes BufferAllocator not RootAllocator, maybe this could just be AllocatorTestRule/getAllocator?



##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/accessor/impl/calendar/ArrowFlightJdbcTimeStampVectorAccessor.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.accessor.impl.calendar;
+
+import static org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeStampVectorGetter.Getter;
+import static org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeStampVectorGetter.Holder;
+import static org.apache.arrow.driver.jdbc.accessor.impl.calendar.ArrowFlightJdbcTimeStampVectorGetter.createGetter;
+
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDateTime;
+import java.time.temporal.ChronoUnit;
+import java.util.Calendar;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntSupplier;
+
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessor;
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessorFactory;
+import org.apache.arrow.vector.TimeStampVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.util.DateUtility;
+
+/**
+ * Accessor for the Arrow types extending from {@link TimeStampVector}.
+ */
+public class ArrowFlightJdbcTimeStampVectorAccessor extends ArrowFlightJdbcAccessor {
+
+  private final TimeZone timeZone;
+  private final Getter getter;
+  private final TimeUnit timeUnit;
+  private final LongToLocalDateTime longToLocalDateTime;
+  private final Holder holder;
+
+  /**
+   * Functional interface used to convert a number (in any time resolution) to LocalDateTime.
+   */
+  interface LongToLocalDateTime {
+    LocalDateTime fromLong(long value);
+  }
+
+  /**
+   * Instantiate a ArrowFlightJdbcTimeStampVectorAccessor for given vector.
+   */
+  public ArrowFlightJdbcTimeStampVectorAccessor(TimeStampVector vector,
+                                                IntSupplier currentRowSupplier,
+                                                ArrowFlightJdbcAccessorFactory.WasNullConsumer setCursorWasNull) {
+    super(currentRowSupplier, setCursorWasNull);
+    this.holder = new Holder();
+    this.getter = createGetter(vector);
+
+    this.timeZone = getTimeZoneForVector(vector);
+    this.timeUnit = getTimeUnitForVector(vector);
+    this.longToLocalDateTime = getLongToLocalDateTimeForVector(vector, this.timeZone);
+  }
+
+  @Override
+  public Class<?> getObjectClass() {
+    return Timestamp.class;
+  }
+
+  @Override
+  public Object getObject() {
+    return this.getTimestamp(null);
+  }
+
+  private LocalDateTime getLocalDateTime(Calendar calendar) {
+    getter.get(getCurrentRow(), holder);
+    this.wasNull = holder.isSet == 0;
+    this.wasNullConsumer.setWasNull(this.wasNull);
+    if (this.wasNull) {
+      return null;
+    }
+
+    long value = holder.value;
+
+    LocalDateTime localDateTime = this.longToLocalDateTime.fromLong(value);
+
+    if (calendar != null) {
+      TimeZone timeZone = calendar.getTimeZone();
+      long millis = this.timeUnit.toMillis(value);
+      localDateTime = localDateTime
+          .minus(timeZone.getOffset(millis) - this.timeZone.getOffset(millis), ChronoUnit.MILLIS);
+    }
+    return localDateTime;
+  }
+
+  @Override
+  public Date getDate(Calendar calendar) {
+    LocalDateTime localDateTime = getLocalDateTime(calendar);
+    if (localDateTime == null) {
+      return null;
+    }
+
+    return new Date(Timestamp.valueOf(localDateTime).getTime());
+  }
+
+  @Override
+  public Time getTime(Calendar calendar) {
+    LocalDateTime localDateTime = getLocalDateTime(calendar);
+    if (localDateTime == null) {
+      return null;
+    }
+
+    return new Time(Timestamp.valueOf(localDateTime).getTime());
+  }
+
+  @Override
+  public Timestamp getTimestamp(Calendar calendar) {
+    LocalDateTime localDateTime = getLocalDateTime(calendar);
+    if (localDateTime == null) {
+      return null;
+    }
+
+    return Timestamp.valueOf(localDateTime);
+  }
+
+  protected static TimeUnit getTimeUnitForVector(TimeStampVector vector) {
+    ArrowType.Timestamp arrowType =
+        (ArrowType.Timestamp) vector.getField().getFieldType().getType();
+
+    switch (arrowType.getUnit()) {
+      case NANOSECOND:
+        return TimeUnit.NANOSECONDS;
+      case MICROSECOND:
+        return TimeUnit.MICROSECONDS;
+      case MILLISECOND:
+        return TimeUnit.MILLISECONDS;
+      case SECOND:
+        return TimeUnit.SECONDS;
+      default:
+        throw new UnsupportedOperationException("Invalid Arrow time unit");
+    }

Review Comment:
   nit, but this would probably be nice as a helper method on the arrow time unit itself



##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightConnection.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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;
+
+import java.sql.SQLException;
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.arrow.driver.jdbc.client.ArrowFlightSqlClientHandler;
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl;
+import org.apache.arrow.flight.FlightClient;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaFactory;
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+
+/**
+ * Connection to the Arrow Flight server.
+ */
+public final class ArrowFlightConnection extends AvaticaConnection {
+
+  private final BufferAllocator allocator;
+  private final ArrowFlightSqlClientHandler clientHandler;
+  private final ArrowFlightConnectionConfigImpl config;
+  private ExecutorService executorService;
+
+  /**
+   * Creates a new {@link ArrowFlightConnection}.
+   *
+   * @param driver        the {@link ArrowFlightJdbcDriver} to use.
+   * @param factory       the {@link AvaticaFactory} to use.
+   * @param url           the URL to use.
+   * @param properties    the {@link Properties} to use.
+   * @param config        the {@link ArrowFlightConnectionConfigImpl} to use.
+   * @param allocator     the {@link BufferAllocator} to use.
+   * @param clientHandler the {@link ArrowFlightSqlClientHandler} to use.
+   */
+  private ArrowFlightConnection(final ArrowFlightJdbcDriver driver, final AvaticaFactory factory,
+                                final String url, final Properties properties,
+                                final ArrowFlightConnectionConfigImpl config,
+                                final BufferAllocator allocator,
+                                final ArrowFlightSqlClientHandler clientHandler) {
+    super(driver, factory, url, properties);
+    this.config = Preconditions.checkNotNull(config, "Config cannot be null.");
+    this.allocator = Preconditions.checkNotNull(allocator, "Allocator cannot be null.");
+    this.clientHandler = Preconditions.checkNotNull(clientHandler, "Handler cannot be null.");
+  }
+
+  /**
+   * Creates a new {@link ArrowFlightConnection} to a {@link FlightClient}.
+   *
+   * @param driver     the {@link ArrowFlightJdbcDriver} to use.
+   * @param factory    the {@link AvaticaFactory} to use.
+   * @param url        the URL to establish the connection to.
+   * @param properties the {@link Properties} to use for this session.
+   * @param allocator  the {@link BufferAllocator} to use.
+   * @return a new {@link ArrowFlightConnection}.
+   * @throws SQLException on error.
+   */
+  static ArrowFlightConnection createNewConnection(final ArrowFlightJdbcDriver driver,
+                                                   final AvaticaFactory factory,
+                                                   final String url, final Properties properties,
+                                                   final BufferAllocator allocator)
+      throws SQLException {
+    final ArrowFlightConnectionConfigImpl config = new ArrowFlightConnectionConfigImpl(properties);
+    final ArrowFlightSqlClientHandler clientHandler = createNewClientHandler(config, allocator);
+    return new ArrowFlightConnection(driver, factory, url, properties, config, allocator,
+        clientHandler);
+  }
+
+  private static ArrowFlightSqlClientHandler createNewClientHandler(
+      final ArrowFlightConnectionConfigImpl config,
+      final BufferAllocator allocator) throws SQLException {
+    try {
+      return new ArrowFlightSqlClientHandler.Builder()
+          .withHost(config.getHost())
+          .withPort(config.getPort())
+          .withUsername(config.getUser())
+          .withPassword(config.getPassword())
+          .withKeyStorePath(config.getKeyStorePath())
+          .withKeyStorePassword(config.keystorePassword())
+          .withBufferAllocator(allocator)
+          .withTlsEncryption(config.useTls())
+          .withDisableCertificateVerification(config.getDisableCertificateVerification())
+          .withToken(config.getToken())
+          .withCallOptions(config.toCallOption())
+          .build();
+    } catch (final SQLException e) {
+      try {
+        allocator.close();
+      } catch (final Exception allocatorCloseEx) {
+        e.addSuppressed(allocatorCloseEx);
+      }
+      throw e;
+    }
+  }
+
+  void reset() throws SQLException {
+    // Clean up any open Statements
+    try {
+      AutoCloseables.close(statementMap.values());
+    } catch (final Exception e) {
+      throw AvaticaConnection.HELPER.createException(e.getMessage(), e);
+    }
+
+    statementMap.clear();
+
+    // Reset Holdability
+    this.setHoldability(this.metaData.getResultSetHoldability());
+
+    // Reset Meta
+    ((ArrowFlightMetaImpl) this.meta).setDefaultConnectionProperties();
+  }
+
+  /**
+   * Gets the client {@link #clientHandler} backing this connection.
+   *
+   * @return the handler.
+   */
+  ArrowFlightSqlClientHandler getClientHandler() throws SQLException {
+    return clientHandler;
+  }
+
+  /**
+   * Gets the {@link ExecutorService} of this connection.
+   *
+   * @return the {@link #executorService}.
+   */
+  synchronized ExecutorService getExecutorService() {
+    return executorService = executorService == null ?
+        Executors.newFixedThreadPool(config.threadPoolSize(),
+            new DefaultThreadFactory(getClass().getSimpleName())) :
+        executorService;
+  }
+
+  @Override
+  public Properties getClientInfo() {
+    final Properties copy = new Properties();
+    copy.putAll(info);
+    return copy;
+  }
+
+  @Override
+  public void close() throws SQLException {
+    if (executorService != null) {
+      executorService.shutdown();
+    }
+
+    try {
+      AutoCloseables.close(clientHandler);
+      allocator.getChildAllocators().forEach(AutoCloseables::closeNoChecked);
+      AutoCloseables.close(allocator);
+
+      super.close();

Review Comment:
   It might be good to do this all in one call to AutoCloseables to make sure things get closed even if one of the close calls fails. Also, I'm curious why we have to explicitly clean up child allocators here



##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static final String CONNECTION_STRING_EXPECTED = "jdbc:arrow-flight://[host][:port][?param1=value&...]";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }
+
+    new ArrowFlightJdbcDriver().register();
+  }
+
+  @Override
+  public ArrowFlightConnection connect(final String url, final Properties info)
+      throws SQLException {
+    final Properties properties = new Properties(info);
+    properties.putAll(info);

Review Comment:
   Isn't the second call redundant?



##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcCursor.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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;
+
+
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import org.apache.arrow.driver.jdbc.accessor.ArrowFlightJdbcAccessorFactory;
+import org.apache.arrow.util.AutoCloseables;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.util.AbstractCursor;
+import org.apache.calcite.avatica.util.ArrayImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Arrow Flight Jdbc's Cursor class.
+ */
+public class ArrowFlightJdbcCursor extends AbstractCursor {
+
+  private static final Logger LOGGER;
+  private final VectorSchemaRoot root;
+  private final int rowCount;
+  private int currentRow = -1;
+
+  static {
+    LOGGER = LoggerFactory.getLogger(ArrowFlightJdbcCursor.class);

Review Comment:
   nit: why is this not initialized inline above?



##########
java/flight/flight-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/utils/RootAllocatorTestRule.java:
##########
@@ -0,0 +1,820 @@
+/*
+ * 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.utils;
+
+import java.math.BigDecimal;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.Decimal256Vector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FixedSizeBinaryVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.LargeVarBinaryVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.TimeMicroVector;
+import org.apache.arrow.vector.TimeMilliVector;
+import org.apache.arrow.vector.TimeNanoVector;
+import org.apache.arrow.vector.TimeSecVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.TimeStampMilliTZVector;
+import org.apache.arrow.vector.TimeStampMilliVector;
+import org.apache.arrow.vector.TimeStampNanoTZVector;
+import org.apache.arrow.vector.TimeStampNanoVector;
+import org.apache.arrow.vector.TimeStampSecTZVector;
+import org.apache.arrow.vector.TimeStampSecVector;
+import org.apache.arrow.vector.TinyIntVector;
+import org.apache.arrow.vector.UInt1Vector;
+import org.apache.arrow.vector.UInt2Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.UInt8Vector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.complex.FixedSizeListVector;
+import org.apache.arrow.vector.complex.LargeListVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.impl.UnionFixedSizeListWriter;
+import org.apache.arrow.vector.complex.impl.UnionLargeListWriter;
+import org.apache.arrow.vector.complex.impl.UnionListWriter;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+public class RootAllocatorTestRule implements TestRule, AutoCloseable {
+
+  public static final byte MAX_VALUE = Byte.MAX_VALUE;
+  private final BufferAllocator rootAllocator = new RootAllocator();
+
+  private final Random random = new Random(10);
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    return new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+        try {
+          base.evaluate();
+        } finally {
+          close();
+        }
+      }
+    };
+  }
+
+  public BufferAllocator getRootAllocator() {
+    return rootAllocator;
+  }
+
+  @Override
+  public void close() throws Exception {
+    this.rootAllocator.getChildAllocators().forEach(BufferAllocator::close);

Review Comment:
   Hmm, generally code that creates a child allocator should be responsible for cleaning it up itself



##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/utils/ClientAuthenticationUtils.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.utils;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Enumeration;
+import java.util.List;
+
+import org.apache.arrow.flight.CallOption;
+import org.apache.arrow.flight.FlightClient;
+import org.apache.arrow.flight.auth2.BasicAuthCredentialWriter;
+import org.apache.arrow.flight.auth2.ClientIncomingAuthHeaderMiddleware;
+import org.apache.arrow.flight.grpc.CredentialCallOption;
+import org.apache.arrow.util.Preconditions;
+import org.bouncycastle.openssl.jcajce.JcaPEMWriter;
+
+/**
+ * Utils for {@link FlightClientHandler} authentication.
+ */
+public final class ClientAuthenticationUtils {
+
+  private ClientAuthenticationUtils() {
+    // Prevent instantiation.
+  }
+
+  /**
+   * Gets the {@link CredentialCallOption} for the provided authentication info.
+   *
+   * @param client      the client.
+   * @param credential  the credential as CallOptions.
+   * @param options     the {@link CallOption}s to use.
+   * @return the credential call option.
+   */
+  public static CredentialCallOption getAuthenticate(final FlightClient client,
+                                                     final CredentialCallOption credential,
+                                                     final CallOption... options) {
+
+    final List<CallOption> theseOptions = new ArrayList<>();
+    theseOptions.add(credential);
+    theseOptions.addAll(Arrays.asList(options));
+    client.handshake(theseOptions.toArray(new CallOption[0]));
+
+    return (CredentialCallOption) theseOptions.get(0);

Review Comment:
   Why not return `credential` directly?



##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/ArrowFlightJdbcDriver.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.Preconditions;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.org.apache.http.NameValuePair;
+import org.apache.calcite.avatica.org.apache.http.client.utils.URLEncodedUtils;
+
+/**
+ * JDBC driver for querying data from an Apache Arrow Flight server.
+ */
+public class ArrowFlightJdbcDriver extends UnregisteredDriver {
+
+  private static final String CONNECT_STRING_PREFIX = "jdbc:arrow-flight://";
+  private static final String CONNECTION_STRING_EXPECTED = "jdbc:arrow-flight://[host][:port][?param1=value&...]";
+  private static DriverVersion version;
+
+  static {
+    // Special code for supporting Java9 and higher.
+    // Netty requires some extra properties to unlock some native memory management api
+    // Setting this property if not already set externally
+    // This has to be done before any netty class is being loaded
+    final String key = "cfjd.io.netty.tryReflectionSetAccessible";
+    final String tryReflectionSetAccessible = System.getProperty(key);
+    if (tryReflectionSetAccessible == null) {
+      System.setProperty(key, Boolean.TRUE.toString());
+    }

Review Comment:
   This may surprise people; I think we discussed this before though and we don't want to force people to set this property everywhere? (Which is understandable…) It should be called out in documentation, though.



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] jduo commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
jduo commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1092182615

   This PR is a duplicate of #12254 , except targeting the flight-sql-jdbc branch. It's intended to be a snapshot of the prior PR for the purpose of IP clearance.


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1095493260

   @jduo would you like to rebase to try to reduce the committer list?
   
   Also at some point, this needs to have a stable Git commit hash so we can actually fill out the paperwork and hold the vote


-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r846124672


##########
java/flight/flight-jdbc-driver/pom.xml:
##########
@@ -0,0 +1,367 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>arrow-flight</artifactId>
+        <groupId>org.apache.arrow</groupId>
+        <version>8.0.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>flight-jdbc-driver</artifactId>
+    <name>Arrow Flight JDBC Driver</name>
+    <description>(Contrib/Experimental)A library for querying data using a JDBC driver for Arrow Flight.</description>
+    <packaging>jar</packaging>
+    <url>http://maven.apache.org</url>
+
+    <properties>
+        <org.apache.arrow.flight.name>${pom.parent.groupId}:${pom.parent.artifactId}</org.apache.arrow.flight.name>
+        <org.apache.arrow.flight.version>${pom.parent.version}</org.apache.arrow.flight.version>
+        <org.apache.arrow.flight.jdbc-driver.name>${pom.name}</org.apache.arrow.flight.jdbc-driver.name>
+        <org.apache.arrow.flight.jdbc-driver.version>${pom.version}</org.apache.arrow.flight.jdbc-driver.version>
+        <jacoco.ut.execution.data.file>${project.build.directory}/coverage-reports/jacoco-ut.html</jacoco.ut.execution.data.file>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.arrow</groupId>
+            <artifactId>flight-core</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+            <exclusion>
+                <groupId>io.netty</groupId>
+                <artifactId>netty-transport-native-unix-common</artifactId>
+            </exclusion>
+            <exclusion>
+                <groupId>io.netty</groupId>
+                <artifactId>netty-transport-native-kqueue</artifactId>
+                </exclusion>
+            <exclusion>
+	            <groupId>io.netty</groupId>
+                <artifactId>netty-transport-native-epoll</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.apache.arrow/arrow-memory-core -->

Review Comment:
   nit: is it necessary to include these comments?



##########
java/flight/flight-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/utils/ThrowableAssertionUtils.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.utils;
+
+/**
+ * Utility class to avoid upgrading JUnit to version >= 4.13 and keep using code to assert a {@link Throwable}.

Review Comment:
   Hmm, let's file a JIRA to do this (and actually I think we have a mix of junit 4 and 5 here…it would be good if we could upgrade and also standardize on a single matchers framework like Truth or hamcrest or something while we're at it)



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #12830:
URL: https://github.com/apache/arrow/pull/12830#discussion_r929919087


##########
java/flight/flight-jdbc-driver/src/main/java/org/apache/arrow/driver/jdbc/client/ArrowFlightSqlClientHandler.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.FlightRuntimeException;
+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()
+        .map(FlightEndpoint::getTicket)
+        .map(ticket -> sqlClient.getStream(ticket, getOptions()))

Review Comment:
   I see James followed up already. Thanks for taking a look!



-- 
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-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #12830: ARROW-15452: [FlightRPC][Java] JDBC driver for Flight SQL

Posted by GitBox <gi...@apache.org>.
lidavidm commented on PR #12830:
URL: https://github.com/apache/arrow/pull/12830#issuecomment-1164989360

   Looks like everyone's filed ICLAs. I'll get the paperwork filed ASAP


-- 
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-unsubscribe@arrow.apache.org

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