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/11/20 17:31:37 UTC

[GitHub] [arrow] lyndonb-bq opened a new pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

lyndonb-bq opened a new pull request #8724:
URL: https://github.com/apache/arrow/pull/8724


   Current code is a work in progress - looking to get feedback on design and how it fits into current arrow C++ architecture before finalizing comments and fixing linting issues.
   
   - Added support for base 64 encoded username / password auth to match Java implementation
   - Added bearer token receiving and populating of call options to send back
   - Added integration tests that connects C++ client and Java server


----------------------------------------------------------------
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] lyndonb-bq commented on pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#issuecomment-731331899


   @lidavidm 
   Hi David,
   This is currently a work in progress. I was wondering if you could take a look at how this was integrated into the C++ library and let me know if it is okay or if there is a better way to do it? Afterwards I will make any required code changes and fix the code linting/commenting.
   Thanks!


----------------------------------------------------------------
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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);
+    if (bearer_iter == incoming_headers.end()) {
+      return;
+    }
+
+    // Check if the value of the auth token starts with the bearer prefix, latch the token.
+    std::string bearer_val = bearer_iter->second.to_string();
+    if (bearer_val.size() > BEARER_PREFIX.size()) {
+      bool hasPrefix = std::equal(bearer_val.begin(), bearer_val.begin() + BEARER_PREFIX.size(), BEARER_PREFIX.begin(),
+        [] (const char& char1, const char& char2) {
+          return (std::toupper(char1) == std::toupper(char2));
+        }
+      );
+      if (hasPrefix) {
+        *bearer_token = std::make_pair(AUTH_HEADER, bearer_val);
+      }
+    }
+  }
+
+  void ClientBearerTokenMiddleware::CallCompleted(const Status& status) { }
+
+  void ClientBearerTokenFactory::StartCall(const CallInfo& info, std::unique_ptr<ClientMiddleware>* middleware) {
+    *middleware = std::unique_ptr<ClientBearerTokenMiddleware>(new ClientBearerTokenMiddleware(bearer_token));

Review comment:
       std::make_unique?




----------------------------------------------------------------
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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);
+    if (bearer_iter == incoming_headers.end()) {
+      return;
+    }
+
+    // Check if the value of the auth token starts with the bearer prefix, latch the token.
+    std::string bearer_val = bearer_iter->second.to_string();
+    if (bearer_val.size() > BEARER_PREFIX.size()) {
+      bool hasPrefix = std::equal(bearer_val.begin(), bearer_val.begin() + BEARER_PREFIX.size(), BEARER_PREFIX.begin(),
+        [] (const char& char1, const char& char2) {
+          return (std::toupper(char1) == std::toupper(char2));
+        }
+      );
+      if (hasPrefix) {
+        *bearer_token = std::make_pair(AUTH_HEADER, bearer_val);
+      }
+    }
+  }
+
+  void ClientBearerTokenMiddleware::CallCompleted(const Status& status) { }
+
+  void ClientBearerTokenFactory::StartCall(const CallInfo& info, std::unique_ptr<ClientMiddleware>* middleware) {
+    *middleware = std::unique_ptr<ClientBearerTokenMiddleware>(new ClientBearerTokenMiddleware(bearer_token));
+  }
+
+  void ClientBearerTokenFactory::Reset() {
+    *bearer_token = std::make_pair("", "");
+  }
+
+  template<typename ... Args>
+  std::string string_format(const std::string& format, const Args... args) {
+    // Check size requirement for new string and increment by 1 for null terminator.
+    size_t size = std::snprintf(nullptr, 0, format.c_str(), args ...) + 1;
+    if(size <= 0){
+      throw std::runtime_error("Error during string formatting. Format: '" + format + "'.");
+    }
+
+    // Create buffer for new string and write string in.
+    std::unique_ptr<char[]> buf(new char[size]);
+    std::snprintf(buf.get(), size, format.c_str(), args...);
+
+    // Convert to std::string, subtracting size by 1 to trim null terminator.
+    return std::string(buf.get(), buf.get() + size - 1);
+  }
+
+  void AddBasicAuthHeaders(grpc::ClientContext* context, const std::string& username, const std::string& password) {
+    const std::string formatted_credentials = string_format("%s:%s", username.c_str(), password.c_str());
+    context->AddMetadata(AUTH_HEADER, BASIC_PREFIX + base64_encode(formatted_credentials));
+  }
+
+  std::string base64_encode(const std::string& input) {
+     static const std::string base64_chars =

Review comment:
       This doesn't exist in the codebase 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] lidavidm commented on pull request #8724: ARROW-10487 [FlightRPC][C++] Header-based auth in clients

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


   Thanks @lyndonb-bq!


----------------------------------------------------------------
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] lyndonb-bq commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on a change in pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#discussion_r527979053



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -963,8 +967,9 @@ class FlightClient::FlightClientImpl {
 
     std::vector<std::unique_ptr<grpc::experimental::ClientInterceptorFactoryInterface>>
         interceptors;
+    middleware = std::move(options.middleware);

Review comment:
       Goal was so that middleware was retained in this class so I could push and pop it. I am going to adjust the implementation though.

##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -993,6 +998,30 @@ class FlightClient::FlightClientImpl {
     return Status::OK();
   }
 
+  Status AuthenticateBasicToken(std::string username, std::string password,

Review comment:
       yes




----------------------------------------------------------------
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 #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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


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


----------------------------------------------------------------
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 #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -1175,6 +1222,7 @@ class FlightClient::FlightClientImpl {
       noop_auth_check_;
 #endif
   int64_t write_size_limit_bytes_;
+  GrpcClientInterceptorAdapterFactory* interceptor_pointer;

Review comment:
       Please follow naming conventions (`interceptor_pointer_`)

##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -993,6 +1010,36 @@ class FlightClient::FlightClientImpl {
     return Status::OK();
   }
 
+  Status AuthenticateBasicToken(const std::string& username, const std::string& password,
+                                std::pair<std::string, std::string>* bearer_token) {
+    // Add bearer token factory to middleware so it can intercept the bearer token.
+    if (interceptor_pointer != NULLPTR) {

Review comment:
       Honestly, instead of using interceptors, why not use ClientRpc.context (a [grpc::ClientContext](https://grpc.github.io/grpc/cpp/classgrpc_1_1_client_context.html)) to attach the headers (AddMetadata) and retrieve them (GetServerInitialMetadata)? This isn't a general auth mechanism anyways, so I'd rather we keep the implementation simple & tightly scoped in this case.

##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -993,6 +1010,36 @@ class FlightClient::FlightClientImpl {
     return Status::OK();
   }
 
+  Status AuthenticateBasicToken(const std::string& username, const std::string& password,
+                                std::pair<std::string, std::string>* bearer_token) {
+    // Add bearer token factory to middleware so it can intercept the bearer token.
+    if (interceptor_pointer != NULLPTR) {
+      interceptor_pointer->AddMiddlewareFactory(
+          std::make_shared<internal::ClientBearerTokenFactory>(bearer_token));
+    } else {
+      return MakeFlightError(FlightStatusCode::Internal,
+                             "Connect must be called before AuthenticateBasicToken.");
+    }
+    ClientRpc rpc({});

Review comment:
       You may actually want to allow passing call options so things like timeouts can be set.

##########
File path: cpp/src/arrow/flight/client.h
##########
@@ -191,6 +194,14 @@ class ARROW_FLIGHT_EXPORT FlightClient {
   Status Authenticate(const FlightCallOptions& options,
                       std::unique_ptr<ClientAuthHandler> auth_handler);
 
+  /// \brief Authenticate to the server using basic authentication with base 64 encoding.

Review comment:
       nit: base64 encoding is part of the definition of HTTP basic auth




----------------------------------------------------------------
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 #8724: ARROW-10487 [FlightRPC][C++] Header-based auth in clients

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


   


----------------------------------------------------------------
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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);
+    if (bearer_iter == incoming_headers.end()) {
+      return;
+    }
+
+    // Check if the value of the auth token starts with the bearer prefix, latch the token.
+    std::string bearer_val = bearer_iter->second.to_string();
+    if (bearer_val.size() > BEARER_PREFIX.size()) {
+      bool hasPrefix = std::equal(bearer_val.begin(), bearer_val.begin() + BEARER_PREFIX.size(), BEARER_PREFIX.begin(),

Review comment:
       Seems like they use snake_case instead of camelCase.




----------------------------------------------------------------
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] lyndonb-bq commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on a change in pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#discussion_r527968403



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -51,6 +51,7 @@
 
 #include "arrow/flight/client_auth.h"
 #include "arrow/flight/client_middleware.h"
+#include "arrow/flight/client_header_auth_middleware.h"

Review comment:
       Yes, was waiting for feedback on design before proper formatting. I will correct 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] lidavidm commented on pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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


   Note, for formatting, use clang-format; for naming conventions, please follow Google C++ guidelines. See the [developer docs](https://arrow.apache.org/docs/developers/cpp/development.html#code-style-linting-and-ci).


----------------------------------------------------------------
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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -328,7 +332,7 @@ class GrpcClientInterceptorAdapterFactory
     : public grpc::experimental::ClientInterceptorFactoryInterface {
  public:
   GrpcClientInterceptorAdapterFactory(
-      std::vector<std::shared_ptr<ClientMiddlewareFactory>> middleware)
+      std::vector<std::shared_ptr<ClientMiddlewareFactory>>& middleware)

Review comment:
       Should this be const?




----------------------------------------------------------------
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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);
+    if (bearer_iter == incoming_headers.end()) {
+      return;
+    }
+
+    // Check if the value of the auth token starts with the bearer prefix, latch the token.
+    std::string bearer_val = bearer_iter->second.to_string();
+    if (bearer_val.size() > BEARER_PREFIX.size()) {
+      bool hasPrefix = std::equal(bearer_val.begin(), bearer_val.begin() + BEARER_PREFIX.size(), BEARER_PREFIX.begin(),
+        [] (const char& char1, const char& char2) {
+          return (std::toupper(char1) == std::toupper(char2));
+        }
+      );
+      if (hasPrefix) {
+        *bearer_token = std::make_pair(AUTH_HEADER, bearer_val);
+      }
+    }
+  }
+
+  void ClientBearerTokenMiddleware::CallCompleted(const Status& status) { }
+
+  void ClientBearerTokenFactory::StartCall(const CallInfo& info, std::unique_ptr<ClientMiddleware>* middleware) {

Review comment:
       Would it be better to pass a reference instead of a pointer?




----------------------------------------------------------------
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] lyndonb-bq commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on a change in pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#discussion_r530544565



##########
File path: cpp/src/arrow/flight/flight_test.cc
##########
@@ -1010,6 +1140,57 @@ class TestErrorMiddleware : public ::testing::Test {
   std::unique_ptr<FlightServerBase> server_;
 };
 
+class TestBasicHeaderAuthMiddleware : public ::testing::Test {
+ public:
+  void SetUp() {
+    header_middleware_ = std::make_shared<HeaderAuthServerMiddlewareFactory>();
+    bearer_middleware_ = std::make_shared<BearerAuthServerMiddlewareFactory>();
+    std::pair<std::string, std::string> bearer = make_pair(
+        kAuthHeader, std::string(kBearerPrefix) + " " + std::string(kBearerToken));
+    ASSERT_OK(MakeServer<HeaderAuthTestServer>(
+        &server_, &client_,
+        [&](FlightServerOptions* options) {
+          options->auth_handler =
+              std::unique_ptr<ServerAuthHandler>(new NoOpAuthHandler());
+          options->middleware.push_back({"header-auth-server", header_middleware_});
+          options->middleware.push_back({"bearer-auth-server", bearer_middleware_});
+          return Status::OK();
+        },
+        [&](FlightClientOptions* options) { return Status::OK(); }));
+  }
+
+  void RunValidClientAuth() {
+    std::pair<std::string, std::string> bearer_token;
+    ASSERT_OK(client_->AuthenticateBasicToken({}, kValidUsername, kValidPassword,
+                                              &bearer_token));
+    ASSERT_EQ(bearer_token.first, kAuthHeader);
+    ASSERT_EQ(bearer_token.second, (std::string(kBearerPrefix) + kBearerToken));
+    std::unique_ptr<FlightListing> listing;
+    FlightCallOptions call_options;
+    call_options.headers.push_back(bearer_token);
+    ASSERT_OK(client_->ListFlights(call_options, {}, &listing));
+    ASSERT_TRUE(bearer_middleware_->GetIsValid());
+  }
+
+  void RunInvalidClientAuth() {
+    std::pair<std::string, std::string> bearer_token;
+    // Note: Status intentionally ignored because it requires C++ server implementation of
+    // header auth. For now it returns an IOError.
+    arrow::Status status = client_->AuthenticateBasicToken(

Review comment:
       Sorry, this was an artifact of my initial test implementation which was incorrect on the test server, and always caused an error. I forgot to remove it from the negative test once I fixed it. I will correct this.

##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -1198,6 +1227,12 @@ Status FlightClient::Authenticate(const FlightCallOptions& options,
   return impl_->Authenticate(options, std::move(auth_handler));
 }
 
+Status FlightClient::AuthenticateBasicToken(

Review comment:
       Yep, I will make this change.




----------------------------------------------------------------
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] lyndonb-bq commented on pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#issuecomment-733923181


   @lidavidm Hey David, I have made the requested changes. I got a random Javascript failure in one of the CI teardowns, not sure if this is accepted or if you can restart the CI build (I do not have permission to restart it myself).


----------------------------------------------------------------
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 #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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


   By the way, for CMake formatting, please use [cmake_format](https://arrow.apache.org/docs/developers/cpp/development.html#code-style-linting-and-ci)


----------------------------------------------------------------
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] lyndonb-bq commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on a change in pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#discussion_r527978639



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -371,7 +375,7 @@ class GrpcClientInterceptorAdapterFactory
   }
 
  private:
-  std::vector<std::shared_ptr<ClientMiddlewareFactory>> middleware_;
+  std::vector<std::shared_ptr<ClientMiddlewareFactory>>& middleware_;

Review comment:
       It was to keep the reference, but you got me thinking more and I think I should just take a copy and then make a function to allow adding to the middleware on the fly, since the issue is that I need to be able to add and remove middleware from the interceptor. I will rejig 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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -51,6 +51,7 @@
 
 #include "arrow/flight/client_auth.h"
 #include "arrow/flight/client_middleware.h"
+#include "arrow/flight/client_header_auth_middleware.h"

Review comment:
       Is this ordering alphabetical? If so this should be up one...




----------------------------------------------------------------
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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -963,8 +967,9 @@ class FlightClient::FlightClientImpl {
 
     std::vector<std::unique_ptr<grpc::experimental::ClientInterceptorFactoryInterface>>
         interceptors;
+    middleware = std::move(options.middleware);

Review comment:
       Why this change?




----------------------------------------------------------------
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 #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client.h
##########
@@ -191,6 +194,14 @@ class ARROW_FLIGHT_EXPORT FlightClient {
   Status Authenticate(const FlightCallOptions& options,
                       std::unique_ptr<ClientAuthHandler> auth_handler);
 
+  /// \brief Authenticate to the server using the given handler.

Review comment:
       There's no handler in play here.

##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);
+    if (bearer_iter == incoming_headers.end()) {
+      return;
+    }
+
+    // Check if the value of the auth token starts with the bearer prefix, latch the token.
+    std::string bearer_val = bearer_iter->second.to_string();
+    if (bearer_val.size() > BEARER_PREFIX.size()) {
+      bool hasPrefix = std::equal(bearer_val.begin(), bearer_val.begin() + BEARER_PREFIX.size(), BEARER_PREFIX.begin(),
+        [] (const char& char1, const char& char2) {
+          return (std::toupper(char1) == std::toupper(char2));
+        }
+      );
+      if (hasPrefix) {
+        *bearer_token = std::make_pair(AUTH_HEADER, bearer_val);
+      }
+    }
+  }
+
+  void ClientBearerTokenMiddleware::CallCompleted(const Status& status) { }
+
+  void ClientBearerTokenFactory::StartCall(const CallInfo& info, std::unique_ptr<ClientMiddleware>* middleware) {
+    *middleware = std::unique_ptr<ClientBearerTokenMiddleware>(new ClientBearerTokenMiddleware(bearer_token));
+  }
+
+  void ClientBearerTokenFactory::Reset() {
+    *bearer_token = std::make_pair("", "");
+  }
+
+  template<typename ... Args>
+  std::string string_format(const std::string& format, const Args... args) {
+    // Check size requirement for new string and increment by 1 for null terminator.
+    size_t size = std::snprintf(nullptr, 0, format.c_str(), args ...) + 1;
+    if(size <= 0){
+      throw std::runtime_error("Error during string formatting. Format: '" + format + "'.");

Review comment:
       And Arrow disallows exceptions.

##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -328,7 +332,7 @@ class GrpcClientInterceptorAdapterFactory
     : public grpc::experimental::ClientInterceptorFactoryInterface {
  public:
   GrpcClientInterceptorAdapterFactory(
-      std::vector<std::shared_ptr<ClientMiddlewareFactory>> middleware)
+      std::vector<std::shared_ptr<ClientMiddlewareFactory>>& middleware)

Review comment:
       So, I'd rather we store a reference to this interceptor factory instead of a reference to the middleware with a mutable pointer back, i.e. I'd rather have this class own the middleware as it currently does, and have FlightClient call a method of this class to add more middleware at runtime.

##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.h
##########
@@ -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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#pragma once
+
+#include "arrow/flight/client_middleware.h"
+#include "arrow/flight/client_auth.h"
+#include "arrow/flight/client.h"
+
+#ifdef GRPCPP_PP_INCLUDE
+#include <grpcpp/grpcpp.h>
+#if defined(GRPC_NAMESPACE_FOR_TLS_CREDENTIALS_OPTIONS)
+#include <grpcpp/security/tls_credentials_options.h>

Review comment:
       I don't think we need this include?

##########
File path: cpp/src/arrow/flight/client.h
##########
@@ -65,6 +65,9 @@ class ARROW_FLIGHT_EXPORT FlightCallOptions {
 
   /// \brief IPC writer options, if applicable for the call.
   ipc::IpcWriteOptions write_options;
+
+  /// \brief Metadata for client to add to context.
+  std::vector<std::pair<std::string, std::string>> metadata;

Review comment:
       I believe we call it headers elsewhere, so this should stay consistent with that.

##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.h
##########
@@ -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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#pragma once
+
+#include "arrow/flight/client_middleware.h"
+#include "arrow/flight/client_auth.h"
+#include "arrow/flight/client.h"
+
+#ifdef GRPCPP_PP_INCLUDE
+#include <grpcpp/grpcpp.h>
+#if defined(GRPC_NAMESPACE_FOR_TLS_CREDENTIALS_OPTIONS)
+#include <grpcpp/security/tls_credentials_options.h>
+#endif
+#else
+#include <grpc++/grpc++.h>
+#endif
+
+#include <algorithm>
+#include <iostream>
+#include <cctype>
+#include <string>
+
+const std::string AUTH_HEADER = "authorization";

Review comment:
       These should be named kAuthHeader etc. and should go in the .cc file unless we actually want to expose these to users.
   
   Or alternatively, the other constants are currently in one of the _internal.h headers.

##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.h
##########
@@ -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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#pragma once
+
+#include "arrow/flight/client_middleware.h"
+#include "arrow/flight/client_auth.h"
+#include "arrow/flight/client.h"
+
+#ifdef GRPCPP_PP_INCLUDE
+#include <grpcpp/grpcpp.h>
+#if defined(GRPC_NAMESPACE_FOR_TLS_CREDENTIALS_OPTIONS)
+#include <grpcpp/security/tls_credentials_options.h>
+#endif
+#else
+#include <grpc++/grpc++.h>
+#endif
+
+#include <algorithm>
+#include <iostream>
+#include <cctype>
+#include <string>
+
+const std::string AUTH_HEADER = "authorization";
+const std::string BEARER_PREFIX = "Bearer ";
+const std::string BASIC_PREFIX = "Basic ";
+
+namespace arrow {
+namespace flight {
+
+// TODO: Need to add documentation in this file.
+void ARROW_FLIGHT_EXPORT AddBasicAuthHeaders(grpc::ClientContext* context, 

Review comment:
       This is internal - it shouldn't be in a public header. (Ditto for the grpc include.)

##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -371,7 +375,7 @@ class GrpcClientInterceptorAdapterFactory
   }
 
  private:
-  std::vector<std::shared_ptr<ClientMiddlewareFactory>> middleware_;
+  std::vector<std::shared_ptr<ClientMiddlewareFactory>>& middleware_;

Review comment:
       This should be a pointer if not a copy.

##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.h
##########
@@ -0,0 +1,78 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       This should probably be an _internal.h header since I don't think any of this is intended to be directly used outside of the implementation here.

##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);
+    if (bearer_iter == incoming_headers.end()) {
+      return;
+    }
+
+    // Check if the value of the auth token starts with the bearer prefix, latch the token.
+    std::string bearer_val = bearer_iter->second.to_string();
+    if (bearer_val.size() > BEARER_PREFIX.size()) {
+      bool hasPrefix = std::equal(bearer_val.begin(), bearer_val.begin() + BEARER_PREFIX.size(), BEARER_PREFIX.begin(),
+        [] (const char& char1, const char& char2) {
+          return (std::toupper(char1) == std::toupper(char2));
+        }
+      );
+      if (hasPrefix) {
+        *bearer_token = std::make_pair(AUTH_HEADER, bearer_val);
+      }
+    }
+  }
+
+  void ClientBearerTokenMiddleware::CallCompleted(const Status& status) { }
+
+  void ClientBearerTokenFactory::StartCall(const CallInfo& info, std::unique_ptr<ClientMiddleware>* middleware) {
+    *middleware = std::unique_ptr<ClientBearerTokenMiddleware>(new ClientBearerTokenMiddleware(bearer_token));
+  }
+
+  void ClientBearerTokenFactory::Reset() {
+    *bearer_token = std::make_pair("", "");
+  }
+
+  template<typename ... Args>

Review comment:
       I don't think we need an entire templated function to concatenate two strings.

##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -328,7 +332,7 @@ class GrpcClientInterceptorAdapterFactory
     : public grpc::experimental::ClientInterceptorFactoryInterface {
  public:
   GrpcClientInterceptorAdapterFactory(
-      std::vector<std::shared_ptr<ClientMiddlewareFactory>> middleware)
+      std::vector<std::shared_ptr<ClientMiddlewareFactory>>& middleware)

Review comment:
       This should either be const reference or a pointer.




----------------------------------------------------------------
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] lyndonb-bq commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on a change in pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#discussion_r527982262



##########
File path: cpp/src/arrow/flight/client.h
##########
@@ -65,6 +65,9 @@ class ARROW_FLIGHT_EXPORT FlightCallOptions {
 
   /// \brief IPC writer options, if applicable for the call.
   ipc::IpcWriteOptions write_options;
+
+  /// \brief Metadata for client to add to context.
+  std::vector<std::pair<std::string, std::string>> metadata;

Review comment:
       Will correct 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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.h
##########
@@ -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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#pragma once
+
+#include "arrow/flight/client_middleware.h"
+#include "arrow/flight/client_auth.h"
+#include "arrow/flight/client.h"
+
+#ifdef GRPCPP_PP_INCLUDE
+#include <grpcpp/grpcpp.h>
+#if defined(GRPC_NAMESPACE_FOR_TLS_CREDENTIALS_OPTIONS)
+#include <grpcpp/security/tls_credentials_options.h>
+#endif
+#else
+#include <grpc++/grpc++.h>
+#endif
+
+#include <algorithm>
+#include <iostream>
+#include <cctype>
+#include <string>
+
+const std::string AUTH_HEADER = "authorization";

Review comment:
       Do these need to be in the header, or should they be only in the CC.
   If in the header, I think they're created anew for each compilation unit, and thus should be defined as extern with the actual value defined in the CC to avoid multiple instantiations.




----------------------------------------------------------------
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 edited a comment on pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lidavidm edited a comment on pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#issuecomment-731407237


   Note, for formatting, use clang-format; for naming conventions, please follow Google C++ guidelines. See the [developer docs](https://arrow.apache.org/docs/developers/cpp/development.html#code-style-linting-and-ci).
   
   In particular, use snake_case, use pointers instead of non-const references, and note that constants are named kCamelCase not UPPER_SNAKE_CASE.


----------------------------------------------------------------
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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);

Review comment:
       const?




----------------------------------------------------------------
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] lyndonb-bq commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on a change in pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#discussion_r527985763



##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);
+    if (bearer_iter == incoming_headers.end()) {
+      return;
+    }
+
+    // Check if the value of the auth token starts with the bearer prefix, latch the token.
+    std::string bearer_val = bearer_iter->second.to_string();
+    if (bearer_val.size() > BEARER_PREFIX.size()) {
+      bool hasPrefix = std::equal(bearer_val.begin(), bearer_val.begin() + BEARER_PREFIX.size(), BEARER_PREFIX.begin(),
+        [] (const char& char1, const char& char2) {
+          return (std::toupper(char1) == std::toupper(char2));
+        }
+      );
+      if (hasPrefix) {
+        *bearer_token = std::make_pair(AUTH_HEADER, bearer_val);
+      }
+    }
+  }
+
+  void ClientBearerTokenMiddleware::CallCompleted(const Status& status) { }
+
+  void ClientBearerTokenFactory::StartCall(const CallInfo& info, std::unique_ptr<ClientMiddleware>* middleware) {
+    *middleware = std::unique_ptr<ClientBearerTokenMiddleware>(new ClientBearerTokenMiddleware(bearer_token));
+  }
+
+  void ClientBearerTokenFactory::Reset() {
+    *bearer_token = std::make_pair("", "");
+  }
+
+  template<typename ... Args>
+  std::string string_format(const std::string& format, const Args... args) {
+    // Check size requirement for new string and increment by 1 for null terminator.
+    size_t size = std::snprintf(nullptr, 0, format.c_str(), args ...) + 1;
+    if(size <= 0){
+      throw std::runtime_error("Error during string formatting. Format: '" + format + "'.");
+    }
+
+    // Create buffer for new string and write string in.
+    std::unique_ptr<char[]> buf(new char[size]);
+    std::snprintf(buf.get(), size, format.c_str(), args...);
+
+    // Convert to std::string, subtracting size by 1 to trim null terminator.
+    return std::string(buf.get(), buf.get() + size - 1);
+  }
+
+  void AddBasicAuthHeaders(grpc::ClientContext* context, const std::string& username, const std::string& password) {
+    const std::string formatted_credentials = string_format("%s:%s", username.c_str(), password.c_str());
+    context->AddMetadata(AUTH_HEADER, BASIC_PREFIX + base64_encode(formatted_credentials));
+  }
+
+  std::string base64_encode(const std::string& input) {
+     static const std::string base64_chars =

Review comment:
       Yes, I found it. Will remove 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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -371,7 +375,7 @@ class GrpcClientInterceptorAdapterFactory
   }
 
  private:
-  std::vector<std::shared_ptr<ClientMiddlewareFactory>> middleware_;
+  std::vector<std::shared_ptr<ClientMiddlewareFactory>>& middleware_;

Review comment:
       Intention is to keep the reference and not a copy?




----------------------------------------------------------------
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] lyndonb-bq commented on pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#issuecomment-733939862


   > @lidavidm Hey David, I have made the requested changes. I got a random Javascript failure in one of the CI teardowns, not sure if this is accepted or if you can restart the CI build (I do not have permission to restart it myself).
   
   @lidavidm All tests are now passing 🎉 


----------------------------------------------------------------
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] lyndonb-bq commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on a change in pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#discussion_r527981869



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -993,6 +998,30 @@ class FlightClient::FlightClientImpl {
     return Status::OK();
   }
 
+  Status AuthenticateBasicToken(std::string username, std::string password,
+                                std::pair<std::string, std::string>* bearer_token) {
+    // Add bearer token factory to middleware so it can intercept the bearer token.
+    middleware.push_back(std::make_shared<ClientBearerTokenFactory>(bearer_token));

Review comment:
       The raw pointer is unpopulated, so it's passed to the BearerTokenFactory's constructor, which stores it and populated it when it receives the bearer token. I could make the client pass the whole factory in with the bearer token already inside it, but it's more work and requires they understand what's going on more than they otherwise would need to.




----------------------------------------------------------------
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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);
+    if (bearer_iter == incoming_headers.end()) {
+      return;
+    }
+
+    // Check if the value of the auth token starts with the bearer prefix, latch the token.
+    std::string bearer_val = bearer_iter->second.to_string();
+    if (bearer_val.size() > BEARER_PREFIX.size()) {
+      bool hasPrefix = std::equal(bearer_val.begin(), bearer_val.begin() + BEARER_PREFIX.size(), BEARER_PREFIX.begin(),
+        [] (const char& char1, const char& char2) {
+          return (std::toupper(char1) == std::toupper(char2));
+        }
+      );
+      if (hasPrefix) {
+        *bearer_token = std::make_pair(AUTH_HEADER, bearer_val);
+      }
+    }
+  }
+
+  void ClientBearerTokenMiddleware::CallCompleted(const Status& status) { }
+
+  void ClientBearerTokenFactory::StartCall(const CallInfo& info, std::unique_ptr<ClientMiddleware>* middleware) {
+    *middleware = std::unique_ptr<ClientBearerTokenMiddleware>(new ClientBearerTokenMiddleware(bearer_token));
+  }
+
+  void ClientBearerTokenFactory::Reset() {
+    *bearer_token = std::make_pair("", "");
+  }
+
+  template<typename ... Args>
+  std::string string_format(const std::string& format, const Args... args) {
+    // Check size requirement for new string and increment by 1 for null terminator.
+    size_t size = std::snprintf(nullptr, 0, format.c_str(), args ...) + 1;
+    if(size <= 0){

Review comment:
       Nit: spacing between if (, ){




----------------------------------------------------------------
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 #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -1198,6 +1227,12 @@ Status FlightClient::Authenticate(const FlightCallOptions& options,
   return impl_->Authenticate(options, std::move(auth_handler));
 }
 
+Status FlightClient::AuthenticateBasicToken(

Review comment:
       Could we make this `arrow::Result<std::pair<>>`?

##########
File path: cpp/src/arrow/flight/flight_test.cc
##########
@@ -1010,6 +1140,57 @@ class TestErrorMiddleware : public ::testing::Test {
   std::unique_ptr<FlightServerBase> server_;
 };
 
+class TestBasicHeaderAuthMiddleware : public ::testing::Test {
+ public:
+  void SetUp() {
+    header_middleware_ = std::make_shared<HeaderAuthServerMiddlewareFactory>();
+    bearer_middleware_ = std::make_shared<BearerAuthServerMiddlewareFactory>();
+    std::pair<std::string, std::string> bearer = make_pair(
+        kAuthHeader, std::string(kBearerPrefix) + " " + std::string(kBearerToken));
+    ASSERT_OK(MakeServer<HeaderAuthTestServer>(
+        &server_, &client_,
+        [&](FlightServerOptions* options) {
+          options->auth_handler =
+              std::unique_ptr<ServerAuthHandler>(new NoOpAuthHandler());
+          options->middleware.push_back({"header-auth-server", header_middleware_});
+          options->middleware.push_back({"bearer-auth-server", bearer_middleware_});
+          return Status::OK();
+        },
+        [&](FlightClientOptions* options) { return Status::OK(); }));
+  }
+
+  void RunValidClientAuth() {
+    std::pair<std::string, std::string> bearer_token;
+    ASSERT_OK(client_->AuthenticateBasicToken({}, kValidUsername, kValidPassword,
+                                              &bearer_token));
+    ASSERT_EQ(bearer_token.first, kAuthHeader);
+    ASSERT_EQ(bearer_token.second, (std::string(kBearerPrefix) + kBearerToken));
+    std::unique_ptr<FlightListing> listing;
+    FlightCallOptions call_options;
+    call_options.headers.push_back(bearer_token);
+    ASSERT_OK(client_->ListFlights(call_options, {}, &listing));
+    ASSERT_TRUE(bearer_middleware_->GetIsValid());
+  }
+
+  void RunInvalidClientAuth() {
+    std::pair<std::string, std::string> bearer_token;
+    // Note: Status intentionally ignored because it requires C++ server implementation of
+    // header auth. For now it returns an IOError.
+    arrow::Status status = client_->AuthenticateBasicToken(

Review comment:
       If the intent is to test that this fails, we can `ASSERT_RAISES(IOError, status)` - even if we had an implementation of header auth, we'd want this to fail, 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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -993,6 +998,30 @@ class FlightClient::FlightClientImpl {
     return Status::OK();
   }
 
+  Status AuthenticateBasicToken(std::string username, std::string password,
+                                std::pair<std::string, std::string>* bearer_token) {
+    // Add bearer token factory to middleware so it can intercept the bearer token.
+    middleware.push_back(std::make_shared<ClientBearerTokenFactory>(bearer_token));

Review comment:
       This looks odd to create the shared pointer after you've passed in the raw pointer....it seems like the method itself should take a shared pointer.




----------------------------------------------------------------
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] kylepbit commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

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



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -993,6 +998,30 @@ class FlightClient::FlightClientImpl {
     return Status::OK();
   }
 
+  Status AuthenticateBasicToken(std::string username, std::string password,

Review comment:
       Should username/password be passed by const ref?




----------------------------------------------------------------
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] lyndonb-bq commented on a change in pull request #8724: ARROW-10487 [FlightRPC][C++][Python] Header-based auth in clients

Posted by GitBox <gi...@apache.org>.
lyndonb-bq commented on a change in pull request #8724:
URL: https://github.com/apache/arrow/pull/8724#discussion_r528906088



##########
File path: cpp/src/arrow/flight/client_header_auth_middleware.cc
##########
@@ -0,0 +1,124 @@
+// 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.
+
+// Interfaces for defining middleware for Flight clients. Currently
+// experimental.
+
+#include "client_header_auth_middleware.h"
+#include "client_middleware.h"
+#include "client_auth.h"
+#include "client.h"
+
+namespace arrow {
+namespace flight {
+
+  std::string base64_encode(const std::string& input);
+
+  ClientBearerTokenMiddleware::ClientBearerTokenMiddleware(
+    std::pair<std::string, std::string>* bearer_token_)
+        : bearer_token(bearer_token_) { }
+
+  void ClientBearerTokenMiddleware::SendingHeaders(AddCallHeaders* outgoing_headers) { }
+
+  void ClientBearerTokenMiddleware::ReceivedHeaders(
+    const CallHeaders& incoming_headers) {
+    // Grab the auth token if one exists.
+    auto bearer_iter = incoming_headers.find(AUTH_HEADER);
+    if (bearer_iter == incoming_headers.end()) {
+      return;
+    }
+
+    // Check if the value of the auth token starts with the bearer prefix, latch the token.
+    std::string bearer_val = bearer_iter->second.to_string();
+    if (bearer_val.size() > BEARER_PREFIX.size()) {
+      bool hasPrefix = std::equal(bearer_val.begin(), bearer_val.begin() + BEARER_PREFIX.size(), BEARER_PREFIX.begin(),
+        [] (const char& char1, const char& char2) {
+          return (std::toupper(char1) == std::toupper(char2));
+        }
+      );
+      if (hasPrefix) {
+        *bearer_token = std::make_pair(AUTH_HEADER, bearer_val);
+      }
+    }
+  }
+
+  void ClientBearerTokenMiddleware::CallCompleted(const Status& status) { }
+
+  void ClientBearerTokenFactory::StartCall(const CallInfo& info, std::unique_ptr<ClientMiddleware>* middleware) {

Review comment:
       Can't this is a method of the base class, I think it's done this way to allow you to assign a new unique pointer to it without exposing the other middlewares they are already holding in their vector.




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