You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "aiguofer (via GitHub)" <gi...@apache.org> on 2023/02/01 06:05:46 UTC

[GitHub] [arrow] aiguofer opened a new pull request, #33967: GH-33953: [Java] Pass custom headers on every request

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

   <!--
   Thanks for opening a pull request!
   If this is your first pull request you can find detailed information on how 
   to contribute here:
     * [New Contributor's Guide](https://arrow.apache.org/docs/dev/developers/guide/step_by_step/pr_lifecycle.html#reviews-and-merge-of-the-pull-request)
     * [Contributing Overview](https://arrow.apache.org/docs/dev/developers/overview.html)
   
   
   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 GitHub? https://github.com/apache/arrow/issues/new/choose
   
   Opening GitHub issues 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 the pull request title in the following format?
   
       GH-${GITHUB_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   In the case of PARQUET issues on JIRA the title also supports:
   
       PARQUET-${JIRA_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   -->
   
   ### Rationale for this change
   
   Some flight requests don't send custom headers. This PR should fix that.
   
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   ### What changes are included in this PR?
   
   Ensure custom headers are sent across on every request.
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   ### Are these changes tested?
   
   No
   
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   2. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are they covered by existing tests)?
   -->
   
   ### Are there any user-facing changes?
   
   Custom headers should now be attached to every call.
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please uncomment the line below and explain which changes are breaking.
   -->
   <!-- **This PR includes breaking changes to public APIs.** -->
   
   <!--
   Please uncomment the line below (and provide explanation) if the changes fix either (a) a security vulnerability, (b) a bug that caused incorrect or invalid data to be produced, or (c) a bug that causes a crash (even when the API contract is upheld). We use this to highlight fixes to issues that may affect users without their knowledge. For this reason, fixing bugs that cause errors don't count, since those are usually obvious.
   -->
   <!-- **This PR contains a "Critical Fix".** -->


-- 
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] aiguofer commented on a diff in pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "aiguofer (via GitHub)" <gi...@apache.org>.
aiguofer commented on code in PR #33967:
URL: https://github.com/apache/arrow/pull/33967#discussion_r1096954236


##########
java/flight/flight-sql-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/CustomHeaderTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.authentication.UserPasswordAuthentication;
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.driver.jdbc.utils.MockFlightSqlProducer;
+import org.apache.arrow.flight.FlightMethod;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+/**
+ * Tests for custom properties set on JDBC connection being passed as headers to the server.
+ */
+public class CustomHeaderTest {
+
+  @ClassRule
+  public static final FlightServerTestRule FLIGHT_SERVER_TEST_RULE;
+  private static final MockFlightSqlProducer PRODUCER = new MockFlightSqlProducer();
+  private static final String userTest = "user1";
+  private static final String passTest = "pass1";
+
+  static {
+    UserPasswordAuthentication authentication =
+        new UserPasswordAuthentication.Builder()
+            .user(userTest, passTest)
+            .build();
+
+    FLIGHT_SERVER_TEST_RULE = new FlightServerTestRule.Builder()
+        .authentication(authentication)
+        .producer(PRODUCER)
+        .build();
+  }
+
+  private BufferAllocator allocator;
+
+  @Before
+  public void setUp() throws Exception {
+    allocator = new RootAllocator(Long.MAX_VALUE);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    allocator.getChildAllocators().forEach(BufferAllocator::close);
+    AutoCloseables.close(allocator);
+  }
+
+  /**
+   * Checks that custom headers make it to the server on HANDSHAKE.
+   *
+   * @throws SQLException on error.
+   */
+  @Test
+  public void testCustomHeaders()
+      throws Exception {
+    final Properties properties = new Properties();
+
+    properties.put(ArrowFlightConnectionProperty.HOST.camelName(), "localhost");
+    properties.put(ArrowFlightConnectionProperty.PORT.camelName(),
+        FLIGHT_SERVER_TEST_RULE.getPort());
+    properties.put(ArrowFlightConnectionProperty.USER.camelName(),
+        userTest);
+    properties.put(ArrowFlightConnectionProperty.PASSWORD.camelName(),
+        passTest);
+    properties.put("useEncryption", false);
+    properties.put("answer", "42");
+    properties.put("has_towel", "true");
+
+    try (Connection connection = DriverManager.getConnection(
+        "jdbc:arrow-flight-sql://" + FLIGHT_SERVER_TEST_RULE.getHost() + ":" +
+            FLIGHT_SERVER_TEST_RULE.getPort(), properties)) {
+
+      try {
+        // We don't care if the call fails, we just want to make sure the headers got passed
+        connection.getMetaData().getTableTypes();

Review Comment:
   It'd be great to have a sequence of calls that tests most grpc methods, but at this point I'm not familiar enough with Arrow SQL to write that.
   
   Depending on what those calls might look like, it could be better to have multiple tests, but figured this simple test is at least a good starting point.



-- 
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] aiguofer commented on a diff in pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "aiguofer (via GitHub)" <gi...@apache.org>.
aiguofer commented on code in PR #33967:
URL: https://github.com/apache/arrow/pull/33967#discussion_r1097850258


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java:
##########
@@ -353,10 +355,11 @@ public void onCompleted() {
    */
   public ExchangeReaderWriter doExchange(FlightDescriptor descriptor, CallOption... options) {
     Preconditions.checkNotNull(descriptor, "descriptor must not be null");
-    final io.grpc.CallOptions callOptions = CallOptions.wrapStub(asyncStub, options).getCallOptions();
+    FlightServiceStub stub = CallOptions.wrapStub(asyncStub, options);
+    final io.grpc.CallOptions callOptions = stub.getCallOptions();
 
     try {
-      final ClientCall<ArrowMessage, ArrowMessage> call = interceptedChannel.newCall(doExchangeDescriptor, callOptions);
+      final ClientCall<ArrowMessage, ArrowMessage> call = stub.getChannel().newCall(doExchangeDescriptor, callOptions);

Review Comment:
   Oh I meant this would be great as a public method in `FlightServiceStub`. There's a lot of code in there that could use it, but I believe that is an auto-generated source file so I don't know how to change it.
   
   I went ahead and created a private method in `FlightClient` to do the same.



-- 
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] aiguofer commented on pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "aiguofer (via GitHub)" <gi...@apache.org>.
aiguofer commented on PR #33967:
URL: https://github.com/apache/arrow/pull/33967#issuecomment-1412589940

   Unsure what that failure is... maybe the `ManagedChannel` shutdown is not being propagated to other channels? Or maybe it's not waiting long enough for the shutdown to complete (there's a hardcoded 5 sec wait there I believe)


-- 
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 #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm merged PR #33967:
URL: https://github.com/apache/arrow/pull/33967


-- 
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 #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #33967:
URL: https://github.com/apache/arrow/pull/33967#discussion_r1096273528


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java:
##########
@@ -353,10 +355,11 @@ public void onCompleted() {
    */
   public ExchangeReaderWriter doExchange(FlightDescriptor descriptor, CallOption... options) {
     Preconditions.checkNotNull(descriptor, "descriptor must not be null");
-    final io.grpc.CallOptions callOptions = CallOptions.wrapStub(asyncStub, options).getCallOptions();
+    FlightServiceStub stub = CallOptions.wrapStub(asyncStub, options);
+    final io.grpc.CallOptions callOptions = stub.getCallOptions();
 
     try {
-      final ClientCall<ArrowMessage, ArrowMessage> call = interceptedChannel.newCall(doExchangeDescriptor, callOptions);
+      final ClientCall<ArrowMessage, ArrowMessage> call = stub.getChannel().newCall(doExchangeDescriptor, callOptions);

Review Comment:
   Is it possible to factor out a helper that can take care of the wrapStub - getChannel - newCall dance?



-- 
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] ursabot commented on pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "ursabot (via GitHub)" <gi...@apache.org>.
ursabot commented on PR #33967:
URL: https://github.com/apache/arrow/pull/33967#issuecomment-1447336394

   Benchmark runs are scheduled for baseline = 06f28d206ed5ec60750e353d99062f83e47cd0a7 and contender = 969dd290ab93c8629b643134f150e71c1c05c2ff. 969dd290ab93c8629b643134f150e71c1c05c2ff is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/0312836ff28649fea770451ad11688f0...35d46d9ea3bf4839a3ff72c7441fdbc5/)
   [Failed :arrow_down:0.79% :arrow_up:0.15%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/4a18a59960b444e29856168efed7235c...7f8a5de7370e419d8a51b89f5ff26bad/)
   [Finished :arrow_down:1.53% :arrow_up:2.04%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/4b9863e6fbac43639301500c4e5d930d...d3640798a67940e5aa21ea088aabed93/)
   [Failed :arrow_down:0.29% :arrow_up:0.03%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/7b810a7440b64dd99021440515c8a487...4c4c63db178c425094a26af78f69af6b/)
   Buildkite builds:
   [Finished] [`969dd290` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2441)
   [Failed] [`969dd290` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2471)
   [Finished] [`969dd290` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2438)
   [Failed] [`969dd290` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2462)
   [Finished] [`06f28d20` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2440)
   [Finished] [`06f28d20` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2470)
   [Finished] [`06f28d20` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2437)
   [Finished] [`06f28d20` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2461)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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 #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #33967:
URL: https://github.com/apache/arrow/pull/33967#discussion_r1097610270


##########
java/flight/flight-sql-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/CustomHeaderTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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;

Review Comment:
   Please keep the tests in flight-core



##########
java/flight/flight-sql-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/CustomHeaderTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.authentication.UserPasswordAuthentication;
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.driver.jdbc.utils.MockFlightSqlProducer;
+import org.apache.arrow.flight.FlightMethod;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+/**
+ * Tests for custom properties set on JDBC connection being passed as headers to the server.
+ */
+public class CustomHeaderTest {
+
+  @ClassRule
+  public static final FlightServerTestRule FLIGHT_SERVER_TEST_RULE;

Review Comment:
   Instead of the TestRule (which won't work in JUnit 5) please use the test() pattern that most Flight tests use (e.g. see TestClientMiddleware.java)



##########
java/flight/flight-sql-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/CustomHeaderTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.authentication.UserPasswordAuthentication;
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.driver.jdbc.utils.MockFlightSqlProducer;
+import org.apache.arrow.flight.FlightMethod;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+/**
+ * Tests for custom properties set on JDBC connection being passed as headers to the server.
+ */
+public class CustomHeaderTest {
+
+  @ClassRule
+  public static final FlightServerTestRule FLIGHT_SERVER_TEST_RULE;
+  private static final MockFlightSqlProducer PRODUCER = new MockFlightSqlProducer();
+  private static final String userTest = "user1";
+  private static final String passTest = "pass1";
+
+  static {
+    UserPasswordAuthentication authentication =
+        new UserPasswordAuthentication.Builder()
+            .user(userTest, passTest)
+            .build();
+
+    FLIGHT_SERVER_TEST_RULE = new FlightServerTestRule.Builder()
+        .authentication(authentication)
+        .producer(PRODUCER)
+        .build();
+  }
+
+  private BufferAllocator allocator;
+
+  @Before
+  public void setUp() throws Exception {
+    allocator = new RootAllocator(Long.MAX_VALUE);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    allocator.getChildAllocators().forEach(BufferAllocator::close);
+    AutoCloseables.close(allocator);
+  }
+
+  /**
+   * Checks that custom headers make it to the server on HANDSHAKE.
+   *
+   * @throws SQLException on error.
+   */
+  @Test
+  public void testCustomHeaders()
+      throws Exception {
+    final Properties properties = new Properties();
+
+    properties.put(ArrowFlightConnectionProperty.HOST.camelName(), "localhost");
+    properties.put(ArrowFlightConnectionProperty.PORT.camelName(),
+        FLIGHT_SERVER_TEST_RULE.getPort());
+    properties.put(ArrowFlightConnectionProperty.USER.camelName(),
+        userTest);
+    properties.put(ArrowFlightConnectionProperty.PASSWORD.camelName(),
+        passTest);
+    properties.put("useEncryption", false);
+    properties.put("answer", "42");
+    properties.put("has_towel", "true");
+
+    try (Connection connection = DriverManager.getConnection(
+        "jdbc:arrow-flight-sql://" + FLIGHT_SERVER_TEST_RULE.getHost() + ":" +
+            FLIGHT_SERVER_TEST_RULE.getPort(), properties)) {
+
+      try {
+        // We don't care if the call fails, we just want to make sure the headers got passed
+        connection.getMetaData().getTableTypes();

Review Comment:
   It'd be better to do a low-level test of the Flight client since that's what the issue is about, instead of indirectly testing it through the JDBC driver



-- 
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] aiguofer commented on pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "aiguofer (via GitHub)" <gi...@apache.org>.
aiguofer commented on PR #33967:
URL: https://github.com/apache/arrow/pull/33967#issuecomment-1444786855

   for the future... is there some command I can run to auto-format code to checkstyle? I haven't thought about checkstyles compliance in years, this should be automated.


-- 
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] aiguofer commented on pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "aiguofer (via GitHub)" <gi...@apache.org>.
aiguofer commented on PR #33967:
URL: https://github.com/apache/arrow/pull/33967#issuecomment-1444762616

   @lidavidm Now that I'm a bit more familiar with the code base I was able to write some better unit tests. I moved the tests to `flight-core`, removed the use of rules, and I'm now testing each specific flight method instead of using the JDBC connection proxy.


-- 
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] aiguofer commented on a diff in pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "aiguofer (via GitHub)" <gi...@apache.org>.
aiguofer commented on code in PR #33967:
URL: https://github.com/apache/arrow/pull/33967#discussion_r1096380457


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java:
##########
@@ -353,10 +355,11 @@ public void onCompleted() {
    */
   public ExchangeReaderWriter doExchange(FlightDescriptor descriptor, CallOption... options) {
     Preconditions.checkNotNull(descriptor, "descriptor must not be null");
-    final io.grpc.CallOptions callOptions = CallOptions.wrapStub(asyncStub, options).getCallOptions();
+    FlightServiceStub stub = CallOptions.wrapStub(asyncStub, options);
+    final io.grpc.CallOptions callOptions = stub.getCallOptions();
 
     try {
-      final ClientCall<ArrowMessage, ArrowMessage> call = interceptedChannel.newCall(doExchangeDescriptor, callOptions);
+      final ClientCall<ArrowMessage, ArrowMessage> call = stub.getChannel().newCall(doExchangeDescriptor, callOptions);

Review Comment:
   it'd be nice to add something like:
   
   ```
       public <RequestT, ResponseT> ClientCall<RequestT, ResponseT> newCall(MethodDescriptor<RequestT, ResponseT> descriptor) {
         return getChannel().newCall(descriptor, getCallOptions());
       }
   ```
   
   to the stub so we could just do `stub.newCall(descriptor)`, but it looks like that class is auto-generated so I'm not sure where to add 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] aiguofer commented on pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "aiguofer (via GitHub)" <gi...@apache.org>.
aiguofer commented on PR #33967:
URL: https://github.com/apache/arrow/pull/33967#issuecomment-1416531347

   Sure I'll take a stab at writing some unit tests. I was thinking maybe something like `ConnectionTest` with a middleware that checks for the specific header values. 


-- 
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 #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #33967:
URL: https://github.com/apache/arrow/pull/33967#issuecomment-1414407216

   It'll be a while before I can review here but that test is known to be flaky


-- 
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] aiguofer commented on pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "aiguofer (via GitHub)" <gi...@apache.org>.
aiguofer commented on PR #33967:
URL: https://github.com/apache/arrow/pull/33967#issuecomment-1416347829

   Great, good to know! We're currently using a JDBC driver jar I built with these changes for development of our service so we're not currently blocked. Hoping these changes can make it into the next release so we can have users use a published jar once we get to beta. I really appreciate your help and time!


-- 
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] aiguofer commented on a diff in pull request #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "aiguofer (via GitHub)" <gi...@apache.org>.
aiguofer commented on code in PR #33967:
URL: https://github.com/apache/arrow/pull/33967#discussion_r1097851383


##########
java/flight/flight-sql-jdbc-driver/src/test/java/org/apache/arrow/driver/jdbc/CustomHeaderTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.arrow.driver.jdbc.authentication.UserPasswordAuthentication;
+import org.apache.arrow.driver.jdbc.utils.ArrowFlightConnectionConfigImpl.ArrowFlightConnectionProperty;
+import org.apache.arrow.driver.jdbc.utils.MockFlightSqlProducer;
+import org.apache.arrow.flight.FlightMethod;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.util.AutoCloseables;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+/**
+ * Tests for custom properties set on JDBC connection being passed as headers to the server.
+ */
+public class CustomHeaderTest {
+
+  @ClassRule
+  public static final FlightServerTestRule FLIGHT_SERVER_TEST_RULE;
+  private static final MockFlightSqlProducer PRODUCER = new MockFlightSqlProducer();
+  private static final String userTest = "user1";
+  private static final String passTest = "pass1";
+
+  static {
+    UserPasswordAuthentication authentication =
+        new UserPasswordAuthentication.Builder()
+            .user(userTest, passTest)
+            .build();
+
+    FLIGHT_SERVER_TEST_RULE = new FlightServerTestRule.Builder()
+        .authentication(authentication)
+        .producer(PRODUCER)
+        .build();
+  }
+
+  private BufferAllocator allocator;
+
+  @Before
+  public void setUp() throws Exception {
+    allocator = new RootAllocator(Long.MAX_VALUE);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    allocator.getChildAllocators().forEach(BufferAllocator::close);
+    AutoCloseables.close(allocator);
+  }
+
+  /**
+   * Checks that custom headers make it to the server on HANDSHAKE.
+   *
+   * @throws SQLException on error.
+   */
+  @Test
+  public void testCustomHeaders()
+      throws Exception {
+    final Properties properties = new Properties();
+
+    properties.put(ArrowFlightConnectionProperty.HOST.camelName(), "localhost");
+    properties.put(ArrowFlightConnectionProperty.PORT.camelName(),
+        FLIGHT_SERVER_TEST_RULE.getPort());
+    properties.put(ArrowFlightConnectionProperty.USER.camelName(),
+        userTest);
+    properties.put(ArrowFlightConnectionProperty.PASSWORD.camelName(),
+        passTest);
+    properties.put("useEncryption", false);
+    properties.put("answer", "42");
+    properties.put("has_towel", "true");
+
+    try (Connection connection = DriverManager.getConnection(
+        "jdbc:arrow-flight-sql://" + FLIGHT_SERVER_TEST_RULE.getHost() + ":" +
+            FLIGHT_SERVER_TEST_RULE.getPort(), properties)) {
+
+      try {
+        // We don't care if the call fails, we just want to make sure the headers got passed
+        connection.getMetaData().getTableTypes();

Review Comment:
   Yeah makes sense, unfortunately I don't know the code well enough to do that. I know I encountered the issue by having properties in the JDBC connection that didn't make it to the server, and that's how I've been testing that the changes work.
   
   Unfortunately I don't think I have the time right now to dig into it to write better unit tests... some help there would be great.



-- 
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 #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #33967:
URL: https://github.com/apache/arrow/pull/33967#discussion_r1097608310


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java:
##########
@@ -353,10 +355,11 @@ public void onCompleted() {
    */
   public ExchangeReaderWriter doExchange(FlightDescriptor descriptor, CallOption... options) {
     Preconditions.checkNotNull(descriptor, "descriptor must not be null");
-    final io.grpc.CallOptions callOptions = CallOptions.wrapStub(asyncStub, options).getCallOptions();
+    FlightServiceStub stub = CallOptions.wrapStub(asyncStub, options);
+    final io.grpc.CallOptions callOptions = stub.getCallOptions();
 
     try {
-      final ClientCall<ArrowMessage, ArrowMessage> call = interceptedChannel.newCall(doExchangeDescriptor, callOptions);
+      final ClientCall<ArrowMessage, ArrowMessage> call = stub.getChannel().newCall(doExchangeDescriptor, callOptions);

Review Comment:
   It can just be a private method, no?



-- 
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 #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on code in PR #33967:
URL: https://github.com/apache/arrow/pull/33967#discussion_r1097852255


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java:
##########
@@ -353,10 +355,11 @@ public void onCompleted() {
    */
   public ExchangeReaderWriter doExchange(FlightDescriptor descriptor, CallOption... options) {
     Preconditions.checkNotNull(descriptor, "descriptor must not be null");
-    final io.grpc.CallOptions callOptions = CallOptions.wrapStub(asyncStub, options).getCallOptions();
+    FlightServiceStub stub = CallOptions.wrapStub(asyncStub, options);
+    final io.grpc.CallOptions callOptions = stub.getCallOptions();
 
     try {
-      final ClientCall<ArrowMessage, ArrowMessage> call = interceptedChannel.newCall(doExchangeDescriptor, callOptions);
+      final ClientCall<ArrowMessage, ArrowMessage> call = stub.getChannel().newCall(doExchangeDescriptor, callOptions);

Review Comment:
   It's generated by gRPC, so it's not under our control



-- 
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 #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #33967:
URL: https://github.com/apache/arrow/pull/33967#issuecomment-1446333522

   > for the future... is there some command I can run to auto-format code to checkstyle? I haven't thought about checkstyles compliance in years, this should be automated.
   
   Unfortunately no. I would much prefer if we could switch to a formatter like google-java-format rather than checkstyle but that would be a somewhat disruptive refactor.
   
   That said - feel free to raise an issue and get opinions from other Java contributors.


-- 
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 #33967: GH-33953: [Java] Pass custom headers on every request

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm commented on PR #33967:
URL: https://github.com/apache/arrow/pull/33967#issuecomment-1446343108

   Actually, I filed https://github.com/apache/arrow/issues/34364 for checkstyle - feel free to chime in there


-- 
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