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 18:21:04 UTC

[GitHub] [arrow] lyndonb-bq opened a new pull request #8725: ARROW-10526: [FlightRPC][C++][Python] Client cookie middleware

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


   - Added client cookie middleware which caches and monitors expiry
   - Added testing for client cookie middleware


----------------------------------------------------------------
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 #8725: ARROW-10526: [FlightRPC][C++] Client cookie middleware

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


   


----------------------------------------------------------------
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 #8725: ARROW-10526: [FlightRPC][C++] Client cookie middleware

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


   @lidavidm That's a really good point, there is a lot of code that wasn't facing direct testing. I followed suggestion of moving the parsing logic to an internal header (the client_header_internal files seemed appropriate since this code is all related to headers). I added a bunch of tests for all the different internal parsing functions that were added.


----------------------------------------------------------------
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 #8725: ARROW-10526: [FlightRPC][C++][Python] Client cookie middleware

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


   @lidavidm 
   
   Hi David, when you have some time please take a look at the changes I have introduced here. As you can probably tell, this is adds a client side middleware for cookie handling in C++. I am still waiting for the CI to complete so I can make sure that it is good on all platforms, but the code structure should not change much from here unless changes are requested.
   
   Thank you!


----------------------------------------------------------------
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 #8725: ARROW-10526: [FlightRPC][C++] Client cookie middleware

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


   There appears to be a failure in the python build, I think it's caused by something unrelated to this pull request though, I noticed the same here: https://github.com/apache/arrow/runs/1456411775?check_suite_focus=true


----------------------------------------------------------------
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 #8725: ARROW-10526: [FlightRPC][C++] Client cookie middleware

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



##########
File path: cpp/src/arrow/flight/client_header_internal.h
##########
@@ -22,6 +22,8 @@
 
 #include "arrow/flight/client_middleware.h"
 #include "arrow/result.h"
+#include "arrow/util/optional.h"

Review comment:
       Oh and `#include <map>` but perhaps unordered_map is better since you don't care about ordering?

##########
File path: cpp/src/arrow/flight/client_header_internal.h
##########
@@ -22,6 +22,8 @@
 
 #include "arrow/flight/client_middleware.h"
 #include "arrow/result.h"
+#include "arrow/util/optional.h"

Review comment:
       nit: I don't think value_parsing is used in the header, and we're missing `#include <mutex>`, and `#include <functional>` for `binary_function`, `#include <string>`, and `#include <chrono>`

##########
File path: cpp/src/arrow/flight/flight_test.cc
##########
@@ -1186,6 +1188,139 @@ class TestBasicHeaderAuthMiddleware : public ::testing::Test {
   std::shared_ptr<BearerAuthServerMiddlewareFactory> bearer_middleware_;
 };
 
+// This test keeps an internal cookie cache and compares that with the middleware.
+class TestCookieMiddleware : public ::testing::Test {
+ public:
+  // Setup function creates middleware factory and starts it up.
+  void SetUp() {
+    factory_ = GetCookieFactory();
+    CallInfo callInfo;
+    factory_->StartCall(callInfo, &middleware_);
+  }
+
+  // Function to add incoming cookies to middleware and validate them.
+  void AddAndValidate(const std::string& incoming_cookie) {
+    // Add cookie
+    CallHeaders call_headers;
+    call_headers.insert(std::make_pair(arrow::util::string_view("set-cookie"),
+                                       arrow::util::string_view(incoming_cookie)));
+    middleware_->ReceivedHeaders(call_headers);
+    expected_cookie_cache_.UpdateCachedCookies(call_headers);
+
+    // Get cookie from middleware.
+    TestCallHeaders add_call_headers;
+    middleware_->SendingHeaders(&add_call_headers);
+    const std::string actual_cookies = add_call_headers.GetCookies();
+
+    // Validate cookie
+    const std::string expected_cookies = expected_cookie_cache_.GetValidCookiesAsString();
+    const std::vector<std::string> split_expected_cookies =
+        SplitCookies(expected_cookies);
+    const std::vector<std::string> split_actual_cookies = SplitCookies(actual_cookies);
+    EXPECT_EQ(split_expected_cookies, split_actual_cookies);
+  }
+
+  // Function to take a list of cookies and split them into a vector of individual
+  // cookies. This is done because the cookie cache is a map so ordering is not

Review comment:
       Can we use `unordered_map` then?




----------------------------------------------------------------
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 #8725: ARROW-10526: [FlightRPC][C++][Python] Client cookie middleware

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


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


----------------------------------------------------------------
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 #8725: ARROW-10526: [FlightRPC][C++] Client cookie middleware

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



##########
File path: cpp/src/arrow/flight/flight_test.cc
##########
@@ -1186,6 +1188,139 @@ class TestBasicHeaderAuthMiddleware : public ::testing::Test {
   std::shared_ptr<BearerAuthServerMiddlewareFactory> bearer_middleware_;
 };
 
+// This test keeps an internal cookie cache and compares that with the middleware.
+class TestCookieMiddleware : public ::testing::Test {
+ public:
+  // Setup function creates middleware factory and starts it up.
+  void SetUp() {
+    factory_ = GetCookieFactory();
+    CallInfo callInfo;
+    factory_->StartCall(callInfo, &middleware_);
+  }
+
+  // Function to add incoming cookies to middleware and validate them.
+  void AddAndValidate(const std::string& incoming_cookie) {
+    // Add cookie
+    CallHeaders call_headers;
+    call_headers.insert(std::make_pair(arrow::util::string_view("set-cookie"),
+                                       arrow::util::string_view(incoming_cookie)));
+    middleware_->ReceivedHeaders(call_headers);
+    expected_cookie_cache_.UpdateCachedCookies(call_headers);
+
+    // Get cookie from middleware.
+    TestCallHeaders add_call_headers;
+    middleware_->SendingHeaders(&add_call_headers);
+    const std::string actual_cookies = add_call_headers.GetCookies();
+
+    // Validate cookie
+    const std::string expected_cookies = expected_cookie_cache_.GetValidCookiesAsString();
+    const std::vector<std::string> split_expected_cookies =
+        SplitCookies(expected_cookies);
+    const std::vector<std::string> split_actual_cookies = SplitCookies(actual_cookies);
+    EXPECT_EQ(split_expected_cookies, split_actual_cookies);
+  }
+
+  // Function to take a list of cookies and split them into a vector of individual
+  // cookies. This is done because the cookie cache is a map so ordering is not

Review comment:
       Yes, good call. Will change 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 a change in pull request #8725: ARROW-10526: [FlightRPC][C++] Client cookie middleware

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



##########
File path: cpp/src/arrow/flight/flight_test.cc
##########
@@ -1186,6 +1188,159 @@ class TestBasicHeaderAuthMiddleware : public ::testing::Test {
   std::shared_ptr<BearerAuthServerMiddlewareFactory> bearer_middleware_;
 };
 
+// This test has functions that allow adding and removing cookies from a list of cookies.
+// It also automatically adds cookies to an internal list of tracked cookies when they
+// are passed to the middleware.
+class TestCookieMiddleware : public ::testing::Test {

Review comment:
       This is a general comment, but there's a lot of logic here, and some of it duplicates the logic in the middleware class itself. Can we factor out the cookie parsing into an internal header and add direct tests of that logic, and perhaps reuse some of it here (e.g. splitting and parsing cookies) instead of duplicating it? That would also let us directly test the cookie parsing, instead of indirectly through these tests.

##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,327 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+using CookiePair = arrow::util::optional<std::pair<std::string, std::string>>;
+
+/// \brief Parse a cookie header string beginning at the given start_pos and identify
+/// the name and value of an attribute.
+///
+/// \param cookie_header_value The value of the Set-Cookie header.
+/// \param start_pos           An input/output parameter indicating the starting position
+/// of the attribute.
+///                            It will store the position of the next attribute when the
+///                            function returns.
+///
+/// \return Optional cookie key value pair.
+CookiePair ParseCookieAttribute(const std::string& cookie_header_value,
+                                std::string::size_type& start_pos) {
+  std::string::size_type equals_pos = cookie_header_value.find('=', start_pos);
+  if (std::string::npos == equals_pos) {
+    // No cookie attribute.
+    return arrow::util::nullopt;
+  }
+
+  std::string::size_type semi_col_pos = cookie_header_value.find(';', equals_pos);
+  std::string out_key = arrow::internal::TrimString(
+      cookie_header_value.substr(start_pos, equals_pos - start_pos));
+  std::string out_value;
+  if (std::string::npos == semi_col_pos) {
+    // Last item - set start pos to end
+    out_value = arrow::internal::TrimString(cookie_header_value.substr(equals_pos + 1));
+    start_pos = std::string::npos;
+  } else {
+    out_value = arrow::internal::TrimString(
+        cookie_header_value.substr(equals_pos + 1, semi_col_pos - equals_pos - 1));
+    start_pos = semi_col_pos + 1;
+  }
+
+  // Key/Value may be URI-encoded.
+  out_key = arrow::internal::UriUnescape(out_key);
+  out_value = arrow::internal::UriUnescape(out_value);
+
+  // Strip outer quotes on the value.
+  if (out_value.size() >= 2 && out_value[0] == '"' &&
+      out_value[out_value.size() - 1] == '"') {
+    out_value = out_value.substr(1, out_value.size() - 2);
+  }
+
+  // Update the start position for subsequent calls to this function.
+  return std::make_pair(out_key, out_value);
+}
+
+/// \brief Function to fix date string so it is accepted by Windows parsers.
+///
+/// \param date Date to fix.
+void FixDate(std::string* date) {
+  // Abbreviated months in order.
+  static const std::vector<std::string> months = {
+      "JAN", "FEB", "MAR", "APR", "MAY", "JUN", "JUL", "AUG", "SEP", "OCT", "NOV", "DEC"};
+
+  // The date comes in with the following format: Wed, 01 Jan 3000 22:15:36 GMT
+  // Symbolics are not supported by Windows parsing, so we need to convert to
+  // the following format: 01 01 3000 22:15:36
+
+  // String is currently in regular format: 'Wed, 01 Jan 3000 22:15:36 GMT'
+  // Start by removing semicolon and trimming space.
+  *date = arrow::internal::TrimString(date->substr(date->find(",") + 1));
+
+  // String is now in trimmed format: '01 Jan 3000 22:15:36 GMT'
+  // Now swap month to proper month format for Windows.
+  // Start by removing case sensitivity.
+  std::transform(date->begin(), date->end(), date->begin(), ::toupper);
+
+  // Loop through months.
+  for (size_t i = 0; i < months.size(); i++) {
+    // Search the date for the month.
+    auto it = date->find(months[i]);
+    if (it != std::string::npos) {
+      // Create month integer, pad with leading zeros if required.
+      std::string padded_month;
+      if ((i + 1) < 10) {
+        padded_month = "0";
+      }
+      padded_month += std::to_string(i + 1);
+
+      // Replace symbolic month with numeric month.
+      date->replace(it, months[i].length(), padded_month);
+
+      // String is now in format: '01 01 3000 22:15:36 GMT'.
+      break;
+    }
+  }
+
+  // String is now in format '01 01 3000 22:15:36'.
+  auto it = date->find(" GMT");
+  date->erase(it, 4);
+
+  // Sometimes a semicolon is added at the end, if this is the case, remove it.
+  if (date->back() == ';') {
+    date->pop_back();
+  }
+}
+
+struct Cookie {
+  static Cookie parse(const arrow::util::string_view& cookie_header_value) {
+    // Parse the cookie string. If the cookie has an expiration, record it.
+    // If the cookie has a max-age, calculate the current time + max_age and set that as
+    // the expiration.
+    Cookie cookie;
+    cookie.has_expiry_ = false;
+    std::string cookie_value_str(cookie_header_value);
+
+    // There should always be a first match which should be the name and value of the
+    // cookie.
+    std::string::size_type pos = 0;
+    CookiePair cookie_pair = ParseCookieAttribute(cookie_value_str, pos);
+    if (!cookie_pair.has_value()) {
+      // No cookie found. Mark the output cookie as expired.
+      cookie.has_expiry_ = true;
+      cookie.expiration_time_ = std::chrono::system_clock::now();
+    } else {
+      cookie.cookie_name_ = cookie_pair.value().first;
+      cookie.cookie_value_ = cookie_pair.value().second;
+    }
+
+    while (pos < cookie_value_str.size()) {
+      cookie_pair = ParseCookieAttribute(cookie_value_str, pos);
+      if (!cookie_pair.has_value()) {
+        break;
+      }
+
+      std::string cookie_attr_value_str = cookie_pair.value().second;
+      if (arrow::internal::AsciiEqualsCaseInsensitive(cookie_pair.value().first,
+                                                      "max-age")) {
+        // Note: max-age takes precedence over expires. We don't really care about other
+        // attributes and will arbitrarily take the first max-age. We can stop the loop
+        // here.
+        cookie.has_expiry_ = true;
+        int max_age = std::stoi(cookie_attr_value_str);
+        if (max_age <= 0) {
+          // Force expiration.
+          cookie.expiration_time_ = std::chrono::system_clock::now();
+        } else {
+          // Max-age is in seconds.
+          cookie.expiration_time_ =
+              std::chrono::system_clock::now() + std::chrono::seconds(max_age);
+        }
+        break;
+      } else if (arrow::internal::AsciiEqualsCaseInsensitive(cookie_pair.value().first,
+                                                             "expires")) {
+        cookie.has_expiry_ = true;
+        int64_t seconds = 0;
+        FixDate(&cookie_attr_value_str);
+        const char* COOKIE_EXPIRES_FORMAT = "%d %m %Y %H:%M:%S";
+        if (arrow::internal::ParseTimestampStrptime(
+                cookie_attr_value_str.c_str(), cookie_attr_value_str.size(),
+                COOKIE_EXPIRES_FORMAT, false, true, arrow::TimeUnit::SECOND, &seconds)) {
+          cookie.expiration_time_ = std::chrono::time_point<std::chrono::system_clock>(
+              std::chrono::seconds(seconds));
+        } else {
+          // Force expiration.
+          cookie.expiration_time_ = std::chrono::system_clock::now();
+        }
+      }
+    }
+
+    return cookie;
+  }
+
+  bool IsExpired() const {
+    // Check if current-time is less than creation time.
+    return (has_expiry_ && (expiration_time_ <= std::chrono::system_clock::now()));
+  }
+
+  std::string AsCookieString() {
+    // Return the string for the cookie as it would appear in a Cookie header.
+    // Keys must be wrapped in quotes depending on if this is a v1 or v2 cookie.
+    return cookie_name_ + "=\"" + cookie_value_ + "\"";
+  }
+
+  std::string cookie_name_;
+  std::string cookie_value_;
+  std::chrono::time_point<std::chrono::system_clock> expiration_time_;
+  bool has_expiry_;
+};
+}  // end of anonymous namespace
+
+namespace arrow {
+namespace flight {
+
+using CookieCache = std::map<std::string, Cookie, CaseInsensitiveComparator>;
+using CookieHeaderPair =
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&;
+/// \brief Client-side middleware for sending/receiving HTTP cookies.
+class ClientCookieMiddlewareFactory : public ClientMiddlewareFactory {
+ public:
+  void StartCall(const CallInfo& info, std::unique_ptr<ClientMiddleware>* middleware) {
+    ARROW_UNUSED(info);
+    *middleware = std::unique_ptr<ClientMiddleware>(new ClientCookieMiddleware(*this));
+  }
+
+ private:
+  class ClientCookieMiddleware : public ClientMiddleware {
+   public:
+    explicit ClientCookieMiddleware(ClientCookieMiddlewareFactory& factory)
+        : factory_(factory) {}
+
+    void SendingHeaders(AddCallHeaders* outgoing_headers) override {
+      const std::string& cookie_string = factory_.GetValidCookiesAsString();
+      if (!cookie_string.empty()) {
+        outgoing_headers->AddHeader("cookie", cookie_string);
+      }
+    }
+
+    void ReceivedHeaders(const CallHeaders& incoming_headers) override {
+      CookieHeaderPair cookie_header_values = incoming_headers.equal_range("set-cookie");
+      factory_.UpdateCachedCookies(cookie_header_values);
+    }
+
+    void CallCompleted(const Status& status) override {}
+
+   private:
+    ClientCookieMiddlewareFactory& factory_;
+  };
+
+  /// \brief Retrieve the cached cookie values as a string.
+  ///
+  /// \return a string that can be used in a Cookie header representing the cookies that
+  /// have been cached.
+  std::string GetValidCookiesAsString() {
+    const std::lock_guard<std::mutex> guard(mutex_);
+
+    DiscardExpiredCookies();
+    if (cookie_cache_.empty()) {
+      return "";
+    }
+
+    std::string cookie_string = cookie_cache_.begin()->second.AsCookieString();
+    for (auto it = (++cookie_cache_.begin()); cookie_cache_.end() != it; ++it) {
+      cookie_string += "; " + it->second.AsCookieString();
+    }
+    return cookie_string;
+  }
+
+  /// \brief Updates the cache of cookies with new Set-Cookie header values.
+  ///
+  /// \param header_values The range representing header values.
+  void UpdateCachedCookies(CookieHeaderPair header_values) {
+    const std::lock_guard<std::mutex> guard(mutex_);
+
+    for (auto it = header_values.first; it != header_values.second; ++it) {
+      const util::string_view& value = it->second;
+      Cookie cookie = Cookie::parse(value);
+
+      // Cache cookies regardless of whether or not they are expired. The server may have
+      // explicitly sent a Set-Cookie to expire a cached cookie.
+      std::pair<CookieCache::iterator, bool> insertable =
+          cookie_cache_.insert({cookie.cookie_name_, cookie});
+
+      // Force overwrite on insert collision.
+      if (!insertable.second) {
+        insertable.first->second = cookie;
+      }
+    }
+
+    DiscardExpiredCookies();

Review comment:
       Is there any need to discard here if we have to discard before we send cookies anyways?

##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,327 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+using CookiePair = arrow::util::optional<std::pair<std::string, std::string>>;
+
+/// \brief Parse a cookie header string beginning at the given start_pos and identify
+/// the name and value of an attribute.
+///
+/// \param cookie_header_value The value of the Set-Cookie header.
+/// \param start_pos           An input/output parameter indicating the starting position
+/// of the attribute.
+///                            It will store the position of the next attribute when the
+///                            function returns.
+///
+/// \return Optional cookie key value pair.
+CookiePair ParseCookieAttribute(const std::string& cookie_header_value,
+                                std::string::size_type& start_pos) {
+  std::string::size_type equals_pos = cookie_header_value.find('=', start_pos);
+  if (std::string::npos == equals_pos) {
+    // No cookie attribute.
+    return arrow::util::nullopt;
+  }
+
+  std::string::size_type semi_col_pos = cookie_header_value.find(';', equals_pos);
+  std::string out_key = arrow::internal::TrimString(
+      cookie_header_value.substr(start_pos, equals_pos - start_pos));
+  std::string out_value;
+  if (std::string::npos == semi_col_pos) {
+    // Last item - set start pos to end
+    out_value = arrow::internal::TrimString(cookie_header_value.substr(equals_pos + 1));
+    start_pos = std::string::npos;
+  } else {
+    out_value = arrow::internal::TrimString(
+        cookie_header_value.substr(equals_pos + 1, semi_col_pos - equals_pos - 1));
+    start_pos = semi_col_pos + 1;
+  }
+
+  // Key/Value may be URI-encoded.
+  out_key = arrow::internal::UriUnescape(out_key);
+  out_value = arrow::internal::UriUnescape(out_value);
+
+  // Strip outer quotes on the value.
+  if (out_value.size() >= 2 && out_value[0] == '"' &&
+      out_value[out_value.size() - 1] == '"') {
+    out_value = out_value.substr(1, out_value.size() - 2);
+  }
+
+  // Update the start position for subsequent calls to this function.
+  return std::make_pair(out_key, out_value);
+}
+
+/// \brief Function to fix date string so it is accepted by Windows parsers.
+///
+/// \param date Date to fix.
+void FixDate(std::string* date) {
+  // Abbreviated months in order.
+  static const std::vector<std::string> months = {
+      "JAN", "FEB", "MAR", "APR", "MAY", "JUN", "JUL", "AUG", "SEP", "OCT", "NOV", "DEC"};
+
+  // The date comes in with the following format: Wed, 01 Jan 3000 22:15:36 GMT
+  // Symbolics are not supported by Windows parsing, so we need to convert to
+  // the following format: 01 01 3000 22:15:36
+
+  // String is currently in regular format: 'Wed, 01 Jan 3000 22:15:36 GMT'
+  // Start by removing semicolon and trimming space.
+  *date = arrow::internal::TrimString(date->substr(date->find(",") + 1));
+
+  // String is now in trimmed format: '01 Jan 3000 22:15:36 GMT'
+  // Now swap month to proper month format for Windows.
+  // Start by removing case sensitivity.
+  std::transform(date->begin(), date->end(), date->begin(), ::toupper);
+
+  // Loop through months.
+  for (size_t i = 0; i < months.size(); i++) {
+    // Search the date for the month.
+    auto it = date->find(months[i]);
+    if (it != std::string::npos) {
+      // Create month integer, pad with leading zeros if required.
+      std::string padded_month;
+      if ((i + 1) < 10) {
+        padded_month = "0";
+      }
+      padded_month += std::to_string(i + 1);
+
+      // Replace symbolic month with numeric month.
+      date->replace(it, months[i].length(), padded_month);
+
+      // String is now in format: '01 01 3000 22:15:36 GMT'.
+      break;
+    }
+  }
+
+  // String is now in format '01 01 3000 22:15:36'.
+  auto it = date->find(" GMT");
+  date->erase(it, 4);
+
+  // Sometimes a semicolon is added at the end, if this is the case, remove it.
+  if (date->back() == ';') {
+    date->pop_back();
+  }
+}
+
+struct Cookie {
+  static Cookie parse(const arrow::util::string_view& cookie_header_value) {
+    // Parse the cookie string. If the cookie has an expiration, record it.
+    // If the cookie has a max-age, calculate the current time + max_age and set that as
+    // the expiration.
+    Cookie cookie;
+    cookie.has_expiry_ = false;
+    std::string cookie_value_str(cookie_header_value);
+
+    // There should always be a first match which should be the name and value of the
+    // cookie.
+    std::string::size_type pos = 0;
+    CookiePair cookie_pair = ParseCookieAttribute(cookie_value_str, pos);
+    if (!cookie_pair.has_value()) {
+      // No cookie found. Mark the output cookie as expired.
+      cookie.has_expiry_ = true;
+      cookie.expiration_time_ = std::chrono::system_clock::now();
+    } else {
+      cookie.cookie_name_ = cookie_pair.value().first;
+      cookie.cookie_value_ = cookie_pair.value().second;
+    }
+
+    while (pos < cookie_value_str.size()) {
+      cookie_pair = ParseCookieAttribute(cookie_value_str, pos);
+      if (!cookie_pair.has_value()) {
+        break;
+      }
+
+      std::string cookie_attr_value_str = cookie_pair.value().second;
+      if (arrow::internal::AsciiEqualsCaseInsensitive(cookie_pair.value().first,
+                                                      "max-age")) {
+        // Note: max-age takes precedence over expires. We don't really care about other
+        // attributes and will arbitrarily take the first max-age. We can stop the loop
+        // here.
+        cookie.has_expiry_ = true;
+        int max_age = std::stoi(cookie_attr_value_str);
+        if (max_age <= 0) {
+          // Force expiration.
+          cookie.expiration_time_ = std::chrono::system_clock::now();
+        } else {
+          // Max-age is in seconds.
+          cookie.expiration_time_ =
+              std::chrono::system_clock::now() + std::chrono::seconds(max_age);
+        }
+        break;
+      } else if (arrow::internal::AsciiEqualsCaseInsensitive(cookie_pair.value().first,
+                                                             "expires")) {
+        cookie.has_expiry_ = true;
+        int64_t seconds = 0;
+        FixDate(&cookie_attr_value_str);
+        const char* COOKIE_EXPIRES_FORMAT = "%d %m %Y %H:%M:%S";

Review comment:
       Can this be moved to an actual constant following the naming convention (kCookieExpiresFormat or similar)?




----------------------------------------------------------------
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 #8725: ARROW-10526: [FlightRPC][C++][Python] Client cookie middleware

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



##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,353 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+// Parse a cookie header string beginning at the given start_pos and identify the name and
+// value of an attribute.
+//
+// @param cookie_header_value The value of the Set-Cookie header.
+// @param start_pos           An input/output parameter indicating the starting position
+// of the attribute.
+//                            It will store the position of the next attribute when the
+//                            function returns.
+// @param out_key             The name of the attribute.
+// @param out_value           The value of the attribute.
+//
+// @return true if an attribute is found.
+bool ParseCookieAttribute(std::string cookie_header_value,
+                          std::string::size_type& start_pos, std::string& out_key,
+                          std::string& out_value) {
+  std::string::size_type equals_pos = cookie_header_value.find('=', start_pos);
+  if (std::string::npos == equals_pos) {
+    // No cookie attribute.
+    return false;
+  }
+
+  std::string::size_type semi_col_pos = cookie_header_value.find(';', equals_pos);
+  out_key = arrow::internal::TrimString(
+      cookie_header_value.substr(start_pos, equals_pos - start_pos));
+  if (std::string::npos == semi_col_pos && semi_col_pos > equals_pos) {
+    // Last item - set start pos to end
+    out_value = arrow::internal::TrimString(cookie_header_value.substr(equals_pos + 1));
+    start_pos = std::string::npos;
+  } else {
+    out_value = arrow::internal::TrimString(
+        cookie_header_value.substr(equals_pos + 1, semi_col_pos - equals_pos - 1));
+    start_pos = semi_col_pos + 1;
+  }
+
+  // Key/Value may be URI-encoded.
+  out_key = arrow::internal::UriUnescape(arrow::util::string_view(out_key));
+  out_value = arrow::internal::UriUnescape(arrow::util::string_view(out_value));
+
+  // Strip outer quotes on the value.
+  if (out_value.size() >= 2 && out_value[0] == '"' &&
+      out_value[out_value.size() - 1] == '"') {
+    out_value = out_value.substr(1, out_value.size() - 2);
+  }
+
+  // Update the start position for subsequent calls to this function.
+  return true;
+}
+
+// Custom parser for the date in format expected for cookies. This is required because

Review comment:
       Unfortunately the vendored strptime does not support symbolics, which are required for cookie parsing. Instead of using an ad-hoc timestamp parser, I wrote a simple function to convert the symbolics to numerics, so the vendored strptime function will support it. Hope this solution is acceptable, of course I am open to other suggestions though.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [arrow] lidavidm commented on a change in pull request #8725: ARROW-10526: [FlightRPC][C++][Python] Client cookie middleware

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



##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,353 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+// Parse a cookie header string beginning at the given start_pos and identify the name and
+// value of an attribute.
+//
+// @param cookie_header_value The value of the Set-Cookie header.
+// @param start_pos           An input/output parameter indicating the starting position
+// of the attribute.
+//                            It will store the position of the next attribute when the
+//                            function returns.
+// @param out_key             The name of the attribute.
+// @param out_value           The value of the attribute.
+//
+// @return true if an attribute is found.
+bool ParseCookieAttribute(std::string cookie_header_value,

Review comment:
       You may appreciate arrow::util::optional

##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,353 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+// Parse a cookie header string beginning at the given start_pos and identify the name and
+// value of an attribute.
+//
+// @param cookie_header_value The value of the Set-Cookie header.

Review comment:
       Appreciate the docs, can we stick with Doxygen syntax (\brief, \param, etc) even though this won't be in the public docs?

##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,353 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+// Parse a cookie header string beginning at the given start_pos and identify the name and
+// value of an attribute.
+//
+// @param cookie_header_value The value of the Set-Cookie header.
+// @param start_pos           An input/output parameter indicating the starting position
+// of the attribute.
+//                            It will store the position of the next attribute when the
+//                            function returns.
+// @param out_key             The name of the attribute.
+// @param out_value           The value of the attribute.
+//
+// @return true if an attribute is found.
+bool ParseCookieAttribute(std::string cookie_header_value,
+                          std::string::size_type& start_pos, std::string& out_key,

Review comment:
       Please use std::string* for out parameters and const foo& for in parameters.

##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,353 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+// Parse a cookie header string beginning at the given start_pos and identify the name and
+// value of an attribute.
+//
+// @param cookie_header_value The value of the Set-Cookie header.
+// @param start_pos           An input/output parameter indicating the starting position
+// of the attribute.
+//                            It will store the position of the next attribute when the
+//                            function returns.
+// @param out_key             The name of the attribute.
+// @param out_value           The value of the attribute.
+//
+// @return true if an attribute is found.
+bool ParseCookieAttribute(std::string cookie_header_value,
+                          std::string::size_type& start_pos, std::string& out_key,
+                          std::string& out_value) {
+  std::string::size_type equals_pos = cookie_header_value.find('=', start_pos);
+  if (std::string::npos == equals_pos) {
+    // No cookie attribute.
+    return false;
+  }
+
+  std::string::size_type semi_col_pos = cookie_header_value.find(';', equals_pos);
+  out_key = arrow::internal::TrimString(
+      cookie_header_value.substr(start_pos, equals_pos - start_pos));
+  if (std::string::npos == semi_col_pos && semi_col_pos > equals_pos) {
+    // Last item - set start pos to end
+    out_value = arrow::internal::TrimString(cookie_header_value.substr(equals_pos + 1));
+    start_pos = std::string::npos;
+  } else {
+    out_value = arrow::internal::TrimString(
+        cookie_header_value.substr(equals_pos + 1, semi_col_pos - equals_pos - 1));
+    start_pos = semi_col_pos + 1;
+  }
+
+  // Key/Value may be URI-encoded.
+  out_key = arrow::internal::UriUnescape(arrow::util::string_view(out_key));
+  out_value = arrow::internal::UriUnescape(arrow::util::string_view(out_value));
+
+  // Strip outer quotes on the value.
+  if (out_value.size() >= 2 && out_value[0] == '"' &&
+      out_value[out_value.size() - 1] == '"') {
+    out_value = out_value.substr(1, out_value.size() - 2);
+  }
+
+  // Update the start position for subsequent calls to this function.
+  return true;
+}
+
+// Custom parser for the date in format expected for cookies. This is required because

Review comment:
       Do you mean that strptime is not on Windows? There's a vendored strptime available.

##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,353 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+// Parse a cookie header string beginning at the given start_pos and identify the name and
+// value of an attribute.
+//
+// @param cookie_header_value The value of the Set-Cookie header.
+// @param start_pos           An input/output parameter indicating the starting position
+// of the attribute.
+//                            It will store the position of the next attribute when the
+//                            function returns.
+// @param out_key             The name of the attribute.
+// @param out_value           The value of the attribute.
+//
+// @return true if an attribute is found.
+bool ParseCookieAttribute(std::string cookie_header_value,
+                          std::string::size_type& start_pos, std::string& out_key,
+                          std::string& out_value) {
+  std::string::size_type equals_pos = cookie_header_value.find('=', start_pos);
+  if (std::string::npos == equals_pos) {
+    // No cookie attribute.
+    return false;
+  }
+
+  std::string::size_type semi_col_pos = cookie_header_value.find(';', equals_pos);
+  out_key = arrow::internal::TrimString(
+      cookie_header_value.substr(start_pos, equals_pos - start_pos));
+  if (std::string::npos == semi_col_pos && semi_col_pos > equals_pos) {
+    // Last item - set start pos to end
+    out_value = arrow::internal::TrimString(cookie_header_value.substr(equals_pos + 1));
+    start_pos = std::string::npos;
+  } else {
+    out_value = arrow::internal::TrimString(
+        cookie_header_value.substr(equals_pos + 1, semi_col_pos - equals_pos - 1));
+    start_pos = semi_col_pos + 1;
+  }
+
+  // Key/Value may be URI-encoded.
+  out_key = arrow::internal::UriUnescape(arrow::util::string_view(out_key));

Review comment:
       Any reason why UriUnescape can't take a const std::string&?

##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,353 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+// Parse a cookie header string beginning at the given start_pos and identify the name and
+// value of an attribute.
+//
+// @param cookie_header_value The value of the Set-Cookie header.
+// @param start_pos           An input/output parameter indicating the starting position
+// of the attribute.
+//                            It will store the position of the next attribute when the
+//                            function returns.
+// @param out_key             The name of the attribute.
+// @param out_value           The value of the attribute.
+//
+// @return true if an attribute is found.
+bool ParseCookieAttribute(std::string cookie_header_value,
+                          std::string::size_type& start_pos, std::string& out_key,
+                          std::string& out_value) {
+  std::string::size_type equals_pos = cookie_header_value.find('=', start_pos);
+  if (std::string::npos == equals_pos) {
+    // No cookie attribute.
+    return false;
+  }
+
+  std::string::size_type semi_col_pos = cookie_header_value.find(';', equals_pos);
+  out_key = arrow::internal::TrimString(
+      cookie_header_value.substr(start_pos, equals_pos - start_pos));
+  if (std::string::npos == semi_col_pos && semi_col_pos > equals_pos) {

Review comment:
       Isn't this condition redundant? If no semicolon was found, then its position will always be greater than equals_pos

##########
File path: cpp/src/arrow/flight/client_cookie_middleware.h
##########
@@ -0,0 +1,45 @@
+// 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.
+
+// Middleware implementation for sending and receiving HTTP cookies.
+
+#pragma once
+
+#include <memory>
+
+#include "arrow/flight/client_middleware.h"
+
+namespace arrow {
+namespace flight {
+
+/// \brief Client-side middleware for sending/receiving HTTP cookies.
+class ARROW_FLIGHT_EXPORT ClientCookieMiddlewareFactory : public ClientMiddlewareFactory {

Review comment:
       pimpl pattern might be overkill here - what about a freestanding factory function that gives you a shared_ptr<ClientMiddlewareFactory>?

##########
File path: cpp/src/arrow/flight/client_cookie_middleware.cc
##########
@@ -0,0 +1,353 @@
+// 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.
+
+#include "arrow/flight/client_cookie_middleware.h"
+
+#include <chrono>
+#include <map>
+#include <mutex>
+#include <string>
+
+#include "arrow/flight/platform.h"
+#include "arrow/util/string.h"
+#include "arrow/util/uri.h"
+#include "arrow/util/value_parsing.h"
+
+namespace {
+#ifdef _WIN32
+#define strcasecmp stricmp
+#endif
+
+struct CaseInsensitiveComparator
+    : public std::binary_function<std::string, std::string, bool> {
+  bool operator()(const std::string& lhs, const std::string& rhs) const {
+    return strcasecmp(lhs.c_str(), rhs.c_str()) < 0;
+  }
+};
+
+// Parse a cookie header string beginning at the given start_pos and identify the name and
+// value of an attribute.
+//
+// @param cookie_header_value The value of the Set-Cookie header.
+// @param start_pos           An input/output parameter indicating the starting position
+// of the attribute.
+//                            It will store the position of the next attribute when the
+//                            function returns.
+// @param out_key             The name of the attribute.
+// @param out_value           The value of the attribute.
+//
+// @return true if an attribute is found.
+bool ParseCookieAttribute(std::string cookie_header_value,
+                          std::string::size_type& start_pos, std::string& out_key,
+                          std::string& out_value) {
+  std::string::size_type equals_pos = cookie_header_value.find('=', start_pos);
+  if (std::string::npos == equals_pos) {
+    // No cookie attribute.
+    return false;
+  }
+
+  std::string::size_type semi_col_pos = cookie_header_value.find(';', equals_pos);
+  out_key = arrow::internal::TrimString(
+      cookie_header_value.substr(start_pos, equals_pos - start_pos));
+  if (std::string::npos == semi_col_pos && semi_col_pos > equals_pos) {
+    // Last item - set start pos to end
+    out_value = arrow::internal::TrimString(cookie_header_value.substr(equals_pos + 1));
+    start_pos = std::string::npos;
+  } else {
+    out_value = arrow::internal::TrimString(
+        cookie_header_value.substr(equals_pos + 1, semi_col_pos - equals_pos - 1));
+    start_pos = semi_col_pos + 1;
+  }
+
+  // Key/Value may be URI-encoded.
+  out_key = arrow::internal::UriUnescape(arrow::util::string_view(out_key));
+  out_value = arrow::internal::UriUnescape(arrow::util::string_view(out_value));
+
+  // Strip outer quotes on the value.
+  if (out_value.size() >= 2 && out_value[0] == '"' &&
+      out_value[out_value.size() - 1] == '"') {
+    out_value = out_value.substr(1, out_value.size() - 2);
+  }
+
+  // Update the start position for subsequent calls to this function.
+  return true;
+}
+
+// Custom parser for the date in format expected for cookies. This is required because

Review comment:
       I would much rather we not maintain an ad-hoc timestamp parser if at all possible.




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