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/25 17:58:21 UTC

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

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