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 2020/08/18 19:16:52 UTC

[GitHub] [arrow] jduo opened a new pull request #7994: Flight auth redesign

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


   - Remove authentication payloads from the handshake protobuf messages
   - Add server middleware which utilizes ServerAuthHandler instead of an interceptor
   - Change FlightClient builder to taken CallCredentials and supply auth
   - information via HTTP headers.
   - Add client middleware to handle reading/writing the bearer token post-handshake.


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

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



[GitHub] [arrow] lidavidm closed pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
lidavidm closed pull request #7994:
URL: https://github.com/apache/arrow/pull/7994


   


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

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



[GitHub] [arrow] jduo commented on pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

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


   > Some minor notes...I think we should resolve how exactly to treat Handshake in the design docs, though
   
   @lidavidm @pitrou , I've added a comment in the design doc about how we can change the handshake proposal as well as how sessions would fit into this.


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

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



[GitHub] [arrow] ryannicholson commented on a change in pull request #7994: Flight auth redesign

Posted by GitBox <gi...@apache.org>.
ryannicholson commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r472617468



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ClientBearerTokenMiddleware.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.flight.auth;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightClientMiddleware;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Middleware for capturing and sending back bearer tokens.
+ */
+public class ClientBearerTokenMiddleware implements FlightClientMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ClientBearerTokenMiddleware.class);
+
+  private final String bearerToken;
+
+  /**
+   * Factory used within FlightClient.
+   */
+  public static class Factory implements FlightClientMiddleware.Factory {
+    private final String bearerToken = null;
+
+    @Override
+    public FlightClientMiddleware onCallStarted(CallInfo info) {
+      logger.debug("Call name: {}", info.method().name());
+      if (info.method().name().equalsIgnoreCase(AuthConstants.HANDSHAKE_DESCRIPTOR_NAME)) {
+        return new ClientAuthHandshakeMiddleware(this);
+      }
+
+      if (bearerToken == null) {

Review comment:
       Wouldn't this always evaluate as null as line 40 has a final member of the same name initialized to null?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ServerAuthMiddleware.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flight.auth;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.FlightServerMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.grpc.Context;
+
+/**
+ * Middleware that's used to validate credentials during the handshake and verify
+ * the bearer token in subsequent requests.
+ */
+public class ServerAuthMiddleware implements FlightServerMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ServerAuthMiddleware.class);
+
+  /**
+   * Factory for accessing ServerAuthMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerAuthMiddleware> {
+    private final ServerAuthHandler authHandler;
+    private final GeneratedBearerTokenAuthHandler bearerTokenAuthHandler = new GeneratedBearerTokenAuthHandler();
+    private final Map<String, String> tokenToIdentityMap = new ConcurrentHashMap<>();

Review comment:
       This looks unused.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ServerHandshakeWrapper.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.flight.auth;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Contains utility methods for integrating authorization into a GRPC stream.
+ */
+public class ServerHandshakeWrapper {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ServerHandshakeWrapper.class);

Review comment:
       This looks unused.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ServerInterceptorAdapter.java
##########
@@ -87,10 +87,12 @@ public ServerInterceptorAdapter(List<KeyFactory<?>> factories) {
     // Use LinkedHashMap to preserve insertion order
     final Map<FlightServerMiddleware.Key<?>, FlightServerMiddleware> middlewareMap = new LinkedHashMap<>();
     final MetadataAdapter headerAdapter = new MetadataAdapter(headers);
+    Context currentContext = Context.current();
     for (final KeyFactory<?> factory : factories) {
       final FlightServerMiddleware m;
       try {
         m = factory.factory.onCallStarted(info, headerAdapter);
+        currentContext = m.onAuthenticationSuccess(currentContext);

Review comment:
       Could we find an alternative to changing the context while looping through the factories? Instead of having "onAuthenticationSuccess" as a new function in FlightServerMiddleware, could the logic required here be moved into ServerAuthMiddleware and have this check happen through the normal process of calling "onCallStarted"?

##########
File path: java/flight/flight-core/src/test/java/org/apache/arrow/flight/auth/TestBasicAuth.java
##########
@@ -44,29 +44,38 @@
 import org.junit.Ignore;
 import org.junit.Test;
 
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 
 public class TestBasicAuth {
 
   private static final String USERNAME = "flight";
   private static final String PASSWORD = "woohoo";
-  private static final byte[] VALID_TOKEN = "my_token".getBytes(StandardCharsets.UTF_8);
+  private static final String VALID_TOKEN = "my_token";
 
+  private FlightClient.Builder clientBuilder;
   private FlightClient client;
   private FlightServer server;
   private BufferAllocator allocator;
 
   @Test
   public void validAuth() {
-    client.authenticateBasic(USERNAME, PASSWORD);
-    Assert.assertTrue(ImmutableList.copyOf(client.listFlights(Criteria.ALL)).size() == 0);
+    try {
+      client = clientBuilder.callCredentials(new BasicAuthCallCredentials(USERNAME, PASSWORD)).build();
+      client.handshake();
+      Assert.assertTrue(ImmutableList.copyOf(client.listFlights(Criteria.ALL)).size() == 0);
+    } catch (FlightRuntimeException ex) {
+      ex.printStackTrace();
+      System.out.println(ex.status());

Review comment:
       Are these needed?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ServerAuthMiddleware.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flight.auth;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.FlightServerMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.grpc.Context;
+
+/**
+ * Middleware that's used to validate credentials during the handshake and verify
+ * the bearer token in subsequent requests.
+ */
+public class ServerAuthMiddleware implements FlightServerMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ServerAuthMiddleware.class);
+
+  /**
+   * Factory for accessing ServerAuthMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerAuthMiddleware> {
+    private final ServerAuthHandler authHandler;
+    private final GeneratedBearerTokenAuthHandler bearerTokenAuthHandler = new GeneratedBearerTokenAuthHandler();
+    private final Map<String, String> tokenToIdentityMap = new ConcurrentHashMap<>();
+
+    public Factory(ServerAuthHandler authHandler) {
+      this.authHandler = authHandler;
+    }
+
+    public String getPeerForBearer(String bearerToken) {
+      return bearerTokenAuthHandler.getIdentityForBearerToken(bearerToken);
+    }
+
+    @Override
+    public ServerAuthMiddleware onCallStarted(CallInfo callInfo, CallHeaders incomingHeaders) {
+      logger.debug("Call name: {}", callInfo.method().name());
+      if (callInfo.method().name().equals(AuthConstants.HANDSHAKE_DESCRIPTOR_NAME)) {
+        final ServerAuthHandler.HandshakeResult result = authHandler.authenticate(incomingHeaders);
+        final String bearerToken = bearerTokenAuthHandler.registerBearer(result);
+        return new ServerAuthMiddleware(result.getPeerIdentity(), bearerToken);
+      }
+
+      final String bearerToken = AuthUtilities.getValueFromAuthHeader(incomingHeaders, AuthConstants.BEARER_PREFIX);
+      // No bearer token provided. Auth handler may explicitly allow this.
+      if (bearerToken == null) {
+        if (authHandler.validateBearer(null)) {
+          return new ServerAuthMiddleware("", null);
+        }
+        logger.info("Client did not supply a bearer token.");
+        throw new FlightRuntimeException(CallStatus.UNAUTHENTICATED);
+      }
+
+      if (!authHandler.validateBearer(bearerToken) && !bearerTokenAuthHandler.validateBearer(bearerToken)) {
+        logger.info("Bearer token supplied by client was not authorized.");
+        throw new FlightRuntimeException(CallStatus.UNAUTHORIZED);
+      }
+
+      final String peerIdentity = bearerTokenAuthHandler.getIdentityForBearerToken(bearerToken);
+      return new ServerAuthMiddleware(peerIdentity, null);

Review comment:
       Perhaps there is opportunity for an optimization down the road in caching the ServerAuthMiddleware instances with the identity?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/BasicServerAuthHandler.java
##########
@@ -39,37 +42,58 @@ public BasicServerAuthHandler(BasicAuthValidator authValidator) {
     this.authValidator = authValidator;
   }
 
-  /**
-   * Interface that this handler delegates for determining if credentials are valid.
-   */
-  public interface BasicAuthValidator {
+  @Override
+  public HandshakeResult authenticate(CallHeaders headers) {
+    final String authEncoded = AuthUtilities.getValueFromAuthHeader(headers, AuthConstants.BASIC_PREFIX);
+    if (authEncoded == null) {
+      throw new FlightRuntimeException(CallStatus.UNAUTHENTICATED);
+    }
 
-    byte[] getToken(String username, String password) throws Exception;
+    try {
+      // The value has the format Base64(<username>:<password>)
+      final String authDecoded = new String(Base64.getDecoder().decode(authEncoded), StandardCharsets.UTF_8);
+      final String[] authInParts = authDecoded.split(":");
+      if (authInParts.length < 2) {
+        throw new FlightRuntimeException(CallStatus.UNAUTHORIZED);
+      }
 
-    Optional<String> isValid(byte[] token);
+      final String user = authInParts[0];
+      final String[] passwordParts = Arrays.copyOfRange(authInParts, 1, authInParts.length);
+      final String password = String.join(":", passwordParts);

Review comment:
       Would an indexOf(":") call with a substring each for username/password be fewer operations?

##########
File path: java/flight/flight-core/src/test/java/org/apache/arrow/flight/auth/TestBasicAuth.java
##########
@@ -44,29 +44,38 @@
 import org.junit.Ignore;
 import org.junit.Test;
 
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 
 public class TestBasicAuth {
 
   private static final String USERNAME = "flight";
   private static final String PASSWORD = "woohoo";
-  private static final byte[] VALID_TOKEN = "my_token".getBytes(StandardCharsets.UTF_8);
+  private static final String VALID_TOKEN = "my_token";
 
+  private FlightClient.Builder clientBuilder;
   private FlightClient client;
   private FlightServer server;
   private BufferAllocator allocator;
 
   @Test
   public void validAuth() {
-    client.authenticateBasic(USERNAME, PASSWORD);
-    Assert.assertTrue(ImmutableList.copyOf(client.listFlights(Criteria.ALL)).size() == 0);
+    try {
+      client = clientBuilder.callCredentials(new BasicAuthCallCredentials(USERNAME, PASSWORD)).build();
+      client.handshake();
+      Assert.assertTrue(ImmutableList.copyOf(client.listFlights(Criteria.ALL)).size() == 0);

Review comment:
       Could this be changed to assertEquals?




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

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



[GitHub] [arrow] pitrou commented on pull request #7994: Flight auth redesign

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#issuecomment-675987912


   cc @lidavidm 


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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r513690592



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientHandshakeWrapper.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.flight.auth2;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {

Review comment:
       ClientHandshakeWrapper has been simplified, because the handshake using auth2 is no longer iterative.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientHandshakeWrapper.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.flight.auth2;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {
+          // TODO: ARROW-5681
+          throw new RuntimeException("Unauthenticated");
+        }
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw ex;
+      } catch (ExecutionException ex) {
+        final FlightRuntimeException wrappedException = StatusUtils.fromThrowable(ex.getCause());
+        logger.error("Failed on completing future", wrappedException);
+        throw wrappedException;
+      }
+    } catch (StatusRuntimeException sre) {
+      logger.error("Failed with SREe", sre);
+      throw StatusUtils.fromGrpcRuntimeException(sre);
+    } catch (Throwable ex) {
+      logger.error("Failed with unknown", ex);
+      if (ex instanceof FlightRuntimeException) {
+        throw (FlightRuntimeException) ex;
+      }
+      throw StatusUtils.fromThrowable(ex);
+    }
+  }
+
+  private static class HandshakeObserver implements StreamObserver<HandshakeResponse> {
+
+    private volatile StreamObserver<HandshakeRequest> requestObserver;
+    private final CompletableFuture<Boolean> completed;
+
+    public HandshakeObserver() {
+      super();
+      completed = new CompletableFuture<>();
+    }
+
+    @Override
+    public void onNext(HandshakeResponse value) {
+      logger.debug("Got HandshakeResponse");

Review comment:
       Done




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r482260625



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ServerAuthMiddleware.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.flight.auth;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightServerMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Middleware that's used to validate credentials during the handshake and verify
+ * the bearer token in subsequent requests.
+ */
+public class ServerAuthMiddleware implements FlightServerMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ServerAuthMiddleware.class);
+
+  /**
+   * Factory for accessing ServerAuthMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerAuthMiddleware> {
+    private final ServerAuthHandler authHandler;
+    private final GeneratedBearerTokenAuthHandler bearerTokenAuthHandler;
+
+    /**
+     * Construct a factory with the given auth handler.
+     * @param authHandler The auth handler what will be used for authenticating requests.
+     */
+    public Factory(ServerAuthHandler authHandler) {
+      this.authHandler = authHandler;
+      bearerTokenAuthHandler = authHandler.enableCachedCredentials() ?
+          new GeneratedBearerTokenAuthHandler() : null;
+    }
+
+    @Override
+    public ServerAuthMiddleware onCallStarted(CallInfo callInfo, CallHeaders incomingHeaders, CallContext context) {
+      logger.debug("Call name: {}", callInfo.method().name());
+      // Check if bearer token auth is being used, and if we've enabled use of server-generated
+      // bearer tokens.
+      if (authHandler.enableCachedCredentials()) {
+        final String bearerTokenFromHeaders =
+            AuthUtilities.getValueFromAuthHeader(incomingHeaders, AuthConstants.BEARER_PREFIX);
+        if (bearerTokenFromHeaders != null) {
+          final ServerAuthHandler.AuthResult result = bearerTokenAuthHandler.authenticate(incomingHeaders);

Review comment:
       No, the idea is you don't know which header you need to authenticate from. We're going to change this such that bearer token generation is moved out of this middleware and into an implementation of ServerAuthHandler which will make this more claer.




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r477542825



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ServerInterceptorAdapter.java
##########
@@ -87,10 +87,12 @@ public ServerInterceptorAdapter(List<KeyFactory<?>> factories) {
     // Use LinkedHashMap to preserve insertion order
     final Map<FlightServerMiddleware.Key<?>, FlightServerMiddleware> middlewareMap = new LinkedHashMap<>();
     final MetadataAdapter headerAdapter = new MetadataAdapter(headers);
+    Context currentContext = Context.current();
     for (final KeyFactory<?> factory : factories) {
       final FlightServerMiddleware m;
       try {
         m = factory.factory.onCallStarted(info, headerAdapter);
+        currentContext = m.onAuthenticationSuccess(currentContext);

Review comment:
       I created a facade for this and supplied it as an input to onCallStarted.




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r509802432



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java
##########
@@ -156,23 +150,12 @@
   }
 
   /**
-   * Authenticates with a username and password.

Review comment:
       I've restored the old authentication code and moved all new authentication APIs to org.apache.arrow.flight.auth2. I've also restored old authentication tests as well as made a variant of TestBasicAuth for testing Auth2 APIs.




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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r506759118



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/CallOptions.java
##########
@@ -56,7 +56,7 @@ public static CallOption timeout(long duration, TimeUnit unit) {
   /**
    * CallOptions specific to GRPC stubs.
    */
-  interface GrpcCallOption extends CallOption {
+  public interface GrpcCallOption extends CallOption {

Review comment:
       Hmm, that is a problem. We also don't want to expose gRPC details from the core package, though...leaving this as-is is probably simplest, though, seeing as a non-gRPC flight implementation is still a ways off.




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

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



[GitHub] [arrow] keeratsingh commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
keeratsingh commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r507971481



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/CallOptions.java
##########
@@ -56,7 +56,7 @@ public static CallOption timeout(long duration, TimeUnit unit) {
   /**
    * CallOptions specific to GRPC stubs.
    */
-  interface GrpcCallOption extends CallOption {
+  public interface GrpcCallOption extends CallOption {

Review comment:
       Thank you for the prompt response @lidavidm, keeping this as-is.




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r513694924



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/RequestContext.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.flight;
+
+/**
+ * Tracks variables about the current request.
+ */
+public interface RequestContext {
+  /**
+   * Register a variable and a value.
+   * @param key the variable name.
+   * @param value the value.
+   */
+  void put(String key, String value);
+
+  /**
+   * Retrieve a registered variable.
+   * @param key the variable name.
+   * @return the value, or empty string if not found.
+   */
+  String get(String key);
+}

Review comment:
       Done




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r477543928



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java
##########
@@ -84,18 +81,15 @@
    * Create a Flight client from an allocator and a gRPC channel.
    */
   FlightClient(BufferAllocator incomingAllocator, ManagedChannel channel,
-      List<FlightClientMiddleware.Factory> middleware) {
+      List<FlightClientMiddleware.Factory> middleware, CallCredentials callCredentials) {
     this.allocator = incomingAllocator.newChildAllocator("flight-client", 0, Long.MAX_VALUE);
     this.channel = channel;
 
-    final ClientInterceptor[] interceptors;
-    interceptors = new ClientInterceptor[]{authInterceptor, new ClientInterceptorAdapter(middleware)};
-
     // Create a channel with interceptors pre-applied for DoGet and DoPut
-    this.interceptedChannel = ClientInterceptors.intercept(channel, interceptors);
+    this.interceptedChannel = ClientInterceptors.intercept(channel, new ClientInterceptorAdapter(middleware));
 
-    blockingStub = FlightServiceGrpc.newBlockingStub(interceptedChannel);
-    asyncStub = FlightServiceGrpc.newStub(interceptedChannel);
+    blockingStub = FlightServiceGrpc.newBlockingStub(interceptedChannel).withCallCredentials(callCredentials);

Review comment:
       I've added a CredentialCallOptions class which creates and adds CallCredentials to the gRPC stub.




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

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



[GitHub] [arrow] lidavidm commented on pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

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


   Looks like master has the same failure for the Java JNI test, so I've merged this. Thanks again @jduo!


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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r513686533



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.RequestContext;
+import org.apache.arrow.flight.auth2.Auth2Constants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements RequestContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();
+  private static final Context.Key<String> authkey = Context.keyWithDefault(Auth2Constants.PEER_IDENTITY_KEY, "");
+  private Context context = Context.current();
+
+  /**
+   * Retrieves the gRPC context.
+   * @return the gRPC context.
+   */
+  public Context getContext() {
+    return context;
+  }
+
+  @Override
+  public void put(String key, String value) {
+    context = context.withValue(authkey, value);

Review comment:
       I've removed ContextAdapter, and implemented what you suggested earlier (store a map in the gRPC context and expose the map to the middleware methods).




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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r478675271



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/example/integration/AuthBasicProtoScenario.java
##########
@@ -58,40 +58,37 @@ public void doAction(CallContext context, Action action, StreamListener<Result>
   public void buildServer(FlightServer.Builder builder) {
     builder.authHandler(new BasicServerAuthHandler(new BasicServerAuthHandler.BasicAuthValidator() {
       @Override
-      public byte[] getToken(String username, String password) throws Exception {
+      public Optional<String> validateCredentials(String username, String password) throws Exception {
+        if (Strings.isNullOrEmpty(username)) {
+          throw CallStatus.UNAUTHORIZED.withDescription("Credentials not supplied").toRuntimeException();
+        }
+
         if (!USERNAME.equals(username) || !PASSWORD.equals(password)) {
-          throw CallStatus.UNAUTHENTICATED.withDescription("Username or password is invalid.").toRuntimeException();
+          throw CallStatus.UNAUTHORIZED.withDescription("Username or password is invalid.").toRuntimeException();
         }
-        return ("valid:" + username).getBytes(StandardCharsets.UTF_8);
+        return Optional.of("valid:" + username);
       }
 
       @Override
-      public Optional<String> isValid(byte[] token) {
-        if (token != null) {
-          final String credential = new String(token, StandardCharsets.UTF_8);
-          if (credential.startsWith("valid:")) {
-            return Optional.of(credential.substring(6));
-          }
+      public Optional<String> isValid(String token) {
+        if (token.startsWith("valid:")) {
+          return Optional.of(token.substring(6));
         }
         return Optional.empty();
       }
     }));
   }
 
   @Override
-  public void client(BufferAllocator allocator, Location location, FlightClient client) {
-    final FlightRuntimeException e = IntegrationAssertions.assertThrows(FlightRuntimeException.class, () -> {
-      client.listActions().forEach(act -> {
-      });
-    });
-    if (!FlightStatusCode.UNAUTHENTICATED.equals(e.status().code())) {
-      throw new AssertionError("Expected UNAUTHENTICATED but found " + e.status().code(), e);
-    }
-
-    client.authenticate(new BasicClientAuthHandler(USERNAME, PASSWORD));
-    final Result result = client.doAction(new Action("")).next();
-    if (!USERNAME.equals(new String(result.getBody(), StandardCharsets.UTF_8))) {
-      throw new AssertionError("Expected " + USERNAME + " but got " + Arrays.toString(result.getBody()));
+  public void client(BufferAllocator allocator, Location location, FlightClient.Builder clientBuilder) {

Review comment:
       We should still test what happens if the client tries to make an unauthenticated request.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.auth.AuthConstants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements CallContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();

Review comment:
       An internal synchronized cache may be surprising...I'd say if we have to have a cache, it should just use ConcurrentHashMap, and preferably, we should just wrap Context.Key<T> ourselves.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ClientHandshakeWrapper.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.flight.auth;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {
+          // TODO: ARROW-5681
+          throw new RuntimeException("Unauthenticated");
+        }
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw ex;

Review comment:
       Does this actually build? We're throwing a checked exception here

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ServerAuthMiddleware.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.flight.auth;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightServerMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Middleware that's used to validate credentials during the handshake and verify
+ * the bearer token in subsequent requests.
+ */
+public class ServerAuthMiddleware implements FlightServerMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ServerAuthMiddleware.class);
+
+  /**
+   * Factory for accessing ServerAuthMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerAuthMiddleware> {
+    private final ServerAuthHandler authHandler;
+    private final GeneratedBearerTokenAuthHandler bearerTokenAuthHandler;
+
+    /**
+     * Construct a factory with the given auth handler.
+     * @param authHandler The auth handler what will be used for authenticating requests.
+     */
+    public Factory(ServerAuthHandler authHandler) {
+      this.authHandler = authHandler;
+      bearerTokenAuthHandler = authHandler.enableCachedCredentials() ?
+          new GeneratedBearerTokenAuthHandler() : null;
+    }
+
+    @Override
+    public ServerAuthMiddleware onCallStarted(CallInfo callInfo, CallHeaders incomingHeaders, CallContext context) {
+      logger.debug("Call name: {}", callInfo.method().name());

Review comment:
       Let's maybe not spam logs here; logging should go into dedicated logging middleware.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ServerAuthMiddleware.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.flight.auth;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightServerMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Middleware that's used to validate credentials during the handshake and verify
+ * the bearer token in subsequent requests.
+ */
+public class ServerAuthMiddleware implements FlightServerMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ServerAuthMiddleware.class);
+
+  /**
+   * Factory for accessing ServerAuthMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerAuthMiddleware> {
+    private final ServerAuthHandler authHandler;
+    private final GeneratedBearerTokenAuthHandler bearerTokenAuthHandler;
+
+    /**
+     * Construct a factory with the given auth handler.
+     * @param authHandler The auth handler what will be used for authenticating requests.
+     */
+    public Factory(ServerAuthHandler authHandler) {
+      this.authHandler = authHandler;
+      bearerTokenAuthHandler = authHandler.enableCachedCredentials() ?
+          new GeneratedBearerTokenAuthHandler() : null;
+    }
+
+    @Override
+    public ServerAuthMiddleware onCallStarted(CallInfo callInfo, CallHeaders incomingHeaders, CallContext context) {
+      logger.debug("Call name: {}", callInfo.method().name());
+      // Check if bearer token auth is being used, and if we've enabled use of server-generated
+      // bearer tokens.
+      if (authHandler.enableCachedCredentials()) {

Review comment:
       Is it possible for logic like this to be handled by the auth handler itself? I don't see the value of hardcoding a particular caching strategy here.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ClientHandshakeWrapper.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.flight.auth;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {
+          // TODO: ARROW-5681
+          throw new RuntimeException("Unauthenticated");
+        }
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw ex;

Review comment:
       Oh I see, it gets caught and rethrown below

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ServerAuthMiddleware.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.flight.auth;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightServerMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Middleware that's used to validate credentials during the handshake and verify
+ * the bearer token in subsequent requests.
+ */
+public class ServerAuthMiddleware implements FlightServerMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ServerAuthMiddleware.class);
+
+  /**
+   * Factory for accessing ServerAuthMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerAuthMiddleware> {
+    private final ServerAuthHandler authHandler;
+    private final GeneratedBearerTokenAuthHandler bearerTokenAuthHandler;
+
+    /**
+     * Construct a factory with the given auth handler.
+     * @param authHandler The auth handler what will be used for authenticating requests.
+     */
+    public Factory(ServerAuthHandler authHandler) {
+      this.authHandler = authHandler;
+      bearerTokenAuthHandler = authHandler.enableCachedCredentials() ?
+          new GeneratedBearerTokenAuthHandler() : null;
+    }
+
+    @Override
+    public ServerAuthMiddleware onCallStarted(CallInfo callInfo, CallHeaders incomingHeaders, CallContext context) {
+      logger.debug("Call name: {}", callInfo.method().name());
+      // Check if bearer token auth is being used, and if we've enabled use of server-generated
+      // bearer tokens.
+      if (authHandler.enableCachedCredentials()) {
+        final String bearerTokenFromHeaders =
+            AuthUtilities.getValueFromAuthHeader(incomingHeaders, AuthConstants.BEARER_PREFIX);
+        if (bearerTokenFromHeaders != null) {
+          final ServerAuthHandler.AuthResult result = bearerTokenAuthHandler.authenticate(incomingHeaders);
+          context.put(AuthConstants.PEER_IDENTITY_KEY, result.getPeerIdentity());
+          return new ServerAuthMiddleware(result.getPeerIdentity(), result.getBearerToken().get());
+        }
+      }
+
+      // Delegate to server auth handler to do the validation.
+      final ServerAuthHandler.AuthResult result = authHandler.authenticate(incomingHeaders);
+      final String bearerToken;
+      if (authHandler.enableCachedCredentials()) {
+        bearerToken = bearerTokenAuthHandler.registerBearer(result);
+      } else {
+        bearerToken = result.getBearerToken().get();
+      }
+      context.put(AuthConstants.PEER_IDENTITY_KEY, result.getPeerIdentity());
+      return new ServerAuthMiddleware(result.getPeerIdentity(), bearerToken);
+    }
+  }
+
+  private final String bearerToken;
+  private final String peerIdentity;
+
+  public ServerAuthMiddleware(String peerIdentity, String bearerToken) {
+    this.peerIdentity = peerIdentity;
+    this.bearerToken = bearerToken;
+  }
+
+  @Override
+  public void onBeforeSendingHeaders(CallHeaders outgoingHeaders) {
+    if (bearerToken != null &&

Review comment:
       The same goes for outbound headers - an auth handler should be able to send whatever outbound headers it might need, instead of hardcoding a particular header here. SPNEGO is an example of an auth scheme that uses WWW-Negotiate instead of Authorization in responses, for example.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ClientHandshakeWrapper.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.flight.auth;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {
+          // TODO: ARROW-5681
+          throw new RuntimeException("Unauthenticated");
+        }
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw ex;
+      } catch (ExecutionException ex) {
+        logger.error("Failed on completing future", ex.getCause());
+        throw ex.getCause();

Review comment:
       This should probably be `StatusUtils.fromThrowable(ex.getCause())`

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/example/integration/AuthBasicProtoScenario.java
##########
@@ -58,40 +58,37 @@ public void doAction(CallContext context, Action action, StreamListener<Result>
   public void buildServer(FlightServer.Builder builder) {
     builder.authHandler(new BasicServerAuthHandler(new BasicServerAuthHandler.BasicAuthValidator() {
       @Override
-      public byte[] getToken(String username, String password) throws Exception {
+      public Optional<String> validateCredentials(String username, String password) throws Exception {
+        if (Strings.isNullOrEmpty(username)) {
+          throw CallStatus.UNAUTHORIZED.withDescription("Credentials not supplied").toRuntimeException();

Review comment:
       Unauthorized != Unauthenticated, this should be Unauthenticated. The same goes on like 67 below.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/BearerTokenAuthHandler.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flight.auth;
+
+import java.util.Optional;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightRuntimeException;
+
+/**
+ * Partial implementation of ServerAuthHandler for bearer-token based authentication.
+ */
+abstract class BearerTokenAuthHandler implements ServerAuthHandler {
+  @Override
+  public AuthResult authenticate(CallHeaders headers) {
+    final String bearerToken = AuthUtilities.getValueFromAuthHeader(headers, AuthConstants.BEARER_PREFIX);
+    if (bearerToken == null) {
+      throw new FlightRuntimeException(CallStatus.UNAUTHENTICATED);
+    }
+
+    if (!validateBearer(bearerToken)) {
+      throw new FlightRuntimeException(CallStatus.UNAUTHORIZED);

Review comment:
       This should also be Unauthenticated




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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r510146622



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.RequestContext;
+import org.apache.arrow.flight.auth2.Auth2Constants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements RequestContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();
+  private static final Context.Key<String> authkey = Context.keyWithDefault(Auth2Constants.PEER_IDENTITY_KEY, "");
+  private Context context = Context.current();
+
+  /**
+   * Retrieves the gRPC context.
+   * @return the gRPC context.
+   */
+  public Context getContext() {
+    return context;
+  }
+
+  @Override
+  public void put(String key, String value) {
+    context = context.withValue(authkey, value);

Review comment:
       Is it intentional to ignore key here?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.RequestContext;
+import org.apache.arrow.flight.auth2.Auth2Constants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements RequestContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();
+  private static final Context.Key<String> authkey = Context.keyWithDefault(Auth2Constants.PEER_IDENTITY_KEY, "");
+  private Context context = Context.current();
+
+  /**
+   * Retrieves the gRPC context.
+   * @return the gRPC context.
+   */
+  public Context getContext() {
+    return context;
+  }
+
+  @Override
+  public void put(String key, String value) {
+    context = context.withValue(authkey, value);
+  }
+
+  @Override
+  public String get(String key) {
+    return authkey.get(context);
+  }
+
+  private static Context.Key<String> getGrpcKey(String key) {

Review comment:
       This looks unused?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientHandshakeWrapper.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.flight.auth2;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {
+          // TODO: ARROW-5681
+          throw new RuntimeException("Unauthenticated");
+        }
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw ex;
+      } catch (ExecutionException ex) {
+        final FlightRuntimeException wrappedException = StatusUtils.fromThrowable(ex.getCause());
+        logger.error("Failed on completing future", wrappedException);

Review comment:
       I don't know if we need to have all these log statements - the application should handle the exception and log if needed right?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java
##########
@@ -176,6 +184,32 @@ public void authenticate(ClientAuthHandler handler, CallOption... options) {
     authInterceptor.setAuthHandler(handler);
   }
 
+  /**
+   * Authenticates with a username and password.
+   *
+   * @param username the username.
+   * @param password the password.
+   * @return a CredentialCallOption containing a bearer token if the server emitted one, or
+   *     empty if no bearer token was returned. This can be used in subsequent API calls.
+   */
+  public Optional<CredentialCallOption> basicHeaderAuthenticate(String username, String password) {

Review comment:
       nit: maybe 'authenticateBasicToken'?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.auth.AuthConstants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements CallContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();

Review comment:
       Thinking about it, would a better implementation be to store a `Map<String, String>` implementing RequestContext inside the gRPC context, and just pull it out of the context wherever needed? That way you don't have to deal with gRPC context keys. (I do like the context key pattern because it is typed, though!)

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientHandshakeWrapper.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.flight.auth2;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {

Review comment:
       Ah but then again, the timeout will be set on the gRPC stub so this shouldn't block forever.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/BasicCallHeaderAuthenticator.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.flight.auth2;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Optional;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A ServerAuthHandler for username/password authentication.
+ */
+public class BasicCallHeaderAuthenticator implements CallHeaderAuthenticator {
+
+  private static final Logger logger = LoggerFactory.getLogger(BasicCallHeaderAuthenticator.class);
+  private final BasicAuthValidator authValidator;
+
+  public BasicCallHeaderAuthenticator(BasicAuthValidator authValidator) {
+    super();
+    this.authValidator = authValidator;
+  }
+
+  @Override
+  public AuthResult authenticate(CallHeaders headers) {
+    final String authEncoded = AuthUtilities.getValueFromAuthHeader(headers, Auth2Constants.BASIC_PREFIX);
+    if (authEncoded == null) {
+      throw CallStatus.UNAUTHENTICATED.toRuntimeException();
+    }
+
+    try {
+      // The value has the format Base64(<username>:<password>)
+      final String authDecoded = new String(Base64.getDecoder().decode(authEncoded), StandardCharsets.UTF_8);
+      final int colonPos = authDecoded.indexOf(':');
+      if (colonPos == -1) {
+        throw CallStatus.UNAUTHORIZED.toRuntimeException();
+      }
+
+      final String user = authDecoded.substring(0, colonPos);
+      final String password = authDecoded.substring(colonPos + 1);
+      final Optional<String> bearerToken = authValidator.validateCredentials(user, password);
+      return new AuthResult() {
+        @Override
+        public String getPeerIdentity() {
+          return user;
+        }
+
+        @Override
+        public Optional<String> getBearerToken() {
+          return bearerToken;
+        }
+      };
+
+    } catch (UnsupportedEncodingException ex) {
+      throw CallStatus.INTERNAL.withCause(ex).toRuntimeException();
+    } catch (FlightRuntimeException ex) {
+      throw ex;
+    } catch (Exception ex) {
+      throw CallStatus.UNAUTHORIZED.withCause(ex).toRuntimeException();
+    }
+  }
+
+  @Override
+  public boolean validateBearer(String bearerToken) {
+    return false;

Review comment:
       Should this delegate to authValidator?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/RequestContext.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.flight;
+
+/**
+ * Tracks variables about the current request.
+ */
+public interface RequestContext {
+  /**
+   * Register a variable and a value.
+   * @param key the variable name.
+   * @param value the value.
+   */
+  void put(String key, String value);
+
+  /**
+   * Retrieve a registered variable.
+   * @param key the variable name.
+   * @return the value, or empty string if not found.
+   */
+  String get(String key);
+}

Review comment:
       Might it be useful to also provide a `keySet()` method, even if we don't want the full Map interface?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.RequestContext;
+import org.apache.arrow.flight.auth2.Auth2Constants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements RequestContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();
+  private static final Context.Key<String> authkey = Context.keyWithDefault(Auth2Constants.PEER_IDENTITY_KEY, "");
+  private Context context = Context.current();
+
+  /**
+   * Retrieves the gRPC context.
+   * @return the gRPC context.
+   */
+  public Context getContext() {
+    return context;
+  }
+
+  @Override
+  public void put(String key, String value) {
+    context = context.withValue(authkey, value);

Review comment:
       Ditto in `get` below.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientHandshakeWrapper.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.flight.auth2;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {
+          // TODO: ARROW-5681
+          throw new RuntimeException("Unauthenticated");

Review comment:
       Maybe throw a CallStatus.UNAUTHENTICATED here?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientHandshakeWrapper.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.flight.auth2;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {
+          // TODO: ARROW-5681
+          throw new RuntimeException("Unauthenticated");
+        }
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw ex;
+      } catch (ExecutionException ex) {
+        final FlightRuntimeException wrappedException = StatusUtils.fromThrowable(ex.getCause());
+        logger.error("Failed on completing future", wrappedException);
+        throw wrappedException;
+      }
+    } catch (StatusRuntimeException sre) {
+      logger.error("Failed with SREe", sre);
+      throw StatusUtils.fromGrpcRuntimeException(sre);
+    } catch (Throwable ex) {
+      logger.error("Failed with unknown", ex);
+      if (ex instanceof FlightRuntimeException) {
+        throw (FlightRuntimeException) ex;
+      }
+      throw StatusUtils.fromThrowable(ex);
+    }
+  }
+
+  private static class HandshakeObserver implements StreamObserver<HandshakeResponse> {
+
+    private volatile StreamObserver<HandshakeRequest> requestObserver;
+    private final CompletableFuture<Boolean> completed;
+
+    public HandshakeObserver() {
+      super();
+      completed = new CompletableFuture<>();
+    }
+
+    @Override
+    public void onNext(HandshakeResponse value) {
+      logger.debug("Got HandshakeResponse");

Review comment:
       Ditto about logs - we shouldn't log things unless they're really going to be useful for an application developer.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.auth.AuthConstants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements CallContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();

Review comment:
       Also of note, this should at least be a ConcurrentMap since it's static and shared by multiple requests.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientHandshakeWrapper.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.flight.auth2;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {

Review comment:
       It might be good to accept a timeout for this. (CallOptions... in FlightClient#handshake already has a timeout so it would be nice to respect that.)

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientBearerTokenMiddleware.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flight.auth2;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightClientMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Middleware for capturing bearer tokens sent back from the Flight server.
+ */
+public class ClientBearerTokenMiddleware implements FlightClientMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ClientBearerTokenMiddleware.class);
+
+  private final Factory factory;
+
+  /**
+   * Factory used within FlightClient.
+   */
+  public static class Factory implements FlightClientMiddleware.Factory {
+    private final AtomicReference<String> bearerToken = new AtomicReference<>();
+
+    @Override
+    public FlightClientMiddleware onCallStarted(CallInfo info) {
+      logger.debug("Call name: {}", info.method().name());

Review comment:
       Let's not log this.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/RequestContext.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.flight;
+
+/**
+ * Tracks variables about the current request.
+ */
+public interface RequestContext {
+  /**
+   * Register a variable and a value.
+   * @param key the variable name.
+   * @param value the value.
+   */
+  void put(String key, String value);
+
+  /**
+   * Retrieve a registered variable.
+   * @param key the variable name.
+   * @return the value, or empty string if not found.
+   */
+  String get(String key);
+}

Review comment:
       And/or containsKey and delete.




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

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



[GitHub] [arrow] wesm commented on pull request #7994: Flight auth redesign

Posted by GitBox <gi...@apache.org>.
wesm commented on pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#issuecomment-675665511


   This PR includes protocol changes. These will have to be discussed on the mailing list to be accepted


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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r477543461



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java
##########
@@ -156,23 +150,12 @@
   }
 
   /**
-   * Authenticates with a username and password.

Review comment:
       I renamed authenticate() to handshake() now, because authentication is now supplied as a CallOption. I brought back authenticateBasic() and also now have it return a CallOption containing the bearer token credentials if the server sent it back.




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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r479681106



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.auth.AuthConstants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements CallContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();

Review comment:
       You could implement a 
   ```java
   class ContextKey<T> {
       private final Context.Key<T> key;
   }
   ```
   and give it the same restriction (instances are compared by identity, not value), 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.

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r477768470



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ServerInterceptorAdapter.java
##########
@@ -87,10 +87,12 @@ public ServerInterceptorAdapter(List<KeyFactory<?>> factories) {
     // Use LinkedHashMap to preserve insertion order
     final Map<FlightServerMiddleware.Key<?>, FlightServerMiddleware> middlewareMap = new LinkedHashMap<>();
     final MetadataAdapter headerAdapter = new MetadataAdapter(headers);
+    Context currentContext = Context.current();
     for (final KeyFactory<?> factory : factories) {
       final FlightServerMiddleware m;
       try {
         m = factory.factory.onCallStarted(info, headerAdapter);
+        currentContext = m.onAuthenticationSuccess(currentContext);

Review comment:
       I added another input to onCallStarted() that the implementor can use to write context-specific variable 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.

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



[GitHub] [arrow] lidavidm commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r477591584



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/CallOptions.java
##########
@@ -56,7 +56,7 @@ public static CallOption timeout(long duration, TimeUnit unit) {
   /**
    * CallOptions specific to GRPC stubs.
    */
-  interface GrpcCallOption extends CallOption {
+  public interface GrpcCallOption extends CallOption {

Review comment:
       This can't be public in this module - it should have a public subinterface in flight-grpc.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/CallContext.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.flight;
+
+/**
+ * Tracks variables about the current request.
+ */
+public interface CallContext {

Review comment:
       Let's try not to rename existing classes unnecessarily and instead come up with new names. 

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightRuntimeException.java
##########
@@ -29,7 +29,7 @@
   /**
    * Create a new exception from the given status.
    */
-  FlightRuntimeException(CallStatus status) {
+  public FlightRuntimeException(CallStatus status) {

Review comment:
       This shouldn't need to be public, a CallStatus has a toRuntimeException method already




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

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



[GitHub] [arrow] keeratsingh commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
keeratsingh commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r506756240



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/CallOptions.java
##########
@@ -56,7 +56,7 @@ public static CallOption timeout(long duration, TimeUnit unit) {
   /**
    * CallOptions specific to GRPC stubs.
    */
-  interface GrpcCallOption extends CallOption {
+  public interface GrpcCallOption extends CallOption {

Review comment:
       @lidavidm Moving `GrpcCallOption` to `flight-grpc` adds a circular dependency since `flight-grpc` is dependent on `flight-core` for FlightProducer and FlightClient.
   
   Is the intention behind this comment to:
   - move all grpc related dependencies to `flight-grpc` module?
   - Or just move the `GrpcCallOption` to `org.apache.arrow.flight.grpc` package ?




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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r513761908



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/RequestContext.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.flight;
+
+import java.util.Set;
+
+/**
+ * Tracks variables about the current request.
+ */
+public interface RequestContext {
+  /**
+   * Register a variable and a value.
+   * @param key the variable name.
+   * @param value the value.
+   */
+  void put(String key, String value);
+
+  /**
+   * Retrieve a registered variable.
+   * @param key the variable name.
+   * @return the value, or empty string if not found.

Review comment:
       I think this should be null if not found?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/BasicCallHeaderAuthenticator.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.flight.auth2;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Optional;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A ServerAuthHandler for username/password authentication.
+ */
+public class BasicCallHeaderAuthenticator implements CallHeaderAuthenticator {
+
+  private static final Logger logger = LoggerFactory.getLogger(BasicCallHeaderAuthenticator.class);
+  private final BasicAuthValidator authValidator;
+
+  public BasicCallHeaderAuthenticator(BasicAuthValidator authValidator) {
+    super();
+    this.authValidator = authValidator;
+  }
+
+  @Override
+  public AuthResult authenticate(CallHeaders headers) {
+    final String authEncoded = AuthUtilities.getValueFromAuthHeader(headers, Auth2Constants.BASIC_PREFIX);
+    if (authEncoded == null) {
+      throw CallStatus.UNAUTHENTICATED.toRuntimeException();
+    }
+
+    try {
+      // The value has the format Base64(<username>:<password>)
+      final String authDecoded = new String(Base64.getDecoder().decode(authEncoded), StandardCharsets.UTF_8);
+      final int colonPos = authDecoded.indexOf(':');
+      if (colonPos == -1) {
+        throw CallStatus.UNAUTHORIZED.toRuntimeException();

Review comment:
       This is dealing with AuthN, so any use of Unauthorized here should be Unauthenticated

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/RequestContextAdapter.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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Set;
+
+import org.apache.arrow.flight.RequestContext;
+
+import io.grpc.Context;
+
+
+/**
+ * Adapter for holding key value pairs.
+ */
+public class RequestContextAdapter implements RequestContext {
+  private final HashMap<String, String> map = new HashMap<>();

Review comment:
       nit: put the instance variables below the static variables

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/BearerTokenAuthHandler.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.flight.auth2;
+
+import java.util.Optional;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallStatus;
+
+/**
+ * Partial implementation of ServerAuthHandler for bearer-token based authentication.
+ */
+abstract class BearerTokenAuthHandler implements CallHeaderAuthenticator {
+  @Override
+  public AuthResult authenticate(CallHeaders headers) {
+    final String bearerToken = AuthUtilities.getValueFromAuthHeader(headers, Auth2Constants.BEARER_PREFIX);
+    if (bearerToken == null) {
+      throw CallStatus.UNAUTHENTICATED.toRuntimeException();
+    }
+
+    if (!validateBearer(bearerToken)) {
+      throw CallStatus.UNAUTHORIZED.toRuntimeException();

Review comment:
       Ditto here, I think this should be Unauthenticated (we can't validate who you are)

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/CallHeaderAuthenticator.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flight.auth2;
+
+import java.util.Optional;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.FlightRuntimeException;
+
+/**
+ * Interface for Server side authentication handlers.

Review comment:
       While the details are in the proposal, I think it'd help to document the expected auth flow for any implementers here

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/CallHeaderAuthenticator.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.flight.auth2;
+
+import java.util.Optional;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.FlightRuntimeException;
+
+/**
+ * Interface for Server side authentication handlers.
+ */
+public interface CallHeaderAuthenticator {
+  /**
+   * The result of the server analyzing authentication headers.
+   */
+  interface AuthResult {
+    /**
+     * The peer identity that was determined by the handshake process based on the
+     * authentication credentials supplied by the client.
+     *
+     * @return The peer identity.
+     */
+    String getPeerIdentity();
+
+    /**
+     * The bearer token that was generated by the handshake process if applicable.
+     *
+     * @return bearer token, or Optional.empty() if bearer tokens are not supported by the auth mechanism.
+     */
+    default Optional<String> getBearerToken() {
+      return Optional.empty();
+    }
+  }
+
+  /**
+   * Validate the auth headers sent by the client.
+   *
+   * @param headers The headers to authenticate.
+   * @return a handshake result containing a peer identity and optionally a bearer token.
+   * @throws FlightRuntimeException with CallStatus.UNAUTHENTICATED if credentials were not supplied
+   *     or CallStatus.UNAUTHORIZED if credentials were supplied but were not valid.

Review comment:
       Ditto here - Unauthorized is about whether an authenticated peer has permissions to do something, not whether their credentials were valid in the first place (sorry, I'm being a bit pedantic here, but want to make sure the two are separate)




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r513695420



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.auth.AuthConstants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements CallContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();

Review comment:
       I've implemented the use of Map<String,String> as the RequestContext now. This is much better, great suggestion!




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r476900989



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/BasicAuthCallCredentials.java
##########
@@ -17,42 +17,39 @@
 
 package org.apache.arrow.flight.auth;
 
-import java.util.Iterator;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.concurrent.Executor;
 
-import org.apache.arrow.flight.impl.Flight.BasicAuth;
+import io.grpc.CallCredentials;
+import io.grpc.Metadata;
 
 /**
  * A client auth handler that supports username and password.
  */
-public class BasicClientAuthHandler implements ClientAuthHandler {
+public final class BasicAuthCallCredentials extends CallCredentials {

Review comment:
       > I started looking at the changes, but IMO, this can be done a different way.
   > 
   > If the goal here is to be able to use standard gRPC call credentials and the standard Authorization header, then there are only a few changes needed:
   > 
   > `flight-grpc` should re-export `GrpcCallOption`. This would let you define a custom call option that injects a gRPC CallCredentials.
   > https://github.com/apache/arrow/blob/maint-1.0.x/java/flight/flight-core/src/main/java/org/apache/arrow/flight/CallOptions.java#L59
   > 
   > FlightClient should have an option or a way to specify default call options, so you could add the auth call option on every call.
   > 
   > Then, on the server side, you can use `flight-grpc` to get the gRPC service instance and register it with your own gRPC server, and configure authentication there. Your custom auth interceptor could still put the identity of the user in AuthConstants.PEER_IDENTITY_KEY, so that you could still read it from the Flight CallContext.
   > 
   > This PR includes default implementations of various auth options. Those could be included as part of flight-grpc.
   > 
   > I admit I don't love the current auth implementation (specifically, Handshake causes issues because it's intentionally unauthenticated and we should use the standard Authorization header, and we can't handle more complex things like WWW-Authenticate and multiple authentication headers, X-Forwarded-* headers, etc.), but as is, this PR breaks compatibility and re-exports a lot of gRPC types from the Flight core.
   
   Hi David, just want to clarify comments about not re-exporting gRPC types. Currently flight-grpc has very little -- just FlightGrpcUtils. flight-core however has a package that has several gRPC dependencies such as MetadataAdapter, ClientInterceptorAdapter, etc.
   
   Is the intent to move these to flight-grpc? Should I move any new classes I add that depend on gRPC there?
   
   FlightServer itself depends on gRPC as well, and it is in flight-core and not in the gRPC package.
   Would a separate PR be a better place for cleaning up separation of flight-core from grpc?




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r513690262



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientHandshakeWrapper.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.flight.auth2;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.grpc.StatusUtils;
+import org.apache.arrow.flight.impl.Flight.HandshakeRequest;
+import org.apache.arrow.flight.impl.Flight.HandshakeResponse;
+import org.apache.arrow.flight.impl.FlightServiceGrpc.FlightServiceStub;
+
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * Utility class for executing a handshake with a FlightServer.
+ */
+public class ClientHandshakeWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClientHandshakeWrapper.class);
+
+  /**
+   * Do handshake for a client.  The stub will be authenticated after this method returns.
+   *
+   * @param stub The service stub.
+   */
+  public static void doClientHandshake(FlightServiceStub stub) {
+    final HandshakeObserver observer = new HandshakeObserver();
+    try {
+      observer.requestObserver = stub.handshake(observer);
+      observer.requestObserver.onNext(HandshakeRequest.newBuilder().build());
+      observer.requestObserver.onCompleted();
+      try {
+        if (!observer.completed.get()) {
+          // TODO: ARROW-5681
+          throw new RuntimeException("Unauthenticated");
+        }
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw ex;
+      } catch (ExecutionException ex) {
+        final FlightRuntimeException wrappedException = StatusUtils.fromThrowable(ex.getCause());
+        logger.error("Failed on completing future", wrappedException);

Review comment:
       Done




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r478695524



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ContextAdapter.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flight.grpc;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.auth.AuthConstants;
+
+import io.grpc.Context;
+
+/**
+ * Adapter class for gRPC contexts.
+ */
+public class ContextAdapter implements CallContext {
+  // gRPC uses reference equality when looking up keys in a Context. Cache used keys in this static map
+  // so that look ups can succeed.
+  private static final Map<String, Context.Key<String>> usedKeys = new HashMap<>();

Review comment:
       Yeah I wasn't really happy about this.
   
   I'm not sure what you mean by wrapping Context.Key here. The issue is that Context.key("foo") != Context.key("foo") when invoking key.getValue() since gRPC uses reference equality for context keys. We need to reference the _same_ Context.key when writing the value in the middleware and getting the value to set the peer identity before executing Flight calls. However neither side should be aware that they are using gRPC Contexts so we can't a shared static value representing the key as we did before these changes.
   
   Does ConcurrentHashMap allow for atomic writes of the same key if you use putIfAbsent?
   
   




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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #7994: Flight auth redesign

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r472995325



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java
##########
@@ -84,18 +81,15 @@
    * Create a Flight client from an allocator and a gRPC channel.
    */
   FlightClient(BufferAllocator incomingAllocator, ManagedChannel channel,
-      List<FlightClientMiddleware.Factory> middleware) {
+      List<FlightClientMiddleware.Factory> middleware, CallCredentials callCredentials) {
     this.allocator = incomingAllocator.newChildAllocator("flight-client", 0, Long.MAX_VALUE);
     this.channel = channel;
 
-    final ClientInterceptor[] interceptors;
-    interceptors = new ClientInterceptor[]{authInterceptor, new ClientInterceptorAdapter(middleware)};
-
     // Create a channel with interceptors pre-applied for DoGet and DoPut
-    this.interceptedChannel = ClientInterceptors.intercept(channel, interceptors);
+    this.interceptedChannel = ClientInterceptors.intercept(channel, new ClientInterceptorAdapter(middleware));
 
-    blockingStub = FlightServiceGrpc.newBlockingStub(interceptedChannel);
-    asyncStub = FlightServiceGrpc.newStub(interceptedChannel);
+    blockingStub = FlightServiceGrpc.newBlockingStub(interceptedChannel).withCallCredentials(callCredentials);

Review comment:
       IMO, for customizing the underlying gRPC stub, I'd rather see this added to `flight-grpc`. For instance, `GrpcCallOption` here could be re-exported from the `flight-grpc` module:
   
   https://github.com/apache/arrow/blob/maint-1.0.x/java/flight/flight-core/src/main/java/org/apache/arrow/flight/CallOptions.java#L59
   
   Then this could be implemented just as a custom call option (+ perhaps a FlightClient change to specify default call options).

##########
File path: format/Flight.proto
##########
@@ -124,11 +124,6 @@ message HandshakeRequest {
    * A defined protocol version
    */
   uint64 protocol_version = 1;
-

Review comment:
       These are format changes and will need a vote.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightClient.java
##########
@@ -156,23 +150,12 @@
   }
 
   /**
-   * Authenticates with a username and password.

Review comment:
       I don't see any need to remove existing auth methods.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightServerMiddleware.java
##########
@@ -72,6 +74,10 @@
     }
   }
 
+  default Context onAuthenticationSuccess(Context currentContext) {

Review comment:
       We shouldn't re-export gRPC types in this module. Otherwise, this would just have been `io.grpc.ServerInterceptor`.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/BasicAuthCallCredentials.java
##########
@@ -17,42 +17,39 @@
 
 package org.apache.arrow.flight.auth;
 
-import java.util.Iterator;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.concurrent.Executor;
 
-import org.apache.arrow.flight.impl.Flight.BasicAuth;
+import io.grpc.CallCredentials;
+import io.grpc.Metadata;
 
 /**
  * A client auth handler that supports username and password.
  */
-public class BasicClientAuthHandler implements ClientAuthHandler {
+public final class BasicAuthCallCredentials extends CallCredentials {

Review comment:
       Again, let's not re-export gRPC types.




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

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



[GitHub] [arrow] keeratsingh commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
keeratsingh commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r482255128



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/BasicServerAuthHandler.java
##########
@@ -39,37 +41,62 @@ public BasicServerAuthHandler(BasicAuthValidator authValidator) {
     this.authValidator = authValidator;
   }
 
-  /**
-   * Interface that this handler delegates for determining if credentials are valid.
-   */
-  public interface BasicAuthValidator {
+  @Override
+  public AuthResult authenticate(CallHeaders headers) {
+    final String authEncoded = AuthUtilities.getValueFromAuthHeader(headers, AuthConstants.BASIC_PREFIX);
+    if (authEncoded == null) {
+      throw new FlightRuntimeException(CallStatus.UNAUTHENTICATED);
+    }
 
-    byte[] getToken(String username, String password) throws Exception;
+    try {
+      // The value has the format Base64(<username>:<password>)
+      final String authDecoded = new String(Base64.getDecoder().decode(authEncoded), StandardCharsets.UTF_8);
+      final int colonPos = authDecoded.indexOf(':');
+      if (colonPos == -1) {
+        throw new FlightRuntimeException(CallStatus.UNAUTHORIZED);
+      }
 
-    Optional<String> isValid(byte[] token);
+      final String user = authDecoded.substring(0, colonPos);
+      final String password = authDecoded.substring(colonPos + 1);
+      final Optional<String> bearerToken = authValidator.validateCredentials(user, password);
+      return new AuthResult() {
+        @Override
+        public String getPeerIdentity() {
+          return user;
+        }
 
-  }
+        @Override
+        public Optional<String> getBearerToken() {
+          return bearerToken;
+        }
+      };
 
-  @Override
-  public boolean authenticate(ServerAuthSender outgoing, Iterator<byte[]> incoming) {
-    byte[] bytes = incoming.next();
-    try {
-      BasicAuth auth = BasicAuth.parseFrom(bytes);
-      byte[] token = authValidator.getToken(auth.getUsername(), auth.getPassword());
-      outgoing.send(token);
-      return true;
-    } catch (InvalidProtocolBufferException e) {
-      logger.debug("Failure parsing auth message.", e);
-    } catch (Exception e) {
-      logger.debug("Unknown error during authorization.", e);
+    } catch (UnsupportedEncodingException ex) {
+      throw new FlightRuntimeException(CallStatus.INTERNAL.withCause(ex));
+    } catch (FlightRuntimeException ex) {
+      throw ex;
+    } catch (Exception ex) {
+      throw new FlightRuntimeException(CallStatus.UNAUTHORIZED.withCause(ex));
     }
+  }
 
+  @Override
+  public boolean validateBearer(String bearerToken) {
     return false;
   }
 
   @Override
-  public Optional<String> isValid(byte[] token) {
-    return authValidator.isValid(token);
+  public boolean enableCachedCredentials() {
+    return true;
   }
 
+  /**
+   * Interface that this handler delegates to forS determining if credentials are valid.

Review comment:
       Nit: should be for?

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ServerAuthMiddleware.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.flight.auth;
+
+import org.apache.arrow.flight.CallContext;
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightServerMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Middleware that's used to validate credentials during the handshake and verify
+ * the bearer token in subsequent requests.
+ */
+public class ServerAuthMiddleware implements FlightServerMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ServerAuthMiddleware.class);
+
+  /**
+   * Factory for accessing ServerAuthMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerAuthMiddleware> {
+    private final ServerAuthHandler authHandler;
+    private final GeneratedBearerTokenAuthHandler bearerTokenAuthHandler;
+
+    /**
+     * Construct a factory with the given auth handler.
+     * @param authHandler The auth handler what will be used for authenticating requests.
+     */
+    public Factory(ServerAuthHandler authHandler) {
+      this.authHandler = authHandler;
+      bearerTokenAuthHandler = authHandler.enableCachedCredentials() ?
+          new GeneratedBearerTokenAuthHandler() : null;
+    }
+
+    @Override
+    public ServerAuthMiddleware onCallStarted(CallInfo callInfo, CallHeaders incomingHeaders, CallContext context) {
+      logger.debug("Call name: {}", callInfo.method().name());
+      // Check if bearer token auth is being used, and if we've enabled use of server-generated
+      // bearer tokens.
+      if (authHandler.enableCachedCredentials()) {
+        final String bearerTokenFromHeaders =
+            AuthUtilities.getValueFromAuthHeader(incomingHeaders, AuthConstants.BEARER_PREFIX);
+        if (bearerTokenFromHeaders != null) {
+          final ServerAuthHandler.AuthResult result = bearerTokenAuthHandler.authenticate(incomingHeaders);

Review comment:
       @jduo Should the authenticate method take a token as an argument instead of the incoming headers, as we execute the method `getValueFromAuthHeader` again in the authenticate method to get the token from the incoming headers?




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r476883835



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/ServerInterceptorAdapter.java
##########
@@ -87,10 +87,12 @@ public ServerInterceptorAdapter(List<KeyFactory<?>> factories) {
     // Use LinkedHashMap to preserve insertion order
     final Map<FlightServerMiddleware.Key<?>, FlightServerMiddleware> middlewareMap = new LinkedHashMap<>();
     final MetadataAdapter headerAdapter = new MetadataAdapter(headers);
+    Context currentContext = Context.current();
     for (final KeyFactory<?> factory : factories) {
       final FlightServerMiddleware m;
       try {
         m = factory.factory.onCallStarted(info, headerAdapter);
+        currentContext = m.onAuthenticationSuccess(currentContext);

Review comment:
       The issue here is there's no way to alter the Context from onCallStarted. We can get the context but can't get the ServerInterceptorAdapter to use the context.




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r477766787



##########
File path: java/flight/flight-core/src/test/java/org/apache/arrow/flight/auth/TestBasicAuth.java
##########
@@ -44,29 +44,38 @@
 import org.junit.Ignore;
 import org.junit.Test;
 
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 
 public class TestBasicAuth {
 
   private static final String USERNAME = "flight";
   private static final String PASSWORD = "woohoo";
-  private static final byte[] VALID_TOKEN = "my_token".getBytes(StandardCharsets.UTF_8);
+  private static final String VALID_TOKEN = "my_token";
 
+  private FlightClient.Builder clientBuilder;
   private FlightClient client;
   private FlightServer server;
   private BufferAllocator allocator;
 
   @Test
   public void validAuth() {
-    client.authenticateBasic(USERNAME, PASSWORD);
-    Assert.assertTrue(ImmutableList.copyOf(client.listFlights(Criteria.ALL)).size() == 0);
+    try {
+      client = clientBuilder.callCredentials(new BasicAuthCallCredentials(USERNAME, PASSWORD)).build();
+      client.handshake();
+      Assert.assertTrue(ImmutableList.copyOf(client.listFlights(Criteria.ALL)).size() == 0);

Review comment:
       Changed to assertTrue(isEmpty()). But let's focus on the design given this is a POC.




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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

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


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


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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r473125858



##########
File path: format/Flight.proto
##########
@@ -124,11 +124,6 @@ message HandshakeRequest {
    * A defined protocol version
    */
   uint64 protocol_version = 1;
-

Review comment:
       Thanks for the suggestions @lidavidm .




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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #7994: Flight auth redesign

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


   <!--
     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!
   
   Could you open an issue for this pull request on JIRA?
   https://issues.apache.org/jira/browse/ARROW
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${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.

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r476884889



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth/ServerAuthMiddleware.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flight.auth;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightRuntimeException;
+import org.apache.arrow.flight.FlightServerMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.grpc.Context;
+
+/**
+ * Middleware that's used to validate credentials during the handshake and verify
+ * the bearer token in subsequent requests.
+ */
+public class ServerAuthMiddleware implements FlightServerMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ServerAuthMiddleware.class);
+
+  /**
+   * Factory for accessing ServerAuthMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerAuthMiddleware> {
+    private final ServerAuthHandler authHandler;
+    private final GeneratedBearerTokenAuthHandler bearerTokenAuthHandler = new GeneratedBearerTokenAuthHandler();
+    private final Map<String, String> tokenToIdentityMap = new ConcurrentHashMap<>();
+
+    public Factory(ServerAuthHandler authHandler) {
+      this.authHandler = authHandler;
+    }
+
+    public String getPeerForBearer(String bearerToken) {
+      return bearerTokenAuthHandler.getIdentityForBearerToken(bearerToken);
+    }
+
+    @Override
+    public ServerAuthMiddleware onCallStarted(CallInfo callInfo, CallHeaders incomingHeaders) {
+      logger.debug("Call name: {}", callInfo.method().name());
+      if (callInfo.method().name().equals(AuthConstants.HANDSHAKE_DESCRIPTOR_NAME)) {
+        final ServerAuthHandler.HandshakeResult result = authHandler.authenticate(incomingHeaders);
+        final String bearerToken = bearerTokenAuthHandler.registerBearer(result);
+        return new ServerAuthMiddleware(result.getPeerIdentity(), bearerToken);
+      }
+
+      final String bearerToken = AuthUtilities.getValueFromAuthHeader(incomingHeaders, AuthConstants.BEARER_PREFIX);
+      // No bearer token provided. Auth handler may explicitly allow this.
+      if (bearerToken == null) {
+        if (authHandler.validateBearer(null)) {
+          return new ServerAuthMiddleware("", null);
+        }
+        logger.info("Client did not supply a bearer token.");
+        throw new FlightRuntimeException(CallStatus.UNAUTHENTICATED);
+      }
+
+      if (!authHandler.validateBearer(bearerToken) && !bearerTokenAuthHandler.validateBearer(bearerToken)) {
+        logger.info("Bearer token supplied by client was not authorized.");
+        throw new FlightRuntimeException(CallStatus.UNAUTHORIZED);
+      }
+
+      final String peerIdentity = bearerTokenAuthHandler.getIdentityForBearerToken(bearerToken);
+      return new ServerAuthMiddleware(peerIdentity, null);

Review comment:
       Let's get agreement on the design before focusing too much on optimizations.




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

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



[GitHub] [arrow] jduo commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
jduo commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r513690796



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/auth2/ClientBearerTokenMiddleware.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flight.auth2;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightClientMiddleware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Middleware for capturing bearer tokens sent back from the Flight server.
+ */
+public class ClientBearerTokenMiddleware implements FlightClientMiddleware {
+  private static final Logger logger = LoggerFactory.getLogger(ClientBearerTokenMiddleware.class);
+
+  private final Factory factory;
+
+  /**
+   * Factory used within FlightClient.
+   */
+  public static class Factory implements FlightClientMiddleware.Factory {
+    private final AtomicReference<String> bearerToken = new AtomicReference<>();
+
+    @Override
+    public FlightClientMiddleware onCallStarted(CallInfo info) {
+      logger.debug("Call name: {}", info.method().name());

Review comment:
       Done




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

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



[GitHub] [arrow] keeratsingh commented on a change in pull request #7994: ARROW-9804: [FlightRPC] Flight auth redesign

Posted by GitBox <gi...@apache.org>.
keeratsingh commented on a change in pull request #7994:
URL: https://github.com/apache/arrow/pull/7994#discussion_r507971603



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightRuntimeException.java
##########
@@ -29,7 +29,7 @@
   /**
    * Create a new exception from the given status.
    */
-  FlightRuntimeException(CallStatus status) {
+  public FlightRuntimeException(CallStatus status) {

Review comment:
       Addressed in commit:
   https://github.com/apache/arrow/pull/7994/commits/f60fc4a27cfadedc4583548debde52369c725c4a




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

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