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

[GitHub] [arrow] indigophox opened a new pull request, #34817: Add Session management messages, Location URI path accessors

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

   Messages and handlers for SetSessionOptions, GetSessionOptions, and CloseSession added.  This establishes primitives for storage of server session state outside of Arrow Flight itself, allowing for catalog/resource-path type state to persist within and potentially without a Flight SQL connection, and any sort of other connection state desired by an application.
   
   Parsing logic and accessors for Location URI path component added, as raw path and as request headers.  This establishes a standard for application users (typically) to provide a server resource-path or otherwise as a URI path embedded in the string specifying the Location.
   
   [Tests and demo client/server code will be forthcoming as any feedback and integration thereof progresses.]
   
   
   <!--
   Thanks for opening a pull request!
   If this is your first pull request you can find detailed information on how 
   to contribute here:
     * [New Contributor's Guide](https://arrow.apache.org/docs/dev/developers/guide/step_by_step/pr_lifecycle.html#reviews-and-merge-of-the-pull-request)
     * [Contributing Overview](https://arrow.apache.org/docs/dev/developers/overview.html)
   
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/main/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on GitHub? https://github.com/apache/arrow/issues/new/choose
   
   Opening GitHub issues ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename the pull request title in the following format?
   
       GH-${GITHUB_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   In the case of PARQUET issues on JIRA the title also supports:
   
       PARQUET-${JIRA_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   -->
   
   ### Rationale for this change
   
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   ### What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   ### Are these changes tested?
   
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   2. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are they covered by existing tests)?
   -->
   
   ### Are there any user-facing changes?
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please uncomment the line below and explain which changes are breaking.
   -->
   <!-- **This PR includes breaking changes to public APIs.** -->
   
   <!--
   Please uncomment the line below (and provide explanation) if the changes fix either (a) a security vulnerability, (b) a bug that caused incorrect or invalid data to be produced, or (c) a bug that causes a crash (even when the API contract is upheld). We use this to highlight fixes to issues that may affect users without their knowledge. For this reason, fixing bugs that cause errors don't count, since those are usually obvious.
   -->
   <!-- **This PR contains a "Critical Fix".** -->


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/types.h:
##########
@@ -409,6 +409,15 @@ struct ARROW_FLIGHT_EXPORT Location {
   /// \brief Get the scheme of this URI.
   std::string scheme() const;
 
+  /// \brief Get the path of this URI.
+  std::string path() const;
+
+  /// \brief Get the query parameters of this URI.
+  arrow::Result<std::vector<std::pair<std::string, std::string>>> QueryItems() const;
+
+  /// \brief Convert URI path and parameters to headers.
+  arrow::Result<std::vector<std::pair<std::string, std::string>>> AsHeaders() const;

Review Comment:
   It would be good to update the mailing list at some point just to get some eyes on it. (I'll review this when I get a chance. Thanks for bearing with me.)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/client.cc:
##########
@@ -802,6 +802,157 @@ ::arrow::Result<CancelResult> FlightSqlClient::CancelQuery(
   return Status::IOError("Server returned unknown result ", result.result());
 }
 
+::arrow::Result<std::vector<SetSessionOptionResult>> FlightSqlClient::SetSessionOptions(
+    const FlightCallOptions& options,
+    const std::vector<SessionOption>& session_options) {
+  flight_sql_pb::ActionSetSessionOptionsRequest request;
+  for (const SessionOption& in_opt : session_options) {
+    flight_sql_pb::SessionOption* opt = request.add_session_options();
+    const std::string& name = in_opt.option_name;
+    opt->set_option_name(name);
+
+    const SessionOptionValue& value = in_opt.option_value;
+    if (value.index() == std::variant_npos)
+      return Status::Invalid("Undefined SessionOptionValue type ");
+    switch (static_cast<SessionOptionValueType>(value.index())) {

Review Comment:
   Probably worth doing to avoid the code duplication we'd discussed previously—on it.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/client.h:
##########
@@ -329,6 +329,25 @@ class ARROW_FLIGHT_SQL_EXPORT FlightSqlClient {
   /// \param[in] info         The FlightInfo of the query to cancel.
   ::arrow::Result<CancelResult> CancelQuery(const FlightCallOptions& options,
                                             const FlightInfo& info);
+   
+  /// \brief Sets session options.
+  ///
+  /// \param[in] options            RPC-layer hints for this call.
+  /// \param[in] session_options    The session options to set.
+  ::arrow::Result<std::vector<SetSessionOptionResult>> SetSessionOptions(

Review Comment:
   [see [above](https://github.com/apache/arrow/pull/34817#pullrequestreview-1366808831) for discussion]



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -423,6 +494,91 @@ arrow::Result<Result> PackActionResult(ActionCreatePreparedStatementResult resul
   return PackActionResult(pb_result);
 }
 
+arrow::Result<Result> PackActionResult(ActionSetSessionOptionsResult result) {
+  pb::sql::ActionSetSessionOptionsResult pb_result;
+  for (SetSessionOptionResult& res : result.results) {
+    switch (res) {
+      case SetSessionOptionResult::kUnspecified:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_UNSPECIFIED);
+        break;
+      case SetSessionOptionResult::kOk:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_OK);
+        break;
+      case SetSessionOptionResult::kInvalidResult:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_INVALID_VALUE);
+        break;
+      case SetSessionOptionResult::kError:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_ERROR);
+        break;
+    }
+  }
+  return PackActionResult(pb_result);
+}
+
+arrow::Result<Result> PackActionResult(ActionGetSessionOptionsResult result) {
+  pb::sql::ActionGetSessionOptionsResult pb_result;
+  for (const SessionOption& in_opt : result.session_options) {
+    pb::sql::SessionOption& opt = *pb_result.add_session_options();

Review Comment:
   What I mean is
   
   > Right now there isn't a common location outside of types.h and overriding Proto constructors from there is kind of ugly factoring as well.
   
   We can't refactor anything dealing with Protobuf into a public header. We can have internal headers.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;
+    }
+
+    if (!session_id.length()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers,
+                                        session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid(
+            "Invalid or expired "
+            + static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;

Review Comment:
   Can emplace be used here instead?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   As a https://github.com/apache/arrow-flight-sql-postgresql developer, I want to create a new session only when authentication is succeeded. Can we use this feature for this use 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,205 @@
+// 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/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");

Review Comment:
   It should, and apparently, this extends down to HTTP/2 as well



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,205 @@
+// 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/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");

Review Comment:
   gRPC only ever uses lowercase headers (the c++ impl will actually crash if you don't)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }

Review Comment:
   I discussed this fairly extensively with @lidavidm and the existing cookie functionality is heavily intertwined with managing the client-side cookie jar, so essentially we would need to factor the zero-copy stateful parser out as a generator which is getting pretty involved and could accidentally break the existing code.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> CreateNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex mutex_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session_;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session_(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session_(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session_ && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::lock_guard<std::shared_mutex> l(mutex_);
+    if (!session_) {
+      auto [id, s] = factory_->CreateNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const auto pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");
+  for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+    const std::string_view& cookie_header = itr->second;
+    const std::vector<std::pair<std::string, std::string>> cookies =
+        ParseCookieString(cookie_header);
+    for (const std::pair<std::string, std::string>& cookie : cookies) {
+      if (cookie.first == kSessionCookieName) {
+        if (cookie.second.empty())
+          return Status::Invalid("Empty ", kSessionCookieName, " cookie value.");
+        session_id = std::move(cookie.second);
+      }
+    }
+    if (!session_id.empty()) break;
+  }
+
+  if (session_id.empty()) {
+    // No cookie was found
+    *middleware = std::make_shared<ServerSessionMiddlewareImpl>(this, incoming_headers);
+  } else {
+    const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+    if (auto it = session_store_.find(session_id); it == session_store_.end()) {
+      return Status::Invalid("Invalid or expired ", kSessionCookieName, " cookie.");
+    } else {
+      auto session = it->second;
+      *middleware = std::make_shared<ServerSessionMiddlewareImpl>(
+          this, incoming_headers, std::move(session), session_id);
+    }
+  }
+
+  return Status::OK();
+}
+
+/// \brief Get a new, empty session option map and its id key.
+std::pair<std::string, std::shared_ptr<FlightSqlSession>>
+ServerSessionMiddlewareFactory::CreateNewSession() {
+  auto new_id = id_generator_();

Review Comment:
   We may want to explicitly document that the generator should be thread-safe and that it must avoid collisions.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    OK = 1;
+    // The given session option name was an alias for another option name.
+    OK_MAPPED = 2;
+    // The given session option name is invalid.
+    INVALID_NAME = 3;
+    // The session option value is invalid.
+    INVALID_VALUE = 4;
+    // The session option cannot be set.
+    ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}

Review Comment:
   Can we spell this out in the format? i.e. it is implementation-defined when and how to create the session (possibly on authentication or otherwise implicitly)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;

Review Comment:
   > Should we reserve the OK statuses in case for some hitherto-unknown reason we decide to use them eventually (so that they're consistent with the enum ordering used elsewhere)?
   
   This seems overkill.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }

Review Comment:
   Ping, any further thoughts on getting tests working locally?  Are you & David developing on Linux or where is this working properly?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/CloseSessionResult.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class CloseSessionResult {
+  public enum Status {
+    /**
+     * The session close status is unknown. Servers should avoid using this value
+     * (send a NOT_FOUND error if the requested session is not known). Clients can
+     * retry the request.
+     */
+    UNSPECIFIED(Flight.CloseSessionResult.Status.UNSPECIFIED),
+    /**
+     * The session close request is complete.
+     */
+    CLOSED(Flight.CloseSessionResult.Status.CLOSED),
+    /**
+     * The session close request is in progress. The client may retry the request.
+     */
+    CLOSING(Flight.CloseSessionResult.Status.CLOSING),
+    /**
+     * The session is not closeable.
+     */
+    NOT_CLOSABLE(Flight.CloseSessionResult.Status.NOT_CLOSABLE),
+    ;
+
+    private static final Map<Flight.CloseSessionResult.Status, Status> mapFromProto;
+
+    static {
+      for (Status s : values()) mapFromProto.put(s.proto, s);

Review Comment:
   What would probably better than a map is an abstract function on the enum that returns the status.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.

Review Comment:
   "what not how", yeah +1



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(

Review Comment:
   So Cookie::Parse and Cookie::ParseCookieAttribute are both stateful in terms of the string parsing (ok) but also heavily intertwined with the logic for managing the set-cookie semantics.  Really all we're reusing is "split on '; '" and "split on '='" which unfortunately C++ doesn't have canned, but IMO not worth chainsawwing existing cookie_internal code and creating a dependency on it to reuse a conceptually very simple string splitting routine and no further specialized cookie logic.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/client.cc:
##########
@@ -802,6 +802,157 @@ ::arrow::Result<CancelResult> FlightSqlClient::CancelQuery(
   return Status::IOError("Server returned unknown result ", result.result());
 }
 
+::arrow::Result<std::vector<SetSessionOptionResult>> FlightSqlClient::SetSessionOptions(
+    const FlightCallOptions& options,
+    const std::vector<SessionOption>& session_options) {
+  flight_sql_pb::ActionSetSessionOptionsRequest request;
+  for (const SessionOption& in_opt : session_options) {
+    flight_sql_pb::SessionOption* opt = request.add_session_options();
+    const std::string& name = in_opt.option_name;
+    opt->set_option_name(name);
+
+    const SessionOptionValue& value = in_opt.option_value;
+    if (value.index() == std::variant_npos)
+      return Status::Invalid("Undefined SessionOptionValue type ");
+    switch (static_cast<SessionOptionValueType>(value.index())) {

Review Comment:
   (done)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid("Invalid or expired " +
+                               static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());

Review Comment:
   Apparently not.  Good catch, 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {

Review Comment:
   Done.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,182 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+using namespace std::string_literals;
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      if (val_pos == std::string::npos) {
+        // The cookie header is somewhat malformed; ignore the key and continue parsing
+        continue;
+      }
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          if (cookie.second.empty())
+            return Status::Invalid("Empty "s + kSessionCookieName + " cookie value.");

Review Comment:
   Done (in both cases).



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>

Review Comment:
   Absolutely agreed, on it.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid("Invalid or expired " +
+                               static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());

Review Comment:
   Resolving as this will be moot once factored out.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;

Review Comment:
   This is true. Perhaps we can find a terse convention that would point to a common explanation somewhere at the top of the file?
   
   



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   For consistency e.g. https://github.com/apache/arrow/blob/d66780d90c6faf7bf051cbf12b4b68dff098bd54/format/Flight.proto#L241-L255
   
   While it's a valid point to make the documentation of these values more clear, I believe it needs to be done globally to avoid implying differences in the different enum values' meaning if we document them differently.
   
   Presumably "not yet closed" may fit e.g. @kou's use case of rolling auth into sessions, where closing a session would also nullify AuthNZ context attached to the session—it may be desirable to confirm that temporary auth tokens (session tokens) have in fact been successfully invalidated e.g. in a distributed environment.  "Cannot be closed" would then fit a situation where the session was previously CLOSING but acknowledgement from all nodes had not been received before a timeout was reached...



##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   For consistency e.g. https://github.com/apache/arrow/blob/d66780d90c6faf7bf051cbf12b4b68dff098bd54/format/Flight.proto#L241-L255
   
   While it's a valid point to make the documentation of these values more clear, I believe it needs to be done globally to avoid implying differences in the different enum values' meaning if we document them differently.
   
   Presumably "not yet closed" may fit e.g. @kou's use case of rolling auth into sessions, where closing a session would also nullify AuthNZ context attached to the session—it may be desirable to confirm that temporary auth tokens (session tokens) have in fact been successfully invalidated e.g. in a distributed environment.  "Cannot be closed" would then fit a situation where the session was previously CLOSING but acknowledgement from all nodes had not been received before a timeout was reached...



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   For consistency e.g. https://github.com/apache/arrow/blob/d66780d90c6faf7bf051cbf12b4b68dff098bd54/format/Flight.proto#L241-L255
   
   Presumably "not yet closed" may fit e.g. @kou's use case of rolling auth into sessions, where closing a session would also nullify AuthNZ context attached to the session—it may be desirable to confirm that temporary auth tokens (session tokens) have in fact been successfully invalidated e.g. in a distributed environment.  "Cannot be closed" would then fit a situation where the session was previously CLOSING but acknowledgement from all nodes had not been received before a timeout was reached...



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.

Review Comment:
   This isn't intended to be gRPC-specific. Any transport protocol that supports CallHeaders should behave the same.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>

Review Comment:
   There are many UUID libraries available, but there are a limited number that work as well as boost. Most do not randomize with 128-bits of precision (see this discussion: https://stackoverflow.com/questions/24365331/how-can-i-generate-uuid-in-c-without-using-boost-library). Others have a C++17 requirement -- would that be OK to use in the Arrow project?
   
   [crossguid](https://github.com/graeme-hill/crossguid) utilizes operating system calls for generating UUIDs so it should generate them securely enough. It supports Windows, Mac, Linux, and Android -- is this enough platform support for Arrow?
   
   



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,117 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ *
+ * By convention, an attempt to set a valueless SessionOptionValue should
+ * attempt to unset or clear the named option value on the server.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed64 int64_value = 3;
+    double double_value = 4;
+    StringListValue string_list_value = 5;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via a transport-level state management, typically
+ * RFC 6265 HTTP cookies when using an HTTP transport.  The suggested cookie name or state
+ * context key is 'arrow_flight_session_id', although implementations may freely choose their
+ * own name.
+ *
+ * Session creation (if one does not already exist) is implied by this RPC request, however
+ * server implementations may choose to initiate a session that also contains client-provided
+ * session options at any other time, e.g. on authentication, or when any other call is made
+ * and the server wishes to use a session to persist any state (or lack thereof).
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ *
+ * Option names should only be present in the response if they were not successfully
+ * set on the server; that is, a response without an Error for a name provided in the
+ * SetSessionOptionsRequest implies that the named option value was set successfully.
+ */
+message SetSessionOptionsResult {
+  enum ErrorValue {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.

Review Comment:
   There, hopefully that's clear enough that that doesn't happen again :P



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-integration-tests/pom.xml:
##########
@@ -45,6 +45,10 @@
             <groupId>com.google.protobuf</groupId>
             <artifactId>protobuf-java</artifactId>
         </dependency>
+	<dependency>

Review Comment:
   There's also tabs, IntelliJ got into the catnip again...



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
docs/source/format/FlightSql.rst:
##########
@@ -170,6 +170,47 @@ the ``type`` should be ``ClosePreparedStatement``).
     When used with DoPut: execute the query and return the number of
     affected rows.
 
+Flight Server Session Management
+--------------------------------
+
+Flight SQL provides commands to set and update server session variables
+which affect the server behaviour in various ways.  Common options may
+include (depending on the server implementation) ``catalog`` and
+``schema``, indicating the currently-selected catalog and schema for
+queries to be run against.
+
+Clients should prefer, where possible, setting options prior to issuing
+queries and other commands, as some server implementations may require
+these options be set exactly once and prior to any other activity which
+may trigger their implicit setting.
+
+For compatibility with Database Connectivity drivers (JDBC, ODBC, and
+others), it is strongly recommended that server implementations accept
+string representations of all option values which may be provided to the
+driver as part of a server connection string and passed through to the
+server without further conversion.  For ease of use it is also recommended
+to accept and convert other numeric types to the preferred type for an
+option value, however this is not required.
+
+Sessions are persisted between the client and server using an
+implementation-defined mechanism, which is typically RFC 6265 cookies.

Review Comment:
   Do we want to recommend a cookie name for Flight sessions?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   @lidavidm curious how things progress from here and if there's any further action expected from me?  I presume I should be watching the vote when it comes around in case any further feedback does come up?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   > Ohhh I just saw exactly what you were talking about, I'd thought for whatever reason that the result stream was passed to DoAction by the transport but there it is, the stream is constructed by the handler... so ok, I can skip over the more involved stream-wrapper-triggered header ingestion and just move ahead with adding a defaulted flag to CheckAuth (and so on) to defer the ingestion to "elsewhere" and as you said above make that "elsewhere" L528 or so (for consistency do I not want to actually do this before handling an empty result stream, which it would currently happen before?)
   
   I am lost. You would stop calling SendingHeaders where it was previously called, and start calling it explicitly in each RPC handler at an appropriate time. I don't see why empty streams need special treatment.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
cpp/src/arrow/flight/sql/protocol_internal.h:
##########
@@ -24,3 +24,4 @@
 #include "arrow/flight/sql/visibility.h"
 
 #include "arrow/flight/sql/FlightSql.pb.h"  // IWYU pragma: export
+#include "arrow/flight/Flight.pb.h"

Review Comment:
   @lidavidm re: https://github.com/apache/arrow/pull/34817/files#r1245582470 not exactly sure of preference here, whether we consider the Flight messages to be part of the FSQL protocol, or include both protocols in the FSQL client/server modules (more noisy but...).  Preference?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -1072,6 +1260,18 @@ Status FlightSqlServerBase::EndTransaction(const ServerCallContext& context,
   return Status::NotImplemented("EndTransaction not implemented");
 }
 
+arrow::Result<ActionSetSessionOptionsResult> FlightSqlServerBase::SetSessionOptions (
+    const ServerCallContext& context,
+    const ActionSetSessionOptionsRequest& request) {
+  return Status::NotImplemented("SetSessionOptions not implemented");
+}

Review Comment:
   Ok, thanks. As it stands it seems we really do need the middleware for this to make any sense.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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

   Just checking in here - what's the status of things?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValueFactory.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.SessionOptionValue;
+
+public class SessionOptionValueFactory {
+    public static SessionOptionValue makeSessionOption(String value) {
+        return new SessionOptionValueString(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(bool value) {
+        return new SessionOptionValueBool(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(int value) {
+        return new SessionOptionValueInt(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(long value) {
+        return new SessionOptionValueLong(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(float value) {
+        return new SessionOptionValueFloat(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(double value) {
+        return new SessionOptionValueDouble(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(String[] value) {
+        return new SessionOptionValueStringList(value);
+    }
+}
+
+class SessionOptionValueString {

Review Comment:
   > I -think-, unless there are broken edge cases for this in Proto, the most straightforward thing here would be to handle a value variant that maps to an unset `oneof` over the wire. (a) I think we can add that later though and (b) does that sound sane (both the impl and feasibility of deferring it for now)?
   
   @lidavidm any objections to this idea before it moves ahead, "unset" options using the VALUE_NOT_SET case over the wire and presumably then std::variant::valueless_by_exception() and having a visit(Void) overload in Javaland?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   > I kind of agree with @pitrou . Client may not be interested and/or forgot to close the session, or a network or a backend issue may happen which may cause the session not to be closed proactively, so I would also adopt a best effort approach where the client may notify the server the session is to be closed, and the server will do its best to honor the request?
   
   I think most server implementations will likely just use `CLOSING` which may then well progress to either `CLOSED` or a `NOT_FOUND` error, depending on how the service stores (or not) invalidated session identifiers.  That said I think if someone wants to, it may be useful to provide more information to the user on the client end if for example a distributed service is having issues fully propagating session invalidation, which may for example be causing exhaustion of a session count limit or something that the user wants to be able to diagnose.  Nobody has to use them, but if we don't provide enums to express wonky things happening in a distributed context then services won't be able to handle that informatively even if they want to.
   
   > I would also be hesitant of tightening them to auth session instead of keeping separate as they may different lifecycle and TTLs
   
   This is definitely not being explicitly encouraged, but anyone implementing a Flight service is free to do as they wish to simplify their implementation.  This is going to be documented as such.
   
   



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   There's some CI failures as well:
   
   ```
   /arrow/cpp/src/arrow/flight/sql/server_session_middleware.cc: In member function 'std::pair<std::__cxx11::basic_string<char>, std::shared_ptr<arrow::flight::sql::FlightSqlSession> > arrow::flight::sql::ServerSessionMiddlewareFactory::GetNewSession()':
   /arrow/cpp/src/arrow/flight/sql/server_session_middleware.cc:170:14: error: 'unique_lock' in namespace 'std' does not name a template type
     170 |   const std::unique_lock<std::shared_mutex> l(session_store_lock_);
         |              ^~~~~~~~~~~
   /arrow/cpp/src/arrow/flight/sql/server_session_middleware.cc:19:1: note: 'std::unique_lock' is defined in header '<mutex>'; did you forget to '#include <mutex>'?
   ```
   
   ```
   /arrow/cpp/src/arrow/flight/sql/server_session_middleware.cc:35:  Single-parameter constructors should be marked explicit.  [runtime/explicit] [5]
   ```


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,205 @@
+// 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/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");

Review Comment:
   Is this case-sensitive?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);

Review Comment:
   Correct, refactored that a few times and broke it.  Fixed there and elsewhere.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;

Review Comment:
   Certainly.  For example to return that an option value cannot be set on the server, we either expose this or construct a structured error message to capture which option names had which success or error.  This way, drivers or generic clients could easily (and programmatically, without parsing human-readable error/warning text) establish a fallback behaviour given unavailability of specific keys or values.  In a more simple case, some specific name or value failures may be fatal while others may be attempts to set non-critical "preferences" e.g. more efficient result packing or the like.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;

Review Comment:
   Hmm, I'm curious about this. The [protobuf language guide](https://protobuf.dev/programming-guides/proto3/#enum) does state that "there must be a zero value, so that we can use 0 as a numeric default value". It doesn't state what this default value _should mean_, though.
   
   Since this enum denotes an error return, it would be reasonable for the default value to mean success, rather than "unspecified".



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;

Review Comment:
   Isn't this a bit overkill? Do we envision clients doing different things based on per-key results?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;

Review Comment:
   Would it be reasonable for the "default" to be success, and just *only* report failures/errors, and the client can assume any option they set which is not reported in the response was successful?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/CloseSessionRequest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class CloseSessionRequest {
+  public CloseSessionRequest() {}
+
+  CloseSessionRequest(Flight.CloseSessionRequest proto) {}
+
+  Flight.CloseSessionRequest toProtocol() {
+    Flight.CloseSessionRequest.Builder b = Flight.CloseSessionRequest.newBuilder();
+    return b.build();
+  }
+
+  /**
+   * Get the serialized form of this protocol message.
+   *
+   * <p>Intended to help interoperability by allowing non-Flight services to still return Flight types.
+   */
+  public ByteBuffer serialize() {
+    return ByteBuffer.wrap(toProtocol().toByteArray());

Review Comment:
   It seems like making this do that blindly without a check that the object definition remains empty is a bit of a [maintenance] tripwire, and this is called once for a gRPC round-trip so it's pretty inexpensive to leave it doing the whole thing.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;

Review Comment:
   -> existing thread https://github.com/apache/arrow/pull/34817#discussion_r1407471516



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValueFactory.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.SessionOptionValue;
+
+public class SessionOptionValueFactory {
+    public static SessionOptionValue makeSessionOption(String value) {
+        return new SessionOptionValueString(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(bool value) {
+        return new SessionOptionValueBool(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(int value) {
+        return new SessionOptionValueInt(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(long value) {
+        return new SessionOptionValueLong(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(float value) {
+        return new SessionOptionValueFloat(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(double value) {
+        return new SessionOptionValueDouble(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(String[] value) {
+        return new SessionOptionValueStringList(value);
+    }
+}
+
+class SessionOptionValueString {

Review Comment:
   Good point.  Will consider how to integrate that; I believe the Proto default is something like the first value, empty, so an empty string?  But that could conflict with other intents, so we might need an additional dummy oneof state.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   Hey @lidavidm, I hear (to my surprise, I guess?) that it's common to engage maintainers regarding setting up the integration test stuff correctly.  I've been looking over it (in anticipation of it being a DIY thing) but definitely hit a few things that were less than obvious... can you fill me in on how a correctly-set-up set of tests looks structurally (looks like runner.py is the top-level but unclear if there's any language-local stuff driven independently of that for non-cross-language execution of the same tests?) and so on?  Thanks!   (Also can you give me an idea of when you're around/not in the coming weeks so I don't come back around with questions after you're already off for the holidays?)


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   @indigophox I'm also able to assist if the examples that @lidavidm linked to aren't enough to figure it out. Feel free to ping me if you can't get a hold of @lidavidm 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");
+  for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+    const std::string_view& cookie_header = itr->second;
+    const std::vector<std::pair<std::string, std::string>> cookies =
+        ParseCookieString(cookie_header);
+    for (const std::pair<std::string, std::string>& cookie : cookies) {
+      if (cookie.first == kSessionCookieName) {
+        if (cookie.second.empty())
+          return Status::Invalid("Empty ", kSessionCookieName, " cookie value.");
+        session_id = std::move(cookie.second);
+      }
+    }
+    if (!session_id.empty()) break;
+  }
+
+  if (session_id.empty()) {
+    // No cookie was found
+    *middleware = std::make_shared<ServerSessionMiddlewareImpl>(this, incoming_headers);
+  } else {
+    const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+    if (auto it = session_store_.find(session_id); it == session_store_.end()) {
+      return Status::Invalid("Invalid or expired ", kSessionCookieName, " cookie.");
+    } else {
+      auto session = it->second;
+      *middleware = std::make_shared<ServerSessionMiddlewareImpl>(
+          this, incoming_headers, std::move(session), session_id);
+    }
+  }
+
+  return Status::OK();
+}
+
+/// \brief Get a new, empty session option map and its id key.
+std::pair<std::string, std::shared_ptr<FlightSqlSession>>
+ServerSessionMiddlewareFactory::GetNewSession() {
+  std::string new_id = id_generator_();
+  auto session = std::make_shared<FlightSqlSession>();
+
+  const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+  session_store_[new_id] = session;

Review Comment:
   I'd say it's up to the application to provide a generator that avoids collisions (using UUID etc).



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   > trigger session creation on successful auth
   
   My use case is this pattern.
   
   > it is "normally" triggered on DoAction/SetSessionOptions calls
   
   Does this mean that we send user name and password as session options with this approach?
   
   FYI: https://github.com/apache/arrow-flight-sql-postgresql  uses the standard `authentication` header for sending user name and password for now.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>

Review Comment:
   C++17 is ok.
   
   Antoine is right, strictly speaking, we could make this implementation take a generator for unique IDs, and a server implementation could plug a UUID (or other) generator into it. For unit tests, we can just use a random string. Let's do that @indigophox and avoid the dependency entirely.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;

Review Comment:
   I'm ok with that.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/client.h:
##########
@@ -383,6 +383,27 @@ class ARROW_FLIGHT_EXPORT FlightClient {
     return DoExchange({}, descriptor);
   }
 
+  /// \\brief Set server session option(s) by key/value. Sessions are generally
+  /// persisted via HTTP cookies.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The server session options to set
+  ::arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const FlightCallOptions& options, const SetSessionOptionsRequest& request);
+
+  /// \\brief Get the current server session options. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The GetSessionOptions request object.

Review Comment:
   Not a deal breaker. Ok resolved.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid("Invalid or expired " +
+                               static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session, std::string session_id)
+    : factory_(factory),
+      headers_(headers),
+      session_(std::move(session)),
+      session_id_(std::move(session_id)),
+      existing_session(true) {}
+
+void ServerSessionMiddleware::SendingHeaders(AddCallHeaders* addCallHeaders) {
+  if (!existing_session && session_) {
+    addCallHeaders->AddHeader(
+        "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+  }
+}
+
+void ServerSessionMiddleware::CallCompleted(const Status&) {}
+
+bool ServerSessionMiddleware::HasSession() const { return static_cast<bool>(session_); }
+
+std::shared_ptr<FlightSqlSession> ServerSessionMiddleware::GetSession() {
+  if (!session_) session_ = factory_->GetNewSession(&session_id_);
+  return session_;
+}
+
+const CallHeaders& ServerSessionMiddleware::GetCallHeaders() const { return headers_; }
+
+std::shared_ptr<ServerMiddlewareFactory> MakeServerSessionMiddlewareFactory() {
+  return std::shared_ptr<ServerSessionMiddlewareFactory>(
+      new ServerSessionMiddlewareFactory());
+}
+
+::arrow::Result<SessionOptionValue> FlightSqlSession::GetSessionOption(

Review Comment:
   Agreed; done.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   > I'm not sure what you're asking; runner.py just executes binaries supplied to it. The existing C++ tests are the best reference for what is available to you.
   
   @lidavidm So I think the gotcha I was expecting to hit (and did) with Flight SQL is that it appears that the existing integration test infra is built around a FlightServer (that can only be modified by calls to its builder), whereas a lot of the Flight SQL handlers are built into the server rather than the producer... so implementing those as part of the test fixtures is probably impossible as things stand right now?  Do you see any straightforward way to implement and use a customized FlightSqlServerBase implementation, or is the only way what @kou did by cutting in at doAction in a NoOpFlightProducer subclass?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   > There's an integration test that uses FlightSqlServerBase:
   
   Perfect, I see what I misunderstood—whoops.
   
   Can you tell me anything more about running the integration testing stuff locally, namely I see there's e.g. Scenarios.java which has a main() that runs the integration suite language-local, but I'm not clear on where to manually run that from, if it's already running as part of the build as well(?!), or if the cross-language tests can be run locally as well?  TIA!


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
docs/source/format/FlightSql.rst:
##########
@@ -170,6 +170,47 @@ the ``type`` should be ``ClosePreparedStatement``).
     When used with DoPut: execute the query and return the number of
     affected rows.
 
+Flight Server Session Management
+--------------------------------
+
+Flight SQL provides commands to set and update server session variables
+which affect the server behaviour in various ways.  Common options may
+include (depending on the server implementation) ``catalog`` and
+``schema``, indicating the currently-selected catalog and schema for
+queries to be run against.
+
+Clients should prefer, where possible, setting options prior to issuing
+queries and other commands, as some server implementations may require
+these options be set exactly once and prior to any other activity which
+may trigger their implicit setting.
+
+For compatibility with Database Connectivity drivers (JDBC, ODBC, and
+others), it is strongly recommended that server implementations accept
+string representations of all option values which may be provided to the
+driver as part of a server connection string and passed through to the
+server without further conversion.  For ease of use it is also recommended
+to accept and convert other numeric types to the preferred type for an
+option value, however this is not required.
+
+Sessions are persisted between the client and server using an
+implementation-defined mechanism, which is typically RFC 6265 cookies.

Review Comment:
   I'd thought about it but it's opaque to the client so being prescriptive isn't of much value.  The middleware shipped with this uses a name that service implementers are free to use, but it doesn't much matter, and I certainly don't want to imply that Flight cares.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   I've sent out the vote


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   Could you update the PR description?
   
   Could you rebase on main to fix R related CI jobs?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/integration_tests/test_integration.cc:
##########
@@ -744,6 +746,155 @@ class ExpirationTimeRenewFlightEndpointScenario : public Scenario {
   }
 };
 
+/// \brief The server used for testing Session Options.
+///
+/// setSessionOptions has a blacklisted option name and string option value,
+/// both "lol_invalid", which will result in errors attempting to set either.
+class SessionOptionsServer : public sql::FlightSqlServerBase {
+  static inline const std::string invalid_option_name = "lol_invalid";
+  static inline const SessionOptionValue invalid_option_value = "lol_invalid";
+
+  const std::string session_middleware_key;
+  // These will never be threaded so using a plain map and no lock
+  std::map<std::string, SessionOptionValue> session_store_;
+
+ public:
+  explicit SessionOptionsServer(std::string session_middleware_key)
+      : FlightSqlServerBase(),
+        session_middleware_key(std::move(session_middleware_key)) {}
+
+  arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const ServerCallContext& context,
+      const SetSessionOptionsRequest& request) override {
+    SetSessionOptionsResult res;
+
+    sql::ServerSessionMiddleware* middleware =
+        (sql::ServerSessionMiddleware*)context.GetMiddleware(session_middleware_key);
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<sql::FlightSession> session,
+                          middleware->GetSession());
+
+    for (const auto& [name, value] : request.session_options) {
+      // Blacklisted value name
+      if (name == invalid_option_name) {
+        res.errors.emplace(name, SetSessionOptionsResult::Error{
+                                     SetSessionOptionErrorValue::kInvalidName});
+        continue;
+      }
+      // Blacklisted option value
+      if (value == invalid_option_value) {
+        res.errors.emplace(name, SetSessionOptionsResult::Error{
+                                     SetSessionOptionErrorValue::kInvalidValue});
+        continue;
+      }
+      if (std::holds_alternative<std::monostate>(value)) {
+        session->EraseSessionOption(name);
+        continue;
+      }
+      session->SetSessionOption(name, value);
+    }
+
+    return res;
+  }
+
+  arrow::Result<GetSessionOptionsResult> GetSessionOptions(
+      const ServerCallContext& context,
+      const GetSessionOptionsRequest& request) override {
+    auto* middleware = static_cast<sql::ServerSessionMiddleware*>(
+        context.GetMiddleware(session_middleware_key));
+    if (!middleware->HasSession()) {
+      return Status::Invalid("No existing session to get options from.");
+    }
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<sql::FlightSession> session,
+                          middleware->GetSession());
+
+    return GetSessionOptionsResult{session->GetSessionOptions()};
+  }
+
+  arrow::Result<CloseSessionResult> CloseSession(
+      const ServerCallContext& context, const CloseSessionRequest& request) override {
+    // Broken (does not expire cookie) until C++ middleware SendingHeaders handling fixed.

Review Comment:
   Done, pending a push so I don't spam the CI bot too much.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,117 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ *
+ * By convention, an attempt to set a valueless SessionOptionValue should
+ * attempt to unset or clear the named option value on the server.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed64 int64_value = 3;
+    double double_value = 4;
+    StringListValue string_list_value = 5;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via a transport-level state management, typically
+ * RFC 6265 HTTP cookies when using an HTTP transport.  The suggested cookie name or state
+ * context key is 'arrow_flight_session_id', although implementations may freely choose their
+ * own name.
+ *
+ * Session creation (if one does not already exist) is implied by this RPC request, however
+ * server implementations may choose to initiate a session that also contains client-provided
+ * session options at any other time, e.g. on authentication, or when any other call is made
+ * and the server wishes to use a session to persist any state (or lack thereof).
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ *
+ * Option names should only be present in the response if they were not successfully
+ * set on the server; that is, a response without an Error for a name provided in the
+ * SetSessionOptionsRequest implies that the named option value was set successfully.
+ */
+message SetSessionOptionsResult {
+  enum ErrorValue {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.

Review Comment:
   Yeah, just thinking the comment could capture that more succinctly and avoid the repeat discussion (and any other confusion e.g. by service/client implementers etc.).  Going to go at least update mine to that end.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   So looks like this code (generic to all handlers so it would need either a global solution or a flag to determine control flow based on whether the handler provides an alternative) is where we're going off the rails, particularly the last line being inside the "setup" part here instead of after the handler has done -anything- https://github.com/apache/arrow/blob/3fe598ae4dfd7805ab05452dd5ed4b0d6c97d8d5/cpp/src/arrow/flight/transport/grpc/grpc_server.cc#L327-L339


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   (It would also badly break CloseSession, which would have to defer cookie invalidation until a subsequent call whereupon the session cookie would be looked up, identified as previously invalidated, and replaced with a new one as part of StartCall...)


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;
+    }
+
+    if (!session_id.length()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers,
+                                        session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid(
+            "Invalid or expired "
+            + static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session,
+    std::string session_id)
+    : factory_(factory), headers_(headers), session_(session), existing_session(true) {}
+
+void ServerSessionMiddleware::SendingHeaders(AddCallHeaders* addCallHeaders) {
+  if (!existing_session && session_) {
+    addCallHeaders->AddHeader(
+        "set-cookie",
+        static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+  }
+}
+
+void ServerSessionMiddleware::CallCompleted(const Status&) {}
+
+bool ServerSessionMiddleware::HasSession() const {
+  return static_cast<bool>(session_);
+}
+std::shared_ptr<FlightSqlSession> ServerSessionMiddleware::GetSession() {
+  if (!session_)
+    session_ = factory_->GetNewSession(&session_id_);
+  return session_;
+}
+const CallHeaders& ServerSessionMiddleware::GetCallHeaders() const {
+  return headers_;
+}
+
+
+
+std::shared_ptr<ServerMiddlewareFactory> MakeServerSessionMiddlewareFactory() {
+  return std::shared_ptr<ServerSessionMiddlewareFactory>(
+      new ServerSessionMiddlewareFactory());
+}
+
+SessionOptionValue FlightSqlSession::GetSessionOption(const std::string_view k) {
+  const std::shared_lock<std::shared_mutex> l(map_lock_);
+  return map_.at(k);
+}
+void FlightSqlSession::SetSessionOption(const std::string_view, const SessionOptionValue& v) {
+  const std::unique_lock<std::shared_mutex> l(map_lock_);
+  map_[k] = v;
+}
+void FlightSqlSession::EraseSessionOption(const std::string_view) {
+  const std::unique_lock<std::shared_mutex> l(map_lock_);
+  map_.erase(k);

Review Comment:
   What's the intended behavior if k is not found?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;
+    }
+
+    if (!session_id.length()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers,
+                                        session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid(
+            "Invalid or expired "
+            + static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session,
+    std::string session_id)
+    : factory_(factory), headers_(headers), session_(session), existing_session(true) {}
+
+void ServerSessionMiddleware::SendingHeaders(AddCallHeaders* addCallHeaders) {
+  if (!existing_session && session_) {
+    addCallHeaders->AddHeader(
+        "set-cookie",
+        static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+  }
+}
+
+void ServerSessionMiddleware::CallCompleted(const Status&) {}
+
+bool ServerSessionMiddleware::HasSession() const {
+  return static_cast<bool>(session_);
+}
+std::shared_ptr<FlightSqlSession> ServerSessionMiddleware::GetSession() {
+  if (!session_)
+    session_ = factory_->GetNewSession(&session_id_);
+  return session_;
+}
+const CallHeaders& ServerSessionMiddleware::GetCallHeaders() const {
+  return headers_;
+}
+
+
+
+std::shared_ptr<ServerMiddlewareFactory> MakeServerSessionMiddlewareFactory() {
+  return std::shared_ptr<ServerSessionMiddlewareFactory>(
+      new ServerSessionMiddlewareFactory());
+}
+
+SessionOptionValue FlightSqlSession::GetSessionOption(const std::string_view k) {
+  const std::shared_lock<std::shared_mutex> l(map_lock_);
+  return map_.at(k);

Review Comment:
   This will throw if k is not in the map. Is that intended?
   Could use map.find(k) instead then evaluate the return value.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,90 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+class ServerSessionMiddlewareFactory;
+
+static constexpr char const kSessionCookieName[] =
+    "flight_sql_session_id";
+
+class FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+ public:
+  /// \brief Get session option by key
+  SessionOptionValue GetSessionOption(const std::string_view);
+  /// \brief Set session option by key to given value
+  void SetSessionOption(const std::string_view, const SessionOptionValue&);
+  /// \brief Idempotently remove key from this call's Session, if Session & key exist
+  void EraseSessionOption(const std::string_view);
+};
+
+/// \brief A middleware to handle Session option persistence and related *Cookie headers.
+class ARROW_FLIGHT_SQL_EXPORT ServerSessionMiddleware
+    : public ServerMiddleware {
+ public:
+  static constexpr char const kMiddlewareName[] =
+      "arrow::flight::sql::ServerSessionMiddleware";
+
+  std::string name() const override { return kMiddlewareName; }
+  void SendingHeaders(AddCallHeaders*) override;
+  void CallCompleted(const Status&) override;
+
+  /// \brief Is there an existing session (either existing or new)
+  bool HasSession() const;
+  /// \brief Get existing or new call-associated session
+  std::shared_ptr<FlightSqlSession> GetSession();
+  /// \brief Get request headers, in lieu of a provided or created session.
+  const CallHeaders& GetCallHeaders() const;
+
+ protected:
+    friend class ServerSessionMiddlewareFactory;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+  ServerSessionMiddleware(ServerSessionMiddlewareFactory*,
+                          const CallHeaders&);
+  ServerSessionMiddleware(ServerSessionMiddlewareFactory*,
+                          const CallHeaders&,
+                          std::shared_ptr<FlightSqlSession>,
+                          std::string session_id);
+};
+
+/// \brief Returns a ServerMiddlewareFactory that handles Session option storage.
+ARROW_FLIGHT_SQL_EXPORT std::shared_ptr<ServerMiddlewareFactory>
+MakeServerSessionMiddlewareFactory();
+
+} // namespace sql
+} // namespace flight
+} // namespace arrow

Review Comment:
   Nit: please add a newline to the end of file



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   > Just checking in here - what's the status of things?
   
   Just pushed a WIP commit if you want to take a look at the high-level design; don't pay much attention to a lot of the incomplete function code quite yet.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   Re: the lower-level helper question, I think the middleware implementation we were debating whether to include is (a) at the right interface level (bypassing the middleware interface probably doesn't make a lot of sense, and it's necessary to use the Flight Server Middleware interface to allow the handlers to juggle the inbound and particularly outbound session ID cookie local to the call, and (b) a complete reference as to the required mechanics to make everything work should anyone feel the need to try to do the same thing differently.
   
   Re: header support, I'm not 100% sure if you're talking about the session ID or the options themselves.  I am reticent to provide default (part of Arrow vs. app-defined) support of arbitrary option headers, but as these are explicitly not part of the session functionality and work via headers that the client handlers can already access and handle I think supporting that is a no-op.  As to providing headers (instead of full client middleware) as a fallback for keeping/sharing the session ID across e.g. distributed clients or whatever, the session ID is already a header (albeit a cookie) so I can simply provide the accessor we were considering for getting the session ID out of the provided middleware implementation (probably a subclass with just the extra accessor(s)) and then client apps can do as they please with that data.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   The concern was simply minimizing how much more stuff we're jamming into the Arrow package (and if it's not really necessary etc), however I think this is both important enough (to clarify the flow described above so we're not giving users/developers a puzzle to put together) and not more bulk added to the core code (Middleware being an optional helper) so it's probably worth adding an interface and impl to the package.
   
   Do the coding guidelines being used discourage MI?—by default I'm looking at implementing the core Middleware interface, plus the session object getter via another interface that the new server session middleware would also inherit.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
cpp/src/arrow/flight/sql/client.cc:
##########
@@ -31,6 +31,7 @@
 #include "arrow/result.h"
 #include "arrow/util/logging.h"
 
+namespace pb = arrow::flight::protocol;

Review Comment:
   Considered doing this as `flight_pb` however this is more consistent with usage elsewhere and could eliminate uses of `flight_sql_pb` for better uniformity.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   Ok, thanks - that's what I was trying to get at.
   
   Without the middleware, how do you expect servers to do all this? 
   
   What is the concern about having middleware? Is it possible to build a set of low-level helpers (presumably, what a server would need to do if it didn't have middleware) as well as a higher-level middleware using those helpers?
    
   Also, I am not sure the headers are only a legacy path. Not all clients are willing or able to maintain persistent state (and/or persistent state may be unreliable). So we should consider the headers as a first-class path as well.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   > I'm referring to the options themselves; I thought the original proposal was that servers should also support sending options as arbitrary headers, since the existing JDBC driver does that.
   
   At least internally we're looking at moving a bunch of things off of being sent as headers and over to session options.  For backwards compatibility internally as well as in the wild I'm inclined to pursue making the JDBC and ODBC drivers detect the Session Option doAction verbs and use them if present and only fall back to headers if the verbs are absent.  (Relatedly, it might be useful to have the doAction listing populated dynamically or overridden by the app's client subclass as the discrepancy between the listing and what Actions do and do not return a not implemented status message is a bit confusing for this and other purposes...)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   > I'm referring to the options themselves; I thought the original proposal was that servers should also support sending options as arbitrary headers, since the existing JDBC driver does that.
   
   At least internally we're looking at moving a bunch of things off of being sent as headers and over to session options [for robustness, tidiness, header limits, etc.].  For backwards compatibility internally as well as in the wild I'm inclined to pursue making the JDBC and ODBC drivers detect the Session Option doAction verbs and use them if present and only fall back to headers if the verbs are absent.  (Relatedly, it might be useful to have the doAction listing populated dynamically or overridden by the app's client subclass as the discrepancy between the listing and what Actions do and do not return a not implemented status message is a bit confusing for this and other purposes...)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
cpp/src/arrow/flight/sql/types.h:
##########


Review Comment:
   Debatable, I leaned towards "we're already not exposing the ser/des stuff in Flight so why pollute it with enums and so on that don't pertain to anything, when anyone implementing handlers for these has to touch the Proto types anyways".  If you really want they can be moved but I do think it adds orphaned bits of noisy stuff to Flight without them being tied to anything in the Flight component of the package.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.cc:
##########
@@ -463,6 +463,300 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+// Helper for stringifying maps containing various types
+template <typename T>
+ostream& operator<<(std::map<std::string, T>) {
+  std::stringstream ss;
+
+  ss << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : session_options) {
+    std::cout << sep << '[' << k << "]: '" << v << "', ";  // PHOXME v.ToString() not implemented yet
+    sep = ", ";
+  }
+  ss << '}';
+
+  return ss.str();
+}
+static bool CompareSessionOptionMaps(

Review Comment:
   Helpers can go into the anonymous namespace instead of a global static symbol



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;

Review Comment:
   More clear if this is !session_id.empty()
   Or better yet, just create a bool hasSessionId



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,77 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <functional>
+#include <optional>
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+static constexpr char const kSessionCookieName[] = "arrow_flight_session_id";
+
+class FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+
+ public:
+  /// \brief Get session option by key
+  ::arrow::Result<std::optional<SessionOptionValue>>

Review Comment:
   Agreed, just optional<>.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########


Review Comment:
   Ok, your call - I recognize this task has dragged on a long time already because of the review process



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();

Review Comment:
   Sure, 'Get' seems "traditional" but clarity is better.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);

Review Comment:
   While it's malformed and should probably yell at the client, that's harder to diagnose so favoured just ignoring the extraneous cookie name and carrying on.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.h:
##########
@@ -742,6 +746,164 @@ struct ARROW_FLIGHT_EXPORT CancelFlightInfoRequest {
   static arrow::Result<CancelFlightInfoRequest> Deserialize(std::string_view serialized);
 };
 
+/// \brief Variant supporting all possible value types for {Set,Get}SessionOptions
+using SessionOptionValue = std::variant<std::string, bool, int32_t, int64_t, float,
+                                        double, std::vector<std::string>>;
+
+/// \brief The result of setting a session option.
+enum class SetSessionOptionStatus : int8_t {
+  kUnspecified,

Review Comment:
   Done.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {

Review Comment:
   Avoiding TOCTOU here; I don't think there's another way to do that atomically?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/serialization_internal.cc:
##########
@@ -372,6 +372,174 @@ Status ToPayload(const FlightDescriptor& descr, std::shared_ptr<Buffer>* out) {
   return Status::OK();
 }
 
+// SessionOptionValue
+
+Status FromProto(const pb::SessionOptionValue& pb_val
+                                     SessionOptionValue* val) {
+  switch (pb_opt_val.option_value_case()) {
+    case pb::SessionOptionValue::OPTION_VALUE_NOT_SET:
+      return Status::Invalid("Unset option_value for name '" +
+                             pb_opt_name + "'");
+    case pb::SessionOptionValue::kStringValue:
+      val = pb_opt_val.string_value();
+      break;
+    case pb::SessionOptionValue::kBoolValue:
+      val = pb_opt_val.bool_value();
+      break;
+    case pb::SessionOptionValue::kInt32Value:
+      val = pb_opt_val.int32_value();
+      break;
+    case pb::SessionOptionValue::kInt64Value:
+      val = pb_opt_val.int64_value();
+      break;
+    case pb::SessionOptionValue::kFloatValue:
+      val = pb_opt_val.float_value();
+      break;
+    case pb::SessionOptionValue::kDoubleValue:
+      val = pb_opt_val.double_value();
+      break;
+    case pb::SessionOptionValue::kStringListValue:
+      val.emplace<std::vector<std::string>>();
+      std::get<std::vector<std::string>>(val)
+          .reserve(pb_opt_val.string_list_value().values_size());
+      for (const std::string& s : pb_opt_val.string_list_value().values())
+        std::get<std::vector<std::string>>(val).push_back(s);
+      break;
+  }
+  return Status::OK();
+}
+
+Status ToProto(const SessionOptionValue& val
+                                       pb::SessionOptionValue pb_val) {
+  std::visit(overloaded{
+      [&](std::string v) { pb_val.set_string_value(v); },
+      [&](bool v) { pb_val.set_bool_value(v); },
+      [&](int32_t v) { pb_val.set_int32_value(v); },
+      [&](int64_t v) { pb_val.set_int64_value(v); },
+      [&](float v) { pb_val.set_float_value(v); },
+      [&](double v) { pb_val.set_double_value(v); },
+      [&](std::vector<std::string> v) {
+        auto* string_list_value = pb_val.mutable_string_list_value();
+        for (const std::string& s : v)
+          string_list_value->add_values(s);
+      }
+    }, opt_value);
+  return Status::OK();
+}
+
+// map<string, SessionOptionValue>
+
+Status FromProto(const google::protobuf::map<string,
+                                             pb::SessionOptionValue> pb_map,   //PHOXME maybe need to include google/protobuf/map.h ?  shouldn't this be brought in by other headers?
+                 std::map<std::string, SessionOptionValue>* map) {
+  if (pb_map.size() == 0) {
+    return Status::OK();
+  }
+  for (auto& [key, pb_val] : pb_map.session_options()) {
+    RETURN_NOT_OK(FromProto(pb_val, &(*map)[key]));
+  }
+  return Status::OK();
+}
+
+Status ToProto(const std::map<std::string, SessionOptionValue> map,
+               google::protobuf::map<string, pb::SessionOptionValue>* pb_map) {
+  for (const auto & [key, val] : map) {
+    RETURN_NOT_OK(ToProto(val, &pb_map[key]));

Review Comment:
   Caught and fixed before I saw this, ty



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }

Review Comment:
   On that note, circling back around on tests, is there a trick to getting ARROW_BUILD_TESTS to not explode?  Getting a massive pile of errors, topmost of which is:
   
   `gmock-actions.h:342:5: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'`
   
   @kou @lidavidm ?  (C++ dev docs seem to suggest it Just Works)
   
   [Notably this is on macOS 14.1.x]



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }

Review Comment:
   Could you show full error log?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> CreateNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex mutex_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session_;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session_(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session_(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session_ && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::lock_guard<std::shared_mutex> l(mutex_);
+    if (!session_) {
+      auto [id, s] = factory_->CreateNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const auto pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");
+  for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+    const std::string_view& cookie_header = itr->second;
+    const std::vector<std::pair<std::string, std::string>> cookies =
+        ParseCookieString(cookie_header);
+    for (const std::pair<std::string, std::string>& cookie : cookies) {
+      if (cookie.first == kSessionCookieName) {
+        if (cookie.second.empty())
+          return Status::Invalid("Empty ", kSessionCookieName, " cookie value.");
+        session_id = std::move(cookie.second);
+      }
+    }
+    if (!session_id.empty()) break;
+  }
+
+  if (session_id.empty()) {
+    // No cookie was found
+    *middleware = std::make_shared<ServerSessionMiddlewareImpl>(this, incoming_headers);
+  } else {
+    const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+    if (auto it = session_store_.find(session_id); it == session_store_.end()) {
+      return Status::Invalid("Invalid or expired ", kSessionCookieName, " cookie.");
+    } else {
+      auto session = it->second;
+      *middleware = std::make_shared<ServerSessionMiddlewareImpl>(
+          this, incoming_headers, std::move(session), session_id);
+    }
+  }
+
+  return Status::OK();
+}
+
+/// \brief Get a new, empty session option map and its id key.
+std::pair<std::string, std::shared_ptr<FlightSqlSession>>
+ServerSessionMiddlewareFactory::CreateNewSession() {
+  auto new_id = id_generator_();

Review Comment:
   Regarding both this and the other thread (ha) about this, probably worth erroring on collision (don't bother with a retry strategy) and a single SetSO call can fail once (ugly, but the generator's fault) instead of corrupting existing server session state (worse).
   
   [Or the app can retry in its SetSO impl if using a poor generator...]



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   I think for the streaming cases it would be possible to inject a callback into the stream (or stream wrapper as in some of the cases where the Result is wrapped outside of the handler) that would trigger touching the middleware's SendingHeaders on the first stream message posting—this would allow server-streaming handlers to monkey with middleware in a way that affects its outbound headers but also provide a non-breaking default for existing ones and ones that don't care...


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   @lidavidm just re-reading this after doing some other work...
   
   I'm getting the impression the ordering issues you're talking about would apply more to `ServerCallContext::AddHeader` (where the handler impl is in control of the order of operations), whereas the Flight framework has control over when `ServerMiddleware::SendingHeaders` so there's not so much a concern about what's done before/after what by the new feature code?  Just wanted to check we're on the same page about the specifics here, as I'm only dealing with the `SendingHeaders` callback case.
   
   If you can provide some guidance as to the intended architecture here I can probably find the cycles to do the implementation, as I understand you're currently pretty strapped for dev time here.
   
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   @lidavidm ok for purposes of the PR merge I'm going to defang CloseSession in the C++ MW, do you have a preference between these two moderately ugly temporary solutions:
   
   * return CLOSED but leave the client cookie floating around in space and invalid; or
   * return NOT_CLOSABLE (the thing there was so much debate about) because we can't implement closing sessions yet ?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   That could maybe be solved but it would have to be on a per-handler basis (e.g. DoPut/DoExchange can't do this easily). 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   AppVeyor is a timeout; the Java pipeline is a flake


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/integration_tests/test_integration.cc:
##########
@@ -744,6 +746,155 @@ class ExpirationTimeRenewFlightEndpointScenario : public Scenario {
   }
 };
 
+/// \brief The server used for testing Session Options.
+///
+/// setSessionOptions has a blacklisted option name and string option value,

Review Comment:
   ```suggestion
   /// SetSessionOptions has a blacklisted option name and string option value,
   ```



##########
cpp/src/arrow/flight/integration_tests/test_integration.cc:
##########
@@ -744,6 +746,155 @@ class ExpirationTimeRenewFlightEndpointScenario : public Scenario {
   }
 };
 
+/// \brief The server used for testing Session Options.
+///
+/// setSessionOptions has a blacklisted option name and string option value,
+/// both "lol_invalid", which will result in errors attempting to set either.
+class SessionOptionsServer : public sql::FlightSqlServerBase {
+  static inline const std::string invalid_option_name = "lol_invalid";
+  static inline const SessionOptionValue invalid_option_value = "lol_invalid";
+
+  const std::string session_middleware_key;
+  // These will never be threaded so using a plain map and no lock
+  std::map<std::string, SessionOptionValue> session_store_;
+
+ public:
+  explicit SessionOptionsServer(std::string session_middleware_key)
+      : FlightSqlServerBase(),
+        session_middleware_key(std::move(session_middleware_key)) {}
+
+  arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const ServerCallContext& context,
+      const SetSessionOptionsRequest& request) override {
+    SetSessionOptionsResult res;
+
+    sql::ServerSessionMiddleware* middleware =
+        (sql::ServerSessionMiddleware*)context.GetMiddleware(session_middleware_key);
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<sql::FlightSession> session,
+                          middleware->GetSession());
+
+    for (const auto& [name, value] : request.session_options) {
+      // Blacklisted value name
+      if (name == invalid_option_name) {
+        res.errors.emplace(name, SetSessionOptionsResult::Error{
+                                     SetSessionOptionErrorValue::kInvalidName});
+        continue;
+      }
+      // Blacklisted option value
+      if (value == invalid_option_value) {
+        res.errors.emplace(name, SetSessionOptionsResult::Error{
+                                     SetSessionOptionErrorValue::kInvalidValue});
+        continue;
+      }
+      if (std::holds_alternative<std::monostate>(value)) {
+        session->EraseSessionOption(name);
+        continue;
+      }
+      session->SetSessionOption(name, value);
+    }
+
+    return res;
+  }
+
+  arrow::Result<GetSessionOptionsResult> GetSessionOptions(
+      const ServerCallContext& context,
+      const GetSessionOptionsRequest& request) override {
+    auto* middleware = static_cast<sql::ServerSessionMiddleware*>(
+        context.GetMiddleware(session_middleware_key));
+    if (!middleware->HasSession()) {
+      return Status::Invalid("No existing session to get options from.");
+    }
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<sql::FlightSession> session,
+                          middleware->GetSession());
+
+    return GetSessionOptionsResult{session->GetSessionOptions()};
+  }
+
+  arrow::Result<CloseSessionResult> CloseSession(
+      const ServerCallContext& context, const CloseSessionRequest& request) override {
+    // Broken (does not expire cookie) until C++ middleware SendingHeaders handling fixed.

Review Comment:
   Is this still broken?



##########
cpp/src/arrow/flight/integration_tests/test_integration.cc:
##########
@@ -744,6 +746,155 @@ class ExpirationTimeRenewFlightEndpointScenario : public Scenario {
   }
 };
 
+/// \brief The server used for testing Session Options.
+///
+/// setSessionOptions has a blacklisted option name and string option value,
+/// both "lol_invalid", which will result in errors attempting to set either.
+class SessionOptionsServer : public sql::FlightSqlServerBase {
+  static inline const std::string invalid_option_name = "lol_invalid";
+  static inline const SessionOptionValue invalid_option_value = "lol_invalid";
+
+  const std::string session_middleware_key;
+  // These will never be threaded so using a plain map and no lock
+  std::map<std::string, SessionOptionValue> session_store_;
+
+ public:
+  explicit SessionOptionsServer(std::string session_middleware_key)
+      : FlightSqlServerBase(),
+        session_middleware_key(std::move(session_middleware_key)) {}
+
+  arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const ServerCallContext& context,
+      const SetSessionOptionsRequest& request) override {
+    SetSessionOptionsResult res;
+
+    sql::ServerSessionMiddleware* middleware =
+        (sql::ServerSessionMiddleware*)context.GetMiddleware(session_middleware_key);

Review Comment:
   ```suggestion
       auto middleware =
           static_cast<sql::ServerSessionMiddleware*>(context.GetMiddleware(session_middleware_key));
   ```



##########
cpp/src/arrow/flight/integration_tests/test_integration.cc:
##########
@@ -744,6 +746,155 @@ class ExpirationTimeRenewFlightEndpointScenario : public Scenario {
   }
 };
 
+/// \brief The server used for testing Session Options.
+///
+/// setSessionOptions has a blacklisted option name and string option value,
+/// both "lol_invalid", which will result in errors attempting to set either.
+class SessionOptionsServer : public sql::FlightSqlServerBase {
+  static inline const std::string invalid_option_name = "lol_invalid";
+  static inline const SessionOptionValue invalid_option_value = "lol_invalid";
+
+  const std::string session_middleware_key;
+  // These will never be threaded so using a plain map and no lock
+  std::map<std::string, SessionOptionValue> session_store_;
+
+ public:
+  explicit SessionOptionsServer(std::string session_middleware_key)
+      : FlightSqlServerBase(),
+        session_middleware_key(std::move(session_middleware_key)) {}
+
+  arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const ServerCallContext& context,
+      const SetSessionOptionsRequest& request) override {
+    SetSessionOptionsResult res;
+
+    sql::ServerSessionMiddleware* middleware =
+        (sql::ServerSessionMiddleware*)context.GetMiddleware(session_middleware_key);
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<sql::FlightSession> session,
+                          middleware->GetSession());
+
+    for (const auto& [name, value] : request.session_options) {
+      // Blacklisted value name
+      if (name == invalid_option_name) {
+        res.errors.emplace(name, SetSessionOptionsResult::Error{
+                                     SetSessionOptionErrorValue::kInvalidName});
+        continue;
+      }
+      // Blacklisted option value
+      if (value == invalid_option_value) {
+        res.errors.emplace(name, SetSessionOptionsResult::Error{
+                                     SetSessionOptionErrorValue::kInvalidValue});
+        continue;
+      }
+      if (std::holds_alternative<std::monostate>(value)) {
+        session->EraseSessionOption(name);
+        continue;
+      }
+      session->SetSessionOption(name, value);
+    }
+
+    return res;
+  }
+
+  arrow::Result<GetSessionOptionsResult> GetSessionOptions(
+      const ServerCallContext& context,
+      const GetSessionOptionsRequest& request) override {
+    auto* middleware = static_cast<sql::ServerSessionMiddleware*>(
+        context.GetMiddleware(session_middleware_key));
+    if (!middleware->HasSession()) {
+      return Status::Invalid("No existing session to get options from.");
+    }
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<sql::FlightSession> session,
+                          middleware->GetSession());
+
+    return GetSessionOptionsResult{session->GetSessionOptions()};
+  }
+
+  arrow::Result<CloseSessionResult> CloseSession(
+      const ServerCallContext& context, const CloseSessionRequest& request) override {
+    // Broken (does not expire cookie) until C++ middleware SendingHeaders handling fixed.
+    sql::ServerSessionMiddleware* middleware =
+        (sql::ServerSessionMiddleware*)context.GetMiddleware(session_middleware_key);

Review Comment:
   ```suggestion
       auto middleware =
           static_cast<sql::ServerSessionMiddleware*>(context.GetMiddleware(session_middleware_key));
   ```



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   Thanks for the ping, I'll review within the next few days


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   @lidavidm Thanks!
   
   Also, any thoughts on getting the C++ tests to at least build locally?  Getting both tedious and more importantly wasteful to validate anything server-side instead of locally, when building with tests (this is with `ninja-debug-flight-sql`) seems to not work locally:  (note I have up-to-date googletest-1.14.0 installed)
   
   ```
   $ cmake --build .
   [1/611] Building CXX object _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o
   FAILED: _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o 
   /opt/homebrew/bin/ccache /Applications/Xcode.app/Contents/Developer/Toolchains/XcodeDefault.xctoolchain/usr/bin/c++ -DARROW_EXTRA_ERROR_CONTEXT -DARROW_HAVE_NEON -DARROW_WITH_TIMING_TESTS -DGTEST_CREATE_SHARED_LIBRARY=1 -Dgmock_EXPORTS -I/Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include -I/Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock -isystem /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googletest/include -isystem /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googletest -Qunused-arguments -fcolor-diagnostics -Wno-unused-value -Wno-ignored-attributes -g -Werror -O0 -ggdb  -Wno-error -arch arm64 -isysroot /Applications/Xcode.app/Contents/Developer/Platforms/MacOSX.platform/Developer/SDKs/MacOSX14.2.sdk -fPIC -Wall -Wshadow -Werror -Wconversion -DGTEST_HAS_PTHREAD=1 -fexceptions -W -W
 pointer-arith -Wreturn-type -Wcast-qual -Wwrite-strings -Wswitch -Wunused-parameter -Wcast-align -Wchar-subscripts -Winline -Wredundant-decls -std=c++17 -MD -MT _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o -MF _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o.d -o _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o -c /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/src/gmock-all.cc
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/src/gmock-all.cc:39:
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock.h:59:
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:342:5: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
       GTEST_DISALLOW_COPY_AND_ASSIGN_(FixedValueProducer);
       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:353:5: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
       GTEST_DISALLOW_COPY_AND_ASSIGN_(FactoryValueProducer);
       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:427:3: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
     GTEST_DISALLOW_COPY_AND_ASSIGN_(ActionInterface);
     ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:687:27: error: expected parameter declarator
       GTEST_COMPILE_ASSERT_(!std::is_reference<Result>::value,
                             ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:687:27: error: expected ')'
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:687:26: note: to match this '('
       GTEST_COMPILE_ASSERT_(!std::is_reference<Result>::value,
                            ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:687:5: error: a type specifier is required for all declarations
       GTEST_COMPILE_ASSERT_(!std::is_reference<Result>::value,
       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:694:5: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
       GTEST_DISALLOW_COPY_AND_ASSIGN_(Impl);
       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:659:9: error: use of undeclared identifier 'use_ReturnRef_instead_of_Return_to_return_a_reference'
           use_ReturnRef_instead_of_Return_to_return_a_reference);
           ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:763:27: error: use of undeclared identifier 'use_Return_instead_of_ReturnRef_to_return_a_value'
                             use_Return_instead_of_ReturnRef_to_return_a_value);
                             ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:806:9: error: use of undeclared identifier 'use_Return_instead_of_ReturnRefOfCopy_to_return_a_value'
           use_Return_instead_of_ReturnRefOfCopy_to_return_a_value);
           ^
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/src/gmock-all.cc:39:
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock.h:61:
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-function-mocker.h:42:
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-spec-builders.h:75:
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:316:3: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
     GTEST_DISALLOW_COPY_AND_ASSIGN_(StringMatchResultListener);
     ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:538:7: error: use of undeclared identifier 'cannot_convert_non_reference_arg_to_reference'
         cannot_convert_non_reference_arg_to_reference);
         ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:548:7: error: use of undeclared identifier 'conversion_of_arithmetic_types_must_be_lossless'
         conversion_of_arithmetic_types_must_be_lossless);
         ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:683:25: error: use of undeclared identifier 'matcher_and_value_have_different_numbers_of_fields'
                           matcher_and_value_have_different_numbers_of_fields);
                           ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:1168:53: error: use of undeclared identifier 'AnyEq'
   class Eq2Matcher : public PairMatchBase<Eq2Matcher, AnyEq> {
                                                       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:1172:53: error: use of undeclared identifier 'AnyNe'
   class Ne2Matcher : public PairMatchBase<Ne2Matcher, AnyNe> {
                                                       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:1176:53: error: use of undeclared identifier 'AnyLt'
   class Lt2Matcher : public PairMatchBase<Lt2Matcher, AnyLt> {
                                                       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:1180:53: error: use of undeclared identifier 'AnyGt'
   class Gt2Matcher : public PairMatchBase<Gt2Matcher, AnyGt> {
                                                       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug-flight-sql/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:1184:53: error: use of undeclared identifier 'AnyLe'
   class Le2Matcher : public PairMatchBase<Le2Matcher, AnyLe> {
                                                       ^
   fatal error: too many errors emitted, stopping now [-ferror-limit=]
   20 errors generated.
   [2/611] Performing build step for 'jemalloc_ep'
   ninja: build stopped: subcommand failed.
   ```


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Middleware for handling Flight SQL Sessions including session cookie handling.
+ *
+ * Currently experimental.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+  Factory factory;
+  boolean existingSession;
+  private Session session;
+  private String closedSessionId = null;
+
+  public static final String sessionCookieName = "arrow_flight_session_id";
+
+  /**
+   * Factory for managing and accessing ServerSessionMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final Map<String, Session> sessionStore =
+        new ConcurrentHashMap<>();
+    private final Callable<String> idGenerator;
+
+    /**
+     * Construct a factory for ServerSessionMiddleware.
+     *
+     * Factory manages and accesses persistent sessions based on HTTP cookies.
+     *
+     * @param idGenerator A Callable returning unique session id Strings.
+     */
+    public Factory(Callable<String> idGenerator) {
+      this.idGenerator = idGenerator;
+    }
+
+    private synchronized Session createNewSession() {
+      String id;
+      try {
+        id = idGenerator.call();
+      } catch (Exception ignored) {
+        // Most impls aren't going to throw so don't make caller handle a nonexistent checked exception
+        return null;

Review Comment:
   Making both of these throw CallStatus.INTERNAL



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Middleware for handling Flight SQL Sessions including session cookie handling.
+ *
+ * Currently experimental.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+  Factory factory;
+  boolean existingSession;
+  private Session session;
+  private String closedSessionId = null;
+
+  public static final String sessionCookieName = "arrow_flight_session_id";
+
+  /**
+   * Factory for managing and accessing ServerSessionMiddleware.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final Map<String, Session> sessionStore =
+        new ConcurrentHashMap<>();
+    private final Callable<String> idGenerator;
+
+    /**
+     * Construct a factory for ServerSessionMiddleware.
+     *
+     * Factory manages and accesses persistent sessions based on HTTP cookies.
+     *
+     * @param idGenerator A Callable returning unique session id Strings.
+     */
+    public Factory(Callable<String> idGenerator) {
+      this.idGenerator = idGenerator;
+    }
+
+    private synchronized Session createNewSession() {
+      String id;
+      try {
+        id = idGenerator.call();
+      } catch (Exception ignored) {
+        // Most impls aren't going to throw so don't make caller handle a nonexistent checked exception
+        return null;
+      }
+      if (sessionStore.containsKey(id)) {
+        // Collision, should also never happen
+        return null;

Review Comment:
   Making both of these throw CallStatus.INTERNAL



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   My overall time for OSS is heavily limited nowadays. Just ping me and if I don't respond for a while feel free to reach out to my Apache address.
   
   You can follow the example set by other PRs for integration tests, e.g. https://github.com/apache/arrow/pull/38385 or https://github.com/apache/arrow/pull/35178


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   Also the SessionOptionValue<String[]> can't be constructed with the same generic code as all of the other variants.  (Might be possible with an instanceOf check?)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   The callback is required to avoid the called to acceptVisitor also having to be generic (in order to assign a templated type), no?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    OK = 1;
+    // The given session option name was an alias for another option name.
+    OK_MAPPED = 2;
+    // The given session option name is invalid.
+    INVALID_NAME = 3;
+    // The session option value is invalid.
+    INVALID_VALUE = 4;
+    // The session option cannot be set.
+    ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}

Review Comment:
   > should there be a corresponding `CreateSessionRequest`?
   
   This is implicit in SetSessionOptions, OR the server may go ahead and create a session for its own purposes which would then also be tied into the same session.  E.g. @kou has discussed tying AuthN into the session state as well, where SetSessionOptions calls (if any) would be subsequent to the session creation.
   
   > More to the point, should there be a default TTL on sessions (modifiable via options) for the case of a client which doesn't close their session?
   
   Seems like a server implementation detail?  The client can infer this (not robustly) by looking at the cookies emitted by the server, particularly those included in a first SetSessionOptions response.
   
   



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;

Review Comment:
   I'm inclined to do the canonical thing here which is to simply reference https://protobuf.dev/programming-guides/style/#enums in the code (again maybe in a single place as you suggest, w/ "See note <at top or similar>" in the immediate comment), so people wanting to care about why Proto does what it does can read the thing without further ado.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   > What parts would you suggest pruning @pitrou ? Perhaps around the multiple response codes for closing the session?
   
   Basically the comments that I left, but I understand that some of the complexity might be desired/necessary :-)
   
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server.h:
##########
@@ -225,6 +225,27 @@ struct ARROW_FLIGHT_SQL_EXPORT ActionCreatePreparedStatementResult {
   std::string prepared_statement_handle;
 };
 
+/// \brief A request to close the open client session.
+struct ARROW_FLIGHT_SQL_EXPORT ActionCloseSessionRequest {};

Review Comment:
   [see [above](https://github.com/apache/arrow/pull/34817#pullrequestreview-1366808831) for discussion]
   
   More specifically, the SetSessionOptions implementation would inject the Set-Cookie response header as appropriate.  Or in the alternate case of an existing session, it would be accessed via the request headers.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -423,6 +494,91 @@ arrow::Result<Result> PackActionResult(ActionCreatePreparedStatementResult resul
   return PackActionResult(pb_result);
 }
 
+arrow::Result<Result> PackActionResult(ActionSetSessionOptionsResult result) {
+  pb::sql::ActionSetSessionOptionsResult pb_result;
+  for (SetSessionOptionResult& res : result.results) {
+    switch (res) {
+      case SetSessionOptionResult::kUnspecified:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_UNSPECIFIED);
+        break;
+      case SetSessionOptionResult::kOk:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_OK);
+        break;
+      case SetSessionOptionResult::kInvalidResult:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_INVALID_VALUE);
+        break;
+      case SetSessionOptionResult::kError:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_ERROR);
+        break;
+    }
+  }
+  return PackActionResult(pb_result);
+}
+
+arrow::Result<Result> PackActionResult(ActionGetSessionOptionsResult result) {
+  pb::sql::ActionGetSessionOptionsResult pb_result;
+  for (const SessionOption& in_opt : result.session_options) {
+    pb::sql::SessionOption& opt = *pb_result.add_session_options();

Review Comment:
   * This does need to be mutable—do you have a preference for alternative code pattern to build nested Proto messages like this?
   
   * I would definitely like shared code, however I felt the best path would be to overhaul all of the Proto<->native object code and factor all of the similar switch-too-much gunk out of the packing/unpacking code and make all of that code more readable with a wholesale shift to factored-out conversion code.  Right now there isn't a common location outside of types.h and overriding Proto constructors from there is kind of ugly factoring as well.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -1072,6 +1260,18 @@ Status FlightSqlServerBase::EndTransaction(const ServerCallContext& context,
   return Status::NotImplemented("EndTransaction not implemented");
 }
 
+arrow::Result<ActionSetSessionOptionsResult> FlightSqlServerBase::SetSessionOptions (
+    const ServerCallContext& context,
+    const ActionSetSessionOptionsRequest& request) {
+  return Status::NotImplemented("SetSessionOptions not implemented");
+}

Review Comment:
   * Handler would request access to the session object from server middleware.
   * Middleware would either:
     * Already have found a valid session id cookie on call entry and set the session object for that call-local middleware instance
     * Or, if the session id cookie was not provided by the client the server session middleware would generate a new session object and corresponding ID
   * Middleware would then set the response Set-Cookie header accordingly, and return the new OR looked-up session object to the handler.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOption {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  string option_name = 1;
+  oneof option_value {
+    string string_value = 2;
+    bool bool_value = 3;
+    sfixed32 int32_value = 4;
+    sfixed64 int64_value = 5;
+    float float_value = 6;
+    double double_value = 7;

Review Comment:
   Do we need both 32 and 64 bit int, and 32 and 64 bit float? 



##########
cpp/src/arrow/flight/sql/types.h:
##########
@@ -44,6 +44,23 @@ using SqlInfoResult =
 /// \brief Map SQL info identifier to its value.
 using SqlInfoResultMap = std::unordered_map<int32_t, SqlInfoResult>;
 
+/// \brief Variant supporting all possible types for SetSessionOptions
+using SessionOptionValue =
+    std::variant<std::string, bool, int32_t, int64_t, float, double, std::vector<std::string>>;
+
+enum struct SessionOptionValueType : size_t {
+  kString, kBool, kInt32, kInt64, kFloat, kDouble, kStringList
+};
+
+struct ARROW_FLIGHT_SQL_EXPORT SessionOption {
+  std::string option_name;
+  SessionOptionValue option_value;
+
+  explicit SessionOption(std::string name, SessionOptionValue val)
+      : option_name{ std::move(name) }, option_value{ std::move(val) } {}
+  SessionOption() {}

Review Comment:
   Just `SessionOption() = default;`



##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOption {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  string option_name = 1;
+  oneof option_value {
+    string string_value = 2;
+    bool bool_value = 3;
+    sfixed32 int32_value = 4;
+    sfixed64 int64_value = 5;
+    float float_value = 6;
+    double double_value = 7;
+    StringListValue string_list_value = 8;
+  }
+}
+
+message ActionSetSessionOptionsRequest {
+  option (experimental) = true;
+
+  repeated SessionOption session_options = 1;
+}
+
+message ActionSetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum SetSessionOptionResult {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The session cannot be set to the given value.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 2;
+    // The session cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 3;
+  }
+
+  repeated SetSessionOptionResult results = 1;

Review Comment:
   This is one result per option?



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,88 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+class ServerSessionMiddlewareFactory;
+
+static constexpr char const kSessionCookieName[] =
+    "flight_sql_session_id";
+
+class FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;

Review Comment:
   Does order matter? Can we use unordered_map?



##########
cpp/src/arrow/flight/sql/client.cc:
##########
@@ -802,6 +802,157 @@ ::arrow::Result<CancelResult> FlightSqlClient::CancelQuery(
   return Status::IOError("Server returned unknown result ", result.result());
 }
 
+::arrow::Result<std::vector<SetSessionOptionResult>> FlightSqlClient::SetSessionOptions(
+    const FlightCallOptions& options,
+    const std::vector<SessionOption>& session_options) {
+  flight_sql_pb::ActionSetSessionOptionsRequest request;
+  for (const SessionOption& in_opt : session_options) {
+    flight_sql_pb::SessionOption* opt = request.add_session_options();
+    const std::string& name = in_opt.option_name;
+    opt->set_option_name(name);
+
+    const SessionOptionValue& value = in_opt.option_value;
+    if (value.index() == std::variant_npos)
+      return Status::Invalid("Undefined SessionOptionValue type ");
+    switch (static_cast<SessionOptionValueType>(value.index())) {

Review Comment:
   I think the way you're supposed to do this is `std::visit`, and then we don't need the extra enum.



##########
format/FlightSql.proto:
##########


Review Comment:
   FWIW, for other more recent proposals for features like this we've decided to make them usable from base Arrow Flight as well. I think it's also reasonable to do the same here. (There wouldn't be handlers added on the base Flight server - you would have to look for and handle the action yourself - but there were handlers added to the Flight SQL server.)



##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   To circle back here, expectations should be clearly documented in the Protobuf. So here, it should be explicitly stated that the client and server are using a shared header to store the session state, and that the header should be the HTTP Cookie/Set-Cookie header.



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,88 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+class ServerSessionMiddlewareFactory;
+
+static constexpr char const kSessionCookieName[] =
+    "flight_sql_session_id";
+
+class FlightSqlSession {
+ protected:

Review Comment:
   Why protected?



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,88 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+class ServerSessionMiddlewareFactory;
+
+static constexpr char const kSessionCookieName[] =
+    "flight_sql_session_id";
+
+class FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+ public:
+  /// \brief Get session option by key
+  SessionOptionValue GetSessionOption(const std::string&);
+  /// \brief Set session option by key to given value
+  void SetSessionOption(const std::string&, const SessionOptionValue&);

Review Comment:
   nit: prefer string_view over const string&?



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(

Review Comment:
   Can we do this without duplicating the base cookie middleware logic?
   
   If that means that we don't ship the middleware, but only implement it in the unit/integration tests, that might be reasonable, so long as it's clear how servers/clients are expected to implement this (what headers to send/receive, etc.)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOption {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  string option_name = 1;
+  oneof option_value {
+    string string_value = 2;
+    bool bool_value = 3;
+    sfixed32 int32_value = 4;
+    sfixed64 int64_value = 5;
+    float float_value = 6;
+    double double_value = 7;
+    StringListValue string_list_value = 8;
+  }
+}
+
+message ActionSetSessionOptionsRequest {
+  option (experimental) = true;
+
+  repeated SessionOption session_options = 1;
+}
+
+message ActionSetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum SetSessionOptionResult {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The session cannot be set to the given value.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 2;
+    // The session cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 3;
+  }
+
+  repeated SetSessionOptionResult results = 1;

Review Comment:
   Correct.  With the refactor (in progress) to use a Protobuf map to send/set (and get) these will be keyed instead of having matching ordinals which IMO provides a much more intuitively usable result to the client here as well.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
cpp/src/arrow/flight/sql/types.h:
##########


Review Comment:
   Basically, just copy the way Kou handled things here: https://github.com/apache/arrow/pull/36009



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   Sorry I was off on vacation, just grinding through the refactor around the new serialization stuff.  Had to stop off and learn some C++17 I didn't know I didn't know.
   
   Random thought:  Might be useful to have a virtual superclass (probably just 1 not 2 for request/response) for all of the request/response messages, which would also eliminate the duplication of the PackActionResult functions, or even have a non-virtual .SerializeToBuffer() in the superclass and skip PackActionResult entirely :)  I'm sure you're swimming in free time, just figured this is a simplifying thing while the serialization is being neatly factored out of the way.
   
   Will push and/or update you in the next few days once I've finished the refactor of my change around the serialization stuff.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;

Review Comment:
   Minor optimization - if you made cookies non-const and cookie non-const, you could move-assign session_id here to avoid the 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/serialization_internal.cc:
##########
@@ -372,6 +372,174 @@ Status ToPayload(const FlightDescriptor& descr, std::shared_ptr<Buffer>* out) {
   return Status::OK();
 }
 
+// SessionOptionValue
+
+Status FromProto(const pb::SessionOptionValue& pb_val
+                                     SessionOptionValue* val) {
+  switch (pb_opt_val.option_value_case()) {
+    case pb::SessionOptionValue::OPTION_VALUE_NOT_SET:
+      return Status::Invalid("Unset option_value for name '" +
+                             pb_opt_name + "'");
+    case pb::SessionOptionValue::kStringValue:
+      val = pb_opt_val.string_value();
+      break;
+    case pb::SessionOptionValue::kBoolValue:
+      val = pb_opt_val.bool_value();
+      break;
+    case pb::SessionOptionValue::kInt32Value:
+      val = pb_opt_val.int32_value();
+      break;
+    case pb::SessionOptionValue::kInt64Value:
+      val = pb_opt_val.int64_value();
+      break;
+    case pb::SessionOptionValue::kFloatValue:
+      val = pb_opt_val.float_value();
+      break;
+    case pb::SessionOptionValue::kDoubleValue:
+      val = pb_opt_val.double_value();
+      break;
+    case pb::SessionOptionValue::kStringListValue:
+      val.emplace<std::vector<std::string>>();
+      std::get<std::vector<std::string>>(val)
+          .reserve(pb_opt_val.string_list_value().values_size());
+      for (const std::string& s : pb_opt_val.string_list_value().values())
+        std::get<std::vector<std::string>>(val).push_back(s);
+      break;
+  }
+  return Status::OK();
+}
+
+Status ToProto(const SessionOptionValue& val
+                                       pb::SessionOptionValue pb_val) {
+  std::visit(overloaded{
+      [&](std::string v) { pb_val.set_string_value(v); },
+      [&](bool v) { pb_val.set_bool_value(v); },
+      [&](int32_t v) { pb_val.set_int32_value(v); },
+      [&](int64_t v) { pb_val.set_int64_value(v); },
+      [&](float v) { pb_val.set_float_value(v); },
+      [&](double v) { pb_val.set_double_value(v); },
+      [&](std::vector<std::string> v) {
+        auto* string_list_value = pb_val.mutable_string_list_value();
+        for (const std::string& s : v)
+          string_list_value->add_values(s);
+      }
+    }, opt_value);
+  return Status::OK();
+}
+
+// map<string, SessionOptionValue>
+
+Status FromProto(const google::protobuf::map<string,

Review Comment:
   This is passing in a copy of map.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.cc:
##########
@@ -463,6 +463,300 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+// Helper for stringifying maps containing various types
+template <typename T>
+ostream& operator<<(std::map<std::string, T>) {
+  std::stringstream ss;
+
+  ss << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : session_options) {
+    std::cout << sep << '[' << k << "]: '" << v << "', ";  // PHOXME v.ToString() not implemented yet
+    sep = ", ";
+  }
+  ss << '}';
+
+  return ss.str();
+}
+static bool CompareSessionOptionMaps(

Review Comment:
   (I believe I had an extraneous .session_options there from factoring the code out of another location)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/types.h:
##########
@@ -44,6 +44,23 @@ using SqlInfoResult =
 /// \brief Map SQL info identifier to its value.
 using SqlInfoResultMap = std::unordered_map<int32_t, SqlInfoResult>;
 
+/// \brief Variant supporting all possible types for SetSessionOptions
+using SessionOptionValue =
+    std::variant<std::string, bool, int32_t, int64_t, float, double, std::vector<std::string>>;
+
+enum struct SessionOptionValueType : size_t {
+  kString, kBool, kInt32, kInt64, kFloat, kDouble, kStringList
+};
+
+struct ARROW_FLIGHT_SQL_EXPORT SessionOption {
+  std::string option_name;
+  SessionOptionValue option_value;
+
+  explicit SessionOption(std::string name, SessionOptionValue val)
+      : option_name{ std::move(name) }, option_value{ std::move(val) } {}
+  SessionOption() {}

Review Comment:
   (Code removed)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;
+    }
+
+    if (!session_id.length()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers,
+                                        session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid(
+            "Invalid or expired "
+            + static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session,
+    std::string session_id)
+    : factory_(factory), headers_(headers), session_(session), existing_session(true) {}
+
+void ServerSessionMiddleware::SendingHeaders(AddCallHeaders* addCallHeaders) {
+  if (!existing_session && session_) {
+    addCallHeaders->AddHeader(
+        "set-cookie",
+        static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+  }
+}
+
+void ServerSessionMiddleware::CallCompleted(const Status&) {}
+
+bool ServerSessionMiddleware::HasSession() const {
+  return static_cast<bool>(session_);
+}
+std::shared_ptr<FlightSqlSession> ServerSessionMiddleware::GetSession() {
+  if (!session_)
+    session_ = factory_->GetNewSession(&session_id_);
+  return session_;
+}
+const CallHeaders& ServerSessionMiddleware::GetCallHeaders() const {
+  return headers_;
+}
+
+
+
+std::shared_ptr<ServerMiddlewareFactory> MakeServerSessionMiddlewareFactory() {
+  return std::shared_ptr<ServerSessionMiddlewareFactory>(
+      new ServerSessionMiddlewareFactory());
+}
+
+SessionOptionValue FlightSqlSession::GetSessionOption(const std::string_view k) {
+  const std::shared_lock<std::shared_mutex> l(map_lock_);
+  return map_.at(k);
+}
+void FlightSqlSession::SetSessionOption(const std::string_view, const SessionOptionValue& v) {
+  const std::unique_lock<std::shared_mutex> l(map_lock_);
+  map_[k] = v;
+}
+void FlightSqlSession::EraseSessionOption(const std::string_view) {
+  const std::unique_lock<std::shared_mutex> l(map_lock_);
+  map_.erase(k);

Review Comment:
   >   /// \brief Idempotently remove key from this call's Session, if Session & key exist
   
   NOOP.  Good to check this was sane; all good.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;
+    }
+
+    if (!session_id.length()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers,
+                                        session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid(
+            "Invalid or expired "
+            + static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;

Review Comment:
   Done.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {

Review Comment:
   If you're worried about concurrency, use a lock.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   I also kicked off CI so you can see what happens here. I think an R failure means you need to rebase/merge with main


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,77 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <functional>
+#include <optional>
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+static constexpr char const kSessionCookieName[] = "arrow_flight_session_id";
+
+class FlightSqlSession {

Review Comment:
   Done.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");
+  for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+    const std::string_view& cookie_header = itr->second;
+    const std::vector<std::pair<std::string, std::string>> cookies =
+        ParseCookieString(cookie_header);
+    for (const std::pair<std::string, std::string>& cookie : cookies) {
+      if (cookie.first == kSessionCookieName) {
+        if (cookie.second.empty())
+          return Status::Invalid("Empty ", kSessionCookieName, " cookie value.");
+        session_id = std::move(cookie.second);
+      }
+    }
+    if (!session_id.empty()) break;
+  }
+
+  if (session_id.empty()) {
+    // No cookie was found
+    *middleware = std::make_shared<ServerSessionMiddlewareImpl>(this, incoming_headers);
+  } else {
+    const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+    if (auto it = session_store_.find(session_id); it == session_store_.end()) {
+      return Status::Invalid("Invalid or expired ", kSessionCookieName, " cookie.");
+    } else {
+      auto session = it->second;
+      *middleware = std::make_shared<ServerSessionMiddlewareImpl>(
+          this, incoming_headers, std::move(session), session_id);
+    }
+  }
+
+  return Status::OK();
+}
+
+/// \brief Get a new, empty session option map and its id key.
+std::pair<std::string, std::shared_ptr<FlightSqlSession>>
+ServerSessionMiddlewareFactory::GetNewSession() {
+  std::string new_id = id_generator_();
+  auto session = std::make_shared<FlightSqlSession>();
+
+  const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+  session_store_[new_id] = session;

Review Comment:
   Probably.  Collision prevention is a little beyond the scope of this helper, but at the very least it's a consideration that should be documented.  Multiple retries etc. (potentially) is a bit of a rabbit hole but ultimately a valid concern.  If you think something like 10 retries or whatever is a useful default behaviour here that could certainly be done, but probably as a followup task at this point.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }

Review Comment:
   I discussed this [fairly extensively](https://github.com/apache/arrow/pull/34817/files#r1245588032) with @lidavidm and the existing cookie functionality is heavily intertwined with managing the client-side cookie jar, so essentially we would need to factor the zero-copy stateful parser out as a generator which is getting pretty involved and could accidentally break the existing code.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> CreateNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex mutex_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session_;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session_(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session_(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session_ && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::lock_guard<std::shared_mutex> l(mutex_);
+    if (!session_) {
+      auto [id, s] = factory_->CreateNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const auto pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");
+  for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+    const std::string_view& cookie_header = itr->second;
+    const std::vector<std::pair<std::string, std::string>> cookies =
+        ParseCookieString(cookie_header);
+    for (const std::pair<std::string, std::string>& cookie : cookies) {
+      if (cookie.first == kSessionCookieName) {
+        if (cookie.second.empty())
+          return Status::Invalid("Empty ", kSessionCookieName, " cookie value.");
+        session_id = std::move(cookie.second);
+      }
+    }
+    if (!session_id.empty()) break;
+  }
+
+  if (session_id.empty()) {
+    // No cookie was found
+    *middleware = std::make_shared<ServerSessionMiddlewareImpl>(this, incoming_headers);
+  } else {
+    const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+    if (auto it = session_store_.find(session_id); it == session_store_.end()) {
+      return Status::Invalid("Invalid or expired ", kSessionCookieName, " cookie.");
+    } else {
+      auto session = it->second;
+      *middleware = std::make_shared<ServerSessionMiddlewareImpl>(
+          this, incoming_headers, std::move(session), session_id);
+    }
+  }
+
+  return Status::OK();
+}
+
+/// \brief Get a new, empty session option map and its id key.
+std::pair<std::string, std::shared_ptr<FlightSqlSession>>
+ServerSessionMiddlewareFactory::CreateNewSession() {
+  auto new_id = id_generator_();

Review Comment:
   Regarding both this and the other thread (ha) about this, probably worth erroring on collision (don't bother with a retry strategy) and a single SetSO call can fail once (ugly, but the generator's fault) instead of corrupting existing server session state (worse).



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/client.cc:
##########
@@ -793,6 +800,149 @@ ::arrow::Result<CancelResult> FlightSqlClient::CancelQuery(
   return Status::IOError("Server returned unknown result ", result.result());
 }
 
+::arrow::Result<std::map<std::string, SetSessionOptionResult>>
+FlightSqlClient::SetSessionOptions(
+    const FlightCallOptions& options,
+    const std::map<std::string, SessionOptionValue>& session_options) {
+  pb::ActionSetSessionOptionsRequest request;
+  auto* options_map = request.mutable_session_options();
+
+  for (const auto & [name, opt_value] : session_options) {
+    pb::SessionOptionValue pb_opt_value;
+
+    if (opt_value.index() == std::variant_npos)
+      return Status::Invalid("Undefined SessionOptionValue type ");
+
+    std::visit(overloaded{
+      // TODO move this somewhere common that can have Proto-involved code
+      [&](std::string v) { pb_opt_value.set_string_value(v); },
+      [&](bool v) { pb_opt_value.set_bool_value(v); },
+      [&](int32_t v) { pb_opt_value.set_int32_value(v); },
+      [&](int64_t v) { pb_opt_value.set_int64_value(v); },
+      [&](float v) { pb_opt_value.set_float_value(v); },
+      [&](double v) { pb_opt_value.set_double_value(v); },
+      [&](std::vector<std::string> v) {
+        auto* string_list_value = pb_opt_value.mutable_string_list_value();
+        for (const std::string& s : v)

Review Comment:
   Ok:
   
   https://github.com/indigophox/arrow/blob/b8c12abcf57e470307e93b03d5ccf2ce21dc42e5/cpp/src/arrow/flight/serialization_internal.cc#L429-L433



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }

Review Comment:
   ```
   $ cmake --build .
   [19/400] Building CXX object _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o
   FAILED: _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o 
   /opt/homebrew/bin/ccache /Library/Developer/CommandLineTools/usr/bin/c++ -DARROW_EXTRA_ERROR_CONTEXT -DARROW_HAVE_NEON -DARROW_WITH_TIMING_TESTS -DGTEST_CREATE_SHARED_LIBRARY=1 -Dgmock_EXPORTS -I/Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include -I/Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock -isystem /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googletest/include -isystem /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googletest -Qunused-arguments -fcolor-diagnostics -Wno-unused-value -Wno-ignored-attributes -g -Werror -O0 -ggdb  -Wno-error -arch arm64 -isysroot /Library/Developer/CommandLineTools/SDKs/MacOSX14.0.sdk -fPIC -Wall -Wshadow -Werror -Wconversion -DGTEST_HAS_PTHREAD=1 -fexceptions -W -Wpointer-arith -Wreturn-type -Wcast-qual -Wwrite-strings -Wswitch -Wunused-parameter -Wcast-align -Wchar-subscripts -Winline -Wredu
 ndant-decls -std=c++17 -MD -MT _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o -MF _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o.d -o _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o -c /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/src/gmock-all.cc
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/src/gmock-all.cc:39:
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock.h:59:
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:342:5: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
       GTEST_DISALLOW_COPY_AND_ASSIGN_(FixedValueProducer);
       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:353:5: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
       GTEST_DISALLOW_COPY_AND_ASSIGN_(FactoryValueProducer);
       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:427:3: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
     GTEST_DISALLOW_COPY_AND_ASSIGN_(ActionInterface);
     ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:687:27: error: expected parameter declarator
       GTEST_COMPILE_ASSERT_(!std::is_reference<Result>::value,
                             ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:687:27: error: expected ')'
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:687:26: note: to match this '('
       GTEST_COMPILE_ASSERT_(!std::is_reference<Result>::value,
                            ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:687:5: error: a type specifier is required for all declarations
       GTEST_COMPILE_ASSERT_(!std::is_reference<Result>::value,
       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:694:5: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
       GTEST_DISALLOW_COPY_AND_ASSIGN_(Impl);
       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:659:9: error: use of undeclared identifier 'use_ReturnRef_instead_of_Return_to_return_a_reference'
           use_ReturnRef_instead_of_Return_to_return_a_reference);
           ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:763:27: error: use of undeclared identifier 'use_Return_instead_of_ReturnRef_to_return_a_value'
                             use_Return_instead_of_ReturnRef_to_return_a_value);
                             ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-actions.h:806:9: error: use of undeclared identifier 'use_Return_instead_of_ReturnRefOfCopy_to_return_a_value'
           use_Return_instead_of_ReturnRefOfCopy_to_return_a_value);
           ^
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/src/gmock-all.cc:39:
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock.h:61:
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-function-mocker.h:42:
   In file included from /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-spec-builders.h:75:
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:316:3: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
     GTEST_DISALLOW_COPY_AND_ASSIGN_(StringMatchResultListener);
     ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:538:7: error: use of undeclared identifier 'cannot_convert_non_reference_arg_to_reference'
         cannot_convert_non_reference_arg_to_reference);
         ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:548:7: error: use of undeclared identifier 'conversion_of_arithmetic_types_must_be_lossless'
         conversion_of_arithmetic_types_must_be_lossless);
         ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:683:25: error: use of undeclared identifier 'matcher_and_value_have_different_numbers_of_fields'
                           matcher_and_value_have_different_numbers_of_fields);
                           ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:2490:5: error: unknown type name 'GTEST_DISALLOW_COPY_AND_ASSIGN_'
       GTEST_DISALLOW_COPY_AND_ASSIGN_(Impl);
       ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:2505:7: error: expected parameter declarator
         !IsHashTable<GTEST_REMOVE_REFERENCE_AND_CONST_(RhsContainer)>::value,
         ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:2505:7: error: expected ')'
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:2504:24: note: to match this '('
     GTEST_COMPILE_ASSERT_(
                          ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:2504:3: error: a type specifier is required for all declarations
     GTEST_COMPILE_ASSERT_(
     ^
   /Users/pauln/git/arrow/cpp/session-management-build-debug/_deps/googletest-src/googlemock/include/gmock/gmock-matchers.h:2527:9: error: use of undeclared identifier 'use_UnorderedPointwise_with_hash_tables'
           use_UnorderedPointwise_with_hash_tables);
           ^
   fatal error: too many errors emitted, stopping now [-ferror-limit=]
   20 errors generated.
   [30/400] Building CXX object _deps/googletest-build/googletest/CMakeFiles/gtest.dir/src/gtest-all.cc.o
   ninja: build stopped: subcommand failed.
   ```



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;

Review Comment:
   Hmm, I think it would be nice to make the common case (all-success) easy to handle. I expect per-error inspection to be overkill for most clients, especially those that are only meant to talk to one server type.
   
   So perhaps instead something like this:
   ```proto
     enum OkCode {
       SET_SESSION_OK_UNSPECIFIED = 0;
       SET_SESSION_OK = 1;
       SET_SESSION_OK_MAPPED = 1;
     }
     enum ErrorCode {
       SET_SESSION_ERROR_UNSPECIFIED = 0;
       SET_SESSION_ERROR_INVALID_NAME = 1;
       SET_SESSION_ERROR_INVALID_VALUE = 2;
       SET_SESSION_ERROR_CANNOT_SET = 3;
     }
     message Ok {
       string option_name;
       OkCode code;
     }
     message Error {
       string option_name;
       ErrorCode code;
     }
     // Options which were successfully set
     repeated Ok oks = 1;
     // Options which failed setting
     repeated Error errors = 2;
   ```
   
   (I still think this is overkill, but at least a client would trivially ignore successful options)
   



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;

Review Comment:
   > Would it be reasonable for the "default" to be success, and just _only_ report failures/errors, and the client can assume any option they set which is not reported in the response was successful?
   
   Unfortunately as @lidavidm discussed elsewhere the default/0 enum value is used by Proto3 (in some languages—I believe C++ and not Java?) as a fallback for unrecognized packed values, so we cannot use it as a meaningful value.  It would certainly be nice to check the enum is <= OK_MAPPED or whatnot, rather than a double bounds check, but we can't...



##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;

Review Comment:
   > Would it be reasonable for the "default" to be success, and just _only_ report failures/errors, and the client can assume any option they set which is not reported in the response was successful?
   
   Unfortunately as @lidavidm discussed elsewhere the default/0 enum value is used by Proto3 (in some languages—I believe C++ and not Java?) as a fallback for unrecognized packed values, so we cannot use it as a meaningful value.  It would certainly be nice to check the enum is <= OK_MAPPED or whatnot, rather than a double bounds check, but we can't...



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;

Review Comment:
   > Would it be reasonable for the "default" to be success, and just _only_ report failures/errors, and the client can assume any option they set which is not reported in the response was successful?
   
   I would presumably have to eliminate the OK_MAPPED status, but yes this would keep the parsing consistent with the request structure AND eliminate a lot of work for the client by reducing the MVP to checking the length of the response map.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   > I think if I proposed this my rationale was long lived distributed operations can typically be long lived or hard to interrupt immediately. IIUC sessions seems like they unlikely to have distributed work associated with them (unless we are hoping to do things like unclose a session?)
   
   If for any reason someone uses sessions to persist client state to to doGet, or for AuthN persistence, etc., or uses them in a distributed context in general then closing them is also possibly a distributed operation depending how they're backed.  If they're used for AuthN in particular (as @kou has suggested doing for his purposes) a client/user may want to confirm their (distributed) invalidation.  Also not very expensive to have a couple if enum values in-case here.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    OK = 1;
+    // The given session option name was an alias for another option name.
+    OK_MAPPED = 2;
+    // The given session option name is invalid.
+    INVALID_NAME = 3;
+    // The session option value is invalid.
+    INVALID_VALUE = 4;
+    // The session option cannot be set.
+    ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}

Review Comment:
   should there be a corresponding `CreateSessionRequest`?
   
   More to the point, should there be a default TTL on sessions (modifiable via options) for the case of a client which doesn't close their session?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
cpp/src/arrow/flight/sql/protocol_internal.h:
##########
@@ -24,3 +24,4 @@
 #include "arrow/flight/sql/visibility.h"
 
 #include "arrow/flight/sql/FlightSql.pb.h"  // IWYU pragma: export
+#include "arrow/flight/Flight.pb.h"

Review Comment:
   Not super familiar with how the Windows linkage works here, do you mean:
   
   * Façade(?) (presumable with an MVP set of functionality passed through from the Proto types) in libarrow_flight, i.e. in... flight/protocol_internal.h or another (new?) module?
   * Including the base Flight protocol_internal.h in sql/protocol_internal.h (to expose the symbols for dynamic import)?
   * And I assume no changes required to add a dynamic dependency of libarrow_flight on libarrow_flight_sql (which already exists?)
   
   TIA!



##########
cpp/src/arrow/flight/sql/protocol_internal.h:
##########
@@ -24,3 +24,4 @@
 #include "arrow/flight/sql/visibility.h"
 
 #include "arrow/flight/sql/FlightSql.pb.h"  // IWYU pragma: export
+#include "arrow/flight/Flight.pb.h"

Review Comment:
   Not super familiar with how the Windows linkage works here, do you mean:
   
   * Façade(?) (presumable with an MVP set of functionality passed through from the Proto types) in libarrow_flight, i.e. in... flight/protocol_internal.h or another (new?) module?
   * Including the base Flight protocol_internal.h in sql/protocol_internal.h (to expose the symbols for dynamic import)?
   * And I assume no changes required to add a dynamic dependency of libarrow_flight on libarrow_flight_sql (which already exists?)
   
   TIA!



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
cpp/src/arrow/flight/sql/types.h:
##########


Review Comment:
   Can you point me to example(s)?  Trying to also see (related to the other thread about linkage) how this ties back into a DoAction handler touching Proto data at some point (but possibly not Proto defs, so passing the opaque packed Proto back to a handler in libarrow_flight, only for types belonging to Flight rather than to fSQL?)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   Thanks for the suggestions.
   I haven't try this yet but can I reuse `ServerSessionMiddleware` for my case? Or should I implement a similar middleware from scratch?
   
   > I think if you put any authentication middleware first, then it can reject unauthenticated requests and you can always create a session after that.
   
   Sorry. I was wrong. I need to create a new session for authentication. Because I need to create a separated process to use PostgreSQL's API. Flight SQL is mutlithreading but PostgreSQL's API isn't multithread safe. So I need to use a single thread separated process to use PostgreSQL's API. If the authentication is failed, the session is closed immediately.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.cc:
##########
@@ -468,6 +468,320 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {
+  os << CloseSessionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+// Helpers for stringifying maps containing various types
+std::ostream& operator<<(std::ostream& os, std::vector<std::string> v) {
+  os << '[';
+  std::string sep = "";
+  for (const auto& x : v) {
+    os << sep << '"' << x << '"';
+    sep = ", ";
+  }
+  os << ']';
+
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SessionOptionValue& v) {
+  std::visit([&](const auto& x) { os << x; }, v);
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionsResult::Result& r) {
+  os << '{' << r.status << '}';
+  return os;
+}
+
+template <typename T>
+std::ostream& operator<<(std::ostream& os, std::map<std::string, T> m) {
+  os << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : m) {
+    os << sep << '[' << k << "]: '" << v;
+    sep = ", ";
+  }
+  os << '}';
+
+  return os;
+}
+
+namespace {
+static bool CompareSessionOptionMaps(const std::map<std::string, SessionOptionValue>& a,
+                                     const std::map<std::string, SessionOptionValue>& b) {
+  if (a.size() != b.size()) {
+    return false;
+  }
+  for (const auto& [k, v] : a) {
+    if (const auto it = b.find(k); it == b.end()) {
+      return false;
+    } else {
+      const auto& b_v = it->second;
+      if (v.index() != b_v.index()) {
+        return false;
+      }
+      if (v != b_v) {
+        return false;
+      }
+    }
+  }
+  return true;
+}
+}  // namespace
+
+// SetSessionOptionsRequest
+
+std::string SetSessionOptionsRequest::ToString() const {
+  std::stringstream ss;
+
+  ss << "<SetSessionOptionsRequest session_options=" << session_options << '>';
+
+  return ss.str();
+}
+
+bool SetSessionOptionsRequest::Equals(const SetSessionOptionsRequest& other) const {
+  return CompareSessionOptionMaps(session_options, other.session_options);
+}
+
+arrow::Result<std::string> SetSessionOptionsRequest::SerializeToString() const {
+  pb::SetSessionOptionsRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized SetSessionOptionsRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<SetSessionOptionsRequest> SetSessionOptionsRequest::Deserialize(
+    std::string_view serialized) {
+  // TODO these & SerializeToString should all be factored out to a superclass
+  pb::SetSessionOptionsRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized SetSessionOptionsRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid SetSessionOptionsRequest");
+  }
+  SetSessionOptionsRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// SetSessionOptionsResult
+
+std::string SetSessionOptionsResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<SetSessionOptionsResult results=" << results << '>';
+
+  return ss.str();
+}
+
+bool SetSessionOptionsResult::Equals(const SetSessionOptionsResult& other) const {
+  if (results != other.results) {
+    return false;
+  }
+  return true;
+}
+
+arrow::Result<std::string> SetSessionOptionsResult::SerializeToString() const {
+  pb::SetSessionOptionsResult pb_result;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_result));
+
+  std::string out;
+  if (!pb_result.SerializeToString(&out)) {
+    return Status::IOError("Serialized SetSessionOptionsResult exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<SetSessionOptionsResult> SetSessionOptionsResult::Deserialize(
+    std::string_view serialized) {
+  pb::SetSessionOptionsResult pb_result;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized SetSessionOptionsResult size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_result.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid SetSessionOptionsResult");
+  }
+  SetSessionOptionsResult out;
+  RETURN_NOT_OK(internal::FromProto(pb_result, &out));
+  return out;
+}
+
+// GetSessionOptionsRequest
+
+std::string GetSessionOptionsRequest::ToString() const {
+  return "<GetSessionOptionsRequest>";
+}
+
+bool GetSessionOptionsRequest::Equals(const GetSessionOptionsRequest& other) const {
+  return true;
+}
+
+arrow::Result<std::string> GetSessionOptionsRequest::SerializeToString() const {
+  pb::GetSessionOptionsRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized GetSessionOptionsRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<GetSessionOptionsRequest> GetSessionOptionsRequest::Deserialize(
+    std::string_view serialized) {
+  pb::GetSessionOptionsRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized GetSessionOptionsRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid GetSessionOptionsRequest");
+  }
+  GetSessionOptionsRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// GetSessionOptionsResult
+
+std::string GetSessionOptionsResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<GetSessionOptionsResult session_options=" << session_options << '>';
+
+  return ss.str();
+}
+
+bool GetSessionOptionsResult::Equals(const GetSessionOptionsResult& other) const {
+  return CompareSessionOptionMaps(session_options, other.session_options);
+}
+
+arrow::Result<std::string> GetSessionOptionsResult::SerializeToString() const {
+  pb::GetSessionOptionsResult pb_result;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_result));
+
+  std::string out;
+  if (!pb_result.SerializeToString(&out)) {
+    return Status::IOError("Serialized GetSessionOptionsResult exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<GetSessionOptionsResult> GetSessionOptionsResult::Deserialize(
+    std::string_view serialized) {
+  pb::GetSessionOptionsResult pb_result;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized GetSessionOptionsResult size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_result.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid GetSessionOptionsResult");
+  }
+  GetSessionOptionsResult out;
+  RETURN_NOT_OK(internal::FromProto(pb_result, &out));
+  return out;
+}
+
+// CloseSessionRequest
+
+std::string CloseSessionRequest::ToString() const { return "<CloseSessionRequest>"; }
+
+bool CloseSessionRequest::Equals(const CloseSessionRequest& other) const { return true; }
+
+arrow::Result<std::string> CloseSessionRequest::SerializeToString() const {
+  pb::CloseSessionRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized CloseSessionRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<CloseSessionRequest> CloseSessionRequest::Deserialize(
+    std::string_view serialized) {
+  pb::CloseSessionRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid("Serialized CloseSessionRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid CloseSessionRequest");
+  }
+  CloseSessionRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// CloseSessionResult
+
+std::string CloseSessionResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<CloseSessionResult result=" << status << '>';
+
+  return ss.str();
+}
+
+bool CloseSessionResult::Equals(const CloseSessionResult& other) const {
+  if (status != other.status) {
+    return false;
+  }
+  return true;

Review Comment:
   This message may get extended in future (with human-readable text which may or may not get compared(!)) but fair enough for now.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;

Review Comment:
   In any case, it seems weird to specify a default value that we explicit discourage servers from sending...



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.

Review Comment:
   Looking at UCX, its docs are particularly sparse.  The documentation here can be reworded:  Does UCX have its own "State Management Mechanism" as it were, to persist such things across its individual UCP "calls", which we could generalize the wording here to capture?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValueFactory.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.SessionOptionValue;
+
+public class SessionOptionValueFactory {
+    public static SessionOptionValue makeSessionOption(String value) {
+        return new SessionOptionValueString(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(bool value) {
+        return new SessionOptionValueBool(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(int value) {
+        return new SessionOptionValueInt(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(long value) {
+        return new SessionOptionValueLong(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(float value) {
+        return new SessionOptionValueFloat(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(double value) {
+        return new SessionOptionValueDouble(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(String[] value) {
+        return new SessionOptionValueStringList(value);
+    }
+}
+
+class SessionOptionValueString {

Review Comment:
   Adding to the oneof does lead to the weird case where the server can send an 'unset' value. Do we want to allow that? If so, then this sounds fine to me. For `SessionOptionValue`, I would rather use [`std::monostate`](https://stackoverflow.com/questions/53126513/what-should-i-use-instead-of-void-as-one-of-the-alternative-types-in-an-variant) or similar to indicate the unset 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   That gets us back to needing middleware to access the session store, which just moves the problem around in order to solve it... I'm thinking about doing the other solution very locally, but the header handling needs to be triggered from within the handler itself (not even DoAction() which can't intercede at the right point), so I will need to add the stream interposer to defaut-trigger the header handling on stream onNext & onError & onCompleted for compatibility with existing implementations... objections?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   @lidavidm mostly unrelated but also, I'm not seeing how I get a CallStatus.NOT_FOUND equivalent in C++, am I missing something?  (TIA and sorry for the additional bother)


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;

Review Comment:
   Done in Proto & C++, WIP in Java.  Thanks @zeroshade @pitrou 



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The cancellation status is unknown. Servers should avoid using

Review Comment:
   s/cancellation/close?



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -430,6 +430,34 @@ std::string Location::scheme() const {
   return scheme;
 }
 
+std::string Location::path() const { return uri_->path(); }
+arrow::Result<std::vector<std::pair<std::string, std::string>>> Location::query_items()
+    const {
+  return uri_->query_items();
+}
+
+arrow::Result<std::vector<std::pair<std::string, std::string>>> Location::as_headers()

Review Comment:
   This is far, far too specific to Flight SQL to belong as a method



##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   Is the assumption for all of these that the session/client identifier will be transmitted out-of-band (via headers, etc.)?



##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The cancellation status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOption {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  string option_name = 1;
+  oneof option_value {
+    string string_value = 2;
+    bool bool_value = 3;
+    sfixed32 int32_value = 4;

Review Comment:
   Why sfixed32 instead of just int32?



##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -359,6 +370,66 @@ arrow::Result<ActionEndTransactionRequest> ParseActionEndTransactionRequest(
   return result;
 }
 
+arrow::Result<ActionSetSessionOptionsRequest> ParseActionSetSessionOptionsRequest(
+    const google::protobuf::Any& any) {
+  pb::sql::ActionSetSessionOptionsRequest command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack ActionSetSessionOptionsRequest");
+  }
+
+  ActionSetSessionOptionsRequest result;
+  if (command.session_options_size() > 0) {
+    result.session_options.reserve(command.session_options_size());
+    for (const pb::sql::SessionOption& in_opt : command.session_options()) {
+      const std::string& name = in_opt.option_name();
+      SessionOption opt;
+      switch (in_opt.option_value_case()) {
+        case pb::sql::SessionOption::OPTION_VALUE_NOT_SET:
+          return Status::Invalid("Unset SessionOptionValue for name '" + name + "'");
+        case pb::sql::SessionOption::kStringValue:
+          opt = {name, in_opt.string_value()};
+          break;
+        case pb::sql::SessionOption::kBoolValue:
+          opt = {name, in_opt.bool_value()};
+          break;
+        case pb::sql::SessionOption::kInt32Value:
+          opt = {name, in_opt.int32_value()};
+          break;
+        case pb::sql::SessionOption::kInt64Value:
+          opt = {name, in_opt.int64_value()};
+          break;
+        case pb::sql::SessionOption::kFloatValue:
+          opt = {name, in_opt.float_value()};
+          break;
+        case pb::sql::SessionOption::kDoubleValue:
+          opt = {name, in_opt.double_value()};
+          break;
+        case pb::sql::SessionOption::kStringListValue:
+          std::vector<std::string> vlist;
+          vlist.reserve(in_opt.string_list_value().values_size());
+          for (const std::string& s : in_opt.string_list_value().values())
+            vlist.push_back(s);
+          opt = {name, vlist};
+          break;
+      }
+      result.session_options.push_back(opt);

Review Comment:
   std::move



##########
cpp/src/arrow/flight/sql/client.h:
##########
@@ -329,6 +329,25 @@ class ARROW_FLIGHT_SQL_EXPORT FlightSqlClient {
   /// \param[in] info         The FlightInfo of the query to cancel.
   ::arrow::Result<CancelResult> CancelQuery(const FlightCallOptions& options,
                                             const FlightInfo& info);
+   
+  /// \brief Sets session options.
+  ///
+  /// \param[in] options            RPC-layer hints for this call.
+  /// \param[in] session_options    The session options to set.
+  ::arrow::Result<std::vector<SetSessionOptionResult>> SetSessionOptions(

Review Comment:
   It may be good to make the session identifier explicit so that clients can use it across individual connections? Also to avoid implicit state.



##########
cpp/src/arrow/flight/sql/client.cc:
##########
@@ -802,6 +802,157 @@ ::arrow::Result<CancelResult> FlightSqlClient::CancelQuery(
   return Status::IOError("Server returned unknown result ", result.result());
 }
 
+::arrow::Result<std::vector<SetSessionOptionResult>> FlightSqlClient::SetSessionOptions(
+    const FlightCallOptions& options,
+    const std::vector<SessionOption>& session_options) {
+  flight_sql_pb::ActionSetSessionOptionsRequest request;
+  for (const SessionOption& in_opt : session_options) {
+    flight_sql_pb::SessionOption& opt = *request.add_session_options();
+    const std::string& name = in_opt.option_name;
+    opt.set_option_name(name);
+
+    const SessionOptionValue& value = in_opt.option_value;
+    if (value.index() == std::variant_npos)
+      return Status::Invalid("Undefined SessionOptionValue type ");
+    switch ((SessionOptionValueType)(value.index())) {

Review Comment:
   no C-style casts



##########
cpp/src/arrow/flight/sql/server.h:
##########
@@ -225,6 +225,27 @@ struct ARROW_FLIGHT_SQL_EXPORT ActionCreatePreparedStatementResult {
   std::string prepared_statement_handle;
 };
 
+/// \brief A request to close the open client session.
+struct ARROW_FLIGHT_SQL_EXPORT ActionCloseSessionRequest {};

Review Comment:
   Ditto here - how is the server supposed to get the session identifier?



##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -359,6 +370,66 @@ arrow::Result<ActionEndTransactionRequest> ParseActionEndTransactionRequest(
   return result;
 }
 
+arrow::Result<ActionSetSessionOptionsRequest> ParseActionSetSessionOptionsRequest(
+    const google::protobuf::Any& any) {
+  pb::sql::ActionSetSessionOptionsRequest command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack ActionSetSessionOptionsRequest");
+  }
+
+  ActionSetSessionOptionsRequest result;
+  if (command.session_options_size() > 0) {
+    result.session_options.reserve(command.session_options_size());
+    for (const pb::sql::SessionOption& in_opt : command.session_options()) {
+      const std::string& name = in_opt.option_name();
+      SessionOption opt;
+      switch (in_opt.option_value_case()) {
+        case pb::sql::SessionOption::OPTION_VALUE_NOT_SET:
+          return Status::Invalid("Unset SessionOptionValue for name '" + name + "'");
+        case pb::sql::SessionOption::kStringValue:
+          opt = {name, in_opt.string_value()};
+          break;
+        case pb::sql::SessionOption::kBoolValue:
+          opt = {name, in_opt.bool_value()};
+          break;
+        case pb::sql::SessionOption::kInt32Value:
+          opt = {name, in_opt.int32_value()};
+          break;
+        case pb::sql::SessionOption::kInt64Value:
+          opt = {name, in_opt.int64_value()};
+          break;
+        case pb::sql::SessionOption::kFloatValue:
+          opt = {name, in_opt.float_value()};
+          break;
+        case pb::sql::SessionOption::kDoubleValue:
+          opt = {name, in_opt.double_value()};
+          break;
+        case pb::sql::SessionOption::kStringListValue:
+          std::vector<std::string> vlist;
+          vlist.reserve(in_opt.string_list_value().values_size());
+          for (const std::string& s : in_opt.string_list_value().values())
+            vlist.push_back(s);
+          opt = {name, vlist};
+          break;
+      }
+      result.session_options.push_back(opt);

Review Comment:
   (Though, why not just emplace_back inside the switch-case?)



##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -423,6 +494,91 @@ arrow::Result<Result> PackActionResult(ActionCreatePreparedStatementResult resul
   return PackActionResult(pb_result);
 }
 
+arrow::Result<Result> PackActionResult(ActionSetSessionOptionsResult result) {
+  pb::sql::ActionSetSessionOptionsResult pb_result;
+  for (SetSessionOptionResult& res : result.results) {
+    switch (res) {
+      case SetSessionOptionResult::kUnspecified:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_UNSPECIFIED);
+        break;
+      case SetSessionOptionResult::kOk:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_OK);
+        break;
+      case SetSessionOptionResult::kInvalidResult:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_INVALID_VALUE);
+        break;
+      case SetSessionOptionResult::kError:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_ERROR);
+        break;
+    }
+  }
+  return PackActionResult(pb_result);
+}
+
+arrow::Result<Result> PackActionResult(ActionGetSessionOptionsResult result) {
+  pb::sql::ActionGetSessionOptionsResult pb_result;
+  for (const SessionOption& in_opt : result.session_options) {
+    pb::sql::SessionOption& opt = *pb_result.add_session_options();

Review Comment:
   two things:
   
   - we generally avoid non-const references in Arrow
   - Can this be shared with the client-side code that is effectively the same?



##########
cpp/src/arrow/flight/types.h:
##########
@@ -409,6 +409,15 @@ struct ARROW_FLIGHT_EXPORT Location {
   /// \brief Get the scheme of this URI.
   std::string scheme() const;
 
+  /// \brief Get the path of this URI.
+  std::string path() const;
+
+  /// \brief Get the query parameters of this URI.
+  arrow::Result<std::vector<std::pair<std::string, std::string>>> query_items() const;

Review Comment:
   nit: snake_case is really only used for trivial getters/setters



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kou commented on pull request #34817: Add Session management messages, Location URI path accessors

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

   Could you open a new issue for this pull request? This is not a MINOR change:
   
   > Thanks for opening a pull request!
   > 
   > If this is not a [minor PR](https://github.com/apache/arrow/blob/main/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on GitHub? https://github.com/apache/arrow/issues/new/choose
   > 
   > Opening GitHub issues ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   > 
   > Then could you also rename the pull request title in the following format?
   > 
   > ```
   > GH-${GITHUB_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   > ```
   > 
   > or
   > 
   > ```
   > MINOR: [${COMPONENT}] ${SUMMARY}
   > ```
   > 
   > In the case of PARQUET issues on JIRA the title also supports:
   > 
   > ```
   > PARQUET-${JIRA_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   > ```
   > 
   > See also:
   > 
   >     * [Other pull requests](https://github.com/apache/arrow/pulls/)
   > 
   >     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -359,6 +370,66 @@ arrow::Result<ActionEndTransactionRequest> ParseActionEndTransactionRequest(
   return result;
 }
 
+arrow::Result<ActionSetSessionOptionsRequest> ParseActionSetSessionOptionsRequest(
+    const google::protobuf::Any& any) {
+  pb::sql::ActionSetSessionOptionsRequest command;
+  if (!any.UnpackTo(&command)) {
+    return Status::Invalid("Unable to unpack ActionSetSessionOptionsRequest");
+  }
+
+  ActionSetSessionOptionsRequest result;
+  if (command.session_options_size() > 0) {
+    result.session_options.reserve(command.session_options_size());
+    for (const pb::sql::SessionOption& in_opt : command.session_options()) {
+      const std::string& name = in_opt.option_name();
+      SessionOption opt;
+      switch (in_opt.option_value_case()) {
+        case pb::sql::SessionOption::OPTION_VALUE_NOT_SET:
+          return Status::Invalid("Unset SessionOptionValue for name '" + name + "'");
+        case pb::sql::SessionOption::kStringValue:
+          opt = {name, in_opt.string_value()};
+          break;
+        case pb::sql::SessionOption::kBoolValue:
+          opt = {name, in_opt.bool_value()};
+          break;
+        case pb::sql::SessionOption::kInt32Value:
+          opt = {name, in_opt.int32_value()};
+          break;
+        case pb::sql::SessionOption::kInt64Value:
+          opt = {name, in_opt.int64_value()};
+          break;
+        case pb::sql::SessionOption::kFloatValue:
+          opt = {name, in_opt.float_value()};
+          break;
+        case pb::sql::SessionOption::kDoubleValue:
+          opt = {name, in_opt.double_value()};
+          break;
+        case pb::sql::SessionOption::kStringListValue:
+          std::vector<std::string> vlist;
+          vlist.reserve(in_opt.string_list_value().values_size());
+          for (const std::string& s : in_opt.string_list_value().values())
+            vlist.push_back(s);
+          opt = {name, vlist};
+          break;
+      }
+      result.session_options.push_back(opt);

Review Comment:
   emplace_back requires we have a constructor which isn't out of the question—you think adding that is the better path 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/types.h:
##########
@@ -409,6 +409,15 @@ struct ARROW_FLIGHT_EXPORT Location {
   /// \brief Get the scheme of this URI.
   std::string scheme() const;
 
+  /// \brief Get the path of this URI.
+  std::string path() const;
+
+  /// \brief Get the query parameters of this URI.
+  arrow::Result<std::vector<std::pair<std::string, std::string>>> QueryItems() const;
+
+  /// \brief Convert URI path and parameters to headers.
+  arrow::Result<std::vector<std::pair<std::string, std::string>>> AsHeaders() const;

Review Comment:
   I don't think it's intrinsically part of a location: it's a specific way to transform a URI that makes sense in specific contexts. (Particularly because the notion of 'URI path' == 'catalog' really _only_ makes sense in Flight SQL, and it locks out other interpretations of the URI that other real users would like to have! #34829)
   
   At the very least: the docstring is inadequate and I think the method name could be more specific as well. I would still prefer this as a static helper in the arrow::flight or arrow::flight::sql namespace but I'm willing to let that go if it's clearly documented what is happening here.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
format/Flight.proto:
##########
@@ -360,3 +361,96 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   Ah, sorry. I mean in terms of naming. Looks like I linked to the wrong place, I meant here as an example: https://github.com/apache/arrow/blob/3ad2d0ccd98192d32d54af371bad6b61ea3c4736/format/Flight.proto#L216
   
   So just CloseSessionRequest here. 



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;

Review Comment:
   When (as with lots of software) an option might have more than one common name due e.g. to how server software evolved, e.g. we might be looking at "catalog" and "project" being interchangeable, where one would get mapped to the canonical one and then GetSessionOptions would return the value under that key rather than the one it was set via.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.h:
##########
@@ -750,6 +754,190 @@ struct ARROW_FLIGHT_EXPORT CancelFlightInfoRequest {
   static arrow::Result<CancelFlightInfoRequest> Deserialize(std::string_view serialized);
 };
 
+/// \brief Variant supporting all possible value types for {Set,Get}SessionOptions
+using SessionOptionValue = std::variant<std::string, bool, int32_t, int64_t, float,
+                                        double, std::vector<std::string>>;
+
+/// \brief The result of setting a session option.
+enum class SetSessionOptionStatus : int8_t {
+  /// \brief The status of setting the option is unknown.
+  ///
+  /// Servers should avoid using this value (send a NOT_FOUND error if the requested
+  /// query is not known). Clients can retry the request.
+  kUnspecified,
+  // The session option setting completed successfully.
+  kOk,
+  /// \brief The given session option name was an alias for another option name.
+  kOkMapped,
+  /// \brief The given session option name is invalid.
+  kInvalidKey,

Review Comment:
   ```suggestion
     kInvalidName,
   ```



##########
cpp/src/arrow/flight/types.h:
##########
@@ -750,6 +754,190 @@ struct ARROW_FLIGHT_EXPORT CancelFlightInfoRequest {
   static arrow::Result<CancelFlightInfoRequest> Deserialize(std::string_view serialized);
 };
 
+/// \brief Variant supporting all possible value types for {Set,Get}SessionOptions
+using SessionOptionValue = std::variant<std::string, bool, int32_t, int64_t, float,
+                                        double, std::vector<std::string>>;
+
+/// \brief The result of setting a session option.
+enum class SetSessionOptionStatus : int8_t {
+  /// \brief The status of setting the option is unknown.
+  ///
+  /// Servers should avoid using this value (send a NOT_FOUND error if the requested
+  /// query is not known). Clients can retry the request.
+  kUnspecified,
+  // The session option setting completed successfully.
+  kOk,
+  /// \brief The given session option name was an alias for another option name.
+  kOkMapped,
+  /// \brief The given session option name is invalid.
+  kInvalidKey,
+  /// \brief The session option value is invalid.
+  kInvalidValue,
+  /// \brief The session option cannot be set.
+  kError
+};
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r);
+
+/// \brief The result of closing a session.
+enum class CloseSessionStatus : int8_t { kUnspecified, kClosed, kClosing, kNotClosable };

Review Comment:
   Can we add document to each enum values?



##########
cpp/src/arrow/flight/types.h:
##########
@@ -750,6 +754,190 @@ struct ARROW_FLIGHT_EXPORT CancelFlightInfoRequest {
   static arrow::Result<CancelFlightInfoRequest> Deserialize(std::string_view serialized);
 };
 
+/// \brief Variant supporting all possible value types for {Set,Get}SessionOptions
+using SessionOptionValue = std::variant<std::string, bool, int32_t, int64_t, float,
+                                        double, std::vector<std::string>>;
+
+/// \brief The result of setting a session option.
+enum class SetSessionOptionStatus : int8_t {
+  /// \brief The status of setting the option is unknown.
+  ///
+  /// Servers should avoid using this value (send a NOT_FOUND error if the requested
+  /// query is not known). Clients can retry the request.
+  kUnspecified,
+  // The session option setting completed successfully.
+  kOk,
+  /// \brief The given session option name was an alias for another option name.
+  kOkMapped,
+  /// \brief The given session option name is invalid.
+  kInvalidKey,
+  /// \brief The session option value is invalid.
+  kInvalidValue,
+  /// \brief The session option cannot be set.
+  kError
+};
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r);

Review Comment:
   ```suggestion
   std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& status);
   ```



##########
cpp/src/arrow/flight/types.h:
##########
@@ -750,6 +754,190 @@ struct ARROW_FLIGHT_EXPORT CancelFlightInfoRequest {
   static arrow::Result<CancelFlightInfoRequest> Deserialize(std::string_view serialized);
 };
 
+/// \brief Variant supporting all possible value types for {Set,Get}SessionOptions
+using SessionOptionValue = std::variant<std::string, bool, int32_t, int64_t, float,
+                                        double, std::vector<std::string>>;
+
+/// \brief The result of setting a session option.
+enum class SetSessionOptionStatus : int8_t {
+  /// \brief The status of setting the option is unknown.
+  ///
+  /// Servers should avoid using this value (send a NOT_FOUND error if the requested
+  /// query is not known). Clients can retry the request.
+  kUnspecified,
+  // The session option setting completed successfully.
+  kOk,
+  /// \brief The given session option name was an alias for another option name.
+  kOkMapped,
+  /// \brief The given session option name is invalid.
+  kInvalidKey,
+  /// \brief The session option value is invalid.
+  kInvalidValue,
+  /// \brief The session option cannot be set.
+  kError
+};
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r);
+
+/// \brief The result of closing a session.
+enum class CloseSessionStatus : int8_t { kUnspecified, kClosed, kClosing, kNotClosable };
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r);
+
+static const char* const SetSessionOptionStatusNames[] = {
+    "Unspecified", "Ok", "OkMapped", "InvalidKey", "InvalidValue", "Error"};
+static const char* const CloseSessionStatusNames[] = {"Unspecified", "Closed", "Closing",
+                                                      "NotClosable"};
+
+/// \brief A request to set a set of session options by key/value.

Review Comment:
   ```suggestion
   /// \brief A request to set a set of session options by name/value.
   ```



##########
cpp/src/arrow/flight/types.h:
##########
@@ -750,6 +754,190 @@ struct ARROW_FLIGHT_EXPORT CancelFlightInfoRequest {
   static arrow::Result<CancelFlightInfoRequest> Deserialize(std::string_view serialized);
 };
 
+/// \brief Variant supporting all possible value types for {Set,Get}SessionOptions
+using SessionOptionValue = std::variant<std::string, bool, int32_t, int64_t, float,
+                                        double, std::vector<std::string>>;
+
+/// \brief The result of setting a session option.
+enum class SetSessionOptionStatus : int8_t {
+  /// \brief The status of setting the option is unknown.
+  ///
+  /// Servers should avoid using this value (send a NOT_FOUND error if the requested
+  /// query is not known). Clients can retry the request.
+  kUnspecified,
+  // The session option setting completed successfully.
+  kOk,
+  /// \brief The given session option name was an alias for another option name.
+  kOkMapped,
+  /// \brief The given session option name is invalid.
+  kInvalidKey,
+  /// \brief The session option value is invalid.
+  kInvalidValue,
+  /// \brief The session option cannot be set.
+  kError
+};
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r);
+
+/// \brief The result of closing a session.
+enum class CloseSessionStatus : int8_t { kUnspecified, kClosed, kClosing, kNotClosable };
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r);
+
+static const char* const SetSessionOptionStatusNames[] = {
+    "Unspecified", "Ok", "OkMapped", "InvalidKey", "InvalidValue", "Error"};
+static const char* const CloseSessionStatusNames[] = {"Unspecified", "Closed", "Closing",
+                                                      "NotClosable"};
+
+/// \brief A request to set a set of session options by key/value.
+struct ARROW_FLIGHT_EXPORT SetSessionOptionsRequest {
+  std::map<std::string, SessionOptionValue> session_options;
+
+  std::string ToString() const;
+  bool Equals(const SetSessionOptionsRequest& other) const;
+
+  friend bool operator==(const SetSessionOptionsRequest& left,
+                         const SetSessionOptionsRequest& right) {
+    return left.Equals(right);
+  }
+  friend bool operator!=(const SetSessionOptionsRequest& left,
+                         const SetSessionOptionsRequest& right) {
+    return !(left == right);
+  }
+
+  /// \brief Serialize this message to its wire-format representation.
+  arrow::Result<std::string> SerializeToString() const;
+
+  /// \brief Deserialize this message from its wire-format representation.
+  static arrow::Result<SetSessionOptionsRequest> Deserialize(std::string_view serialized);
+};
+
+/// \brief The result(s) of setting session option(s).
+struct ARROW_FLIGHT_EXPORT SetSessionOptionsResult {
+  struct Result {
+    SetSessionOptionStatus status;
+
+    bool Equals(const Result& other) const {
+      if (status != other.status) {
+        return false;
+      }
+      return true;

Review Comment:
   ```suggestion
         return status == other.status;
   ```



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -468,6 +468,320 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {

Review Comment:
   ```suggestion
   std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& status) {
   ```



##########
cpp/src/arrow/flight/types.h:
##########
@@ -750,6 +754,190 @@ struct ARROW_FLIGHT_EXPORT CancelFlightInfoRequest {
   static arrow::Result<CancelFlightInfoRequest> Deserialize(std::string_view serialized);
 };
 
+/// \brief Variant supporting all possible value types for {Set,Get}SessionOptions
+using SessionOptionValue = std::variant<std::string, bool, int32_t, int64_t, float,
+                                        double, std::vector<std::string>>;
+
+/// \brief The result of setting a session option.
+enum class SetSessionOptionStatus : int8_t {
+  /// \brief The status of setting the option is unknown.
+  ///
+  /// Servers should avoid using this value (send a NOT_FOUND error if the requested
+  /// query is not known). Clients can retry the request.
+  kUnspecified,
+  // The session option setting completed successfully.
+  kOk,
+  /// \brief The given session option name was an alias for another option name.
+  kOkMapped,
+  /// \brief The given session option name is invalid.
+  kInvalidKey,
+  /// \brief The session option value is invalid.
+  kInvalidValue,
+  /// \brief The session option cannot be set.
+  kError
+};
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r);
+
+/// \brief The result of closing a session.
+enum class CloseSessionStatus : int8_t { kUnspecified, kClosed, kClosing, kNotClosable };
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r);

Review Comment:
   ```suggestion
   std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& status);
   ```



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -468,6 +468,320 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {
+  os << CloseSessionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+// Helpers for stringifying maps containing various types
+std::ostream& operator<<(std::ostream& os, std::vector<std::string> v) {
+  os << '[';
+  std::string sep = "";
+  for (const auto& x : v) {
+    os << sep << '"' << x << '"';

Review Comment:
   If we want to quote a string value in `std::vector<std::string>` with `"`, we also want to quote `std::string` case too.
   If we don't quote, we may be able to use `JoinStrings()` in `cpp/src/arrow/util/string.h`.



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -468,6 +468,320 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {

Review Comment:
   ```suggestion
   std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& status) {
   ```



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -643,6 +957,21 @@ const ActionType ActionType::kRenewFlightEndpoint =
                "Extend expiration time of the given FlightEndpoint.\n"
                "Request Message: RenewFlightEndpointRequest\n"
                "Response Message: Renewed FlightEndpoint"};
+const ActionType ActionType::kSetSessionOptions =
+    ActionType{"SetSessionOptions",
+               "Set client session options by key/value pairs.\n"

Review Comment:
   ```suggestion
                  "Set client session options by name/value pairs.\n"
   ```



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -468,6 +468,320 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {
+  os << CloseSessionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+// Helpers for stringifying maps containing various types
+std::ostream& operator<<(std::ostream& os, std::vector<std::string> v) {
+  os << '[';
+  std::string sep = "";
+  for (const auto& x : v) {
+    os << sep << '"' << x << '"';
+    sep = ", ";
+  }
+  os << ']';
+
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SessionOptionValue& v) {
+  std::visit([&](const auto& x) { os << x; }, v);
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionsResult::Result& r) {
+  os << '{' << r.status << '}';
+  return os;
+}
+
+template <typename T>
+std::ostream& operator<<(std::ostream& os, std::map<std::string, T> m) {
+  os << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : m) {
+    os << sep << '[' << k << "]: '" << v;
+    sep = ", ";
+  }
+  os << '}';
+
+  return os;
+}
+
+namespace {
+static bool CompareSessionOptionMaps(const std::map<std::string, SessionOptionValue>& a,
+                                     const std::map<std::string, SessionOptionValue>& b) {
+  if (a.size() != b.size()) {
+    return false;
+  }
+  for (const auto& [k, v] : a) {
+    if (const auto it = b.find(k); it == b.end()) {
+      return false;
+    } else {
+      const auto& b_v = it->second;
+      if (v.index() != b_v.index()) {
+        return false;
+      }
+      if (v != b_v) {
+        return false;
+      }
+    }
+  }
+  return true;
+}
+}  // namespace
+
+// SetSessionOptionsRequest
+
+std::string SetSessionOptionsRequest::ToString() const {
+  std::stringstream ss;
+
+  ss << "<SetSessionOptionsRequest session_options=" << session_options << '>';
+
+  return ss.str();
+}
+
+bool SetSessionOptionsRequest::Equals(const SetSessionOptionsRequest& other) const {
+  return CompareSessionOptionMaps(session_options, other.session_options);
+}
+
+arrow::Result<std::string> SetSessionOptionsRequest::SerializeToString() const {
+  pb::SetSessionOptionsRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized SetSessionOptionsRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<SetSessionOptionsRequest> SetSessionOptionsRequest::Deserialize(
+    std::string_view serialized) {
+  // TODO these & SerializeToString should all be factored out to a superclass
+  pb::SetSessionOptionsRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized SetSessionOptionsRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid SetSessionOptionsRequest");
+  }
+  SetSessionOptionsRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// SetSessionOptionsResult
+
+std::string SetSessionOptionsResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<SetSessionOptionsResult results=" << results << '>';
+
+  return ss.str();
+}
+
+bool SetSessionOptionsResult::Equals(const SetSessionOptionsResult& other) const {
+  if (results != other.results) {
+    return false;
+  }
+  return true;
+}
+
+arrow::Result<std::string> SetSessionOptionsResult::SerializeToString() const {
+  pb::SetSessionOptionsResult pb_result;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_result));
+
+  std::string out;
+  if (!pb_result.SerializeToString(&out)) {
+    return Status::IOError("Serialized SetSessionOptionsResult exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<SetSessionOptionsResult> SetSessionOptionsResult::Deserialize(
+    std::string_view serialized) {
+  pb::SetSessionOptionsResult pb_result;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized SetSessionOptionsResult size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_result.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid SetSessionOptionsResult");
+  }
+  SetSessionOptionsResult out;
+  RETURN_NOT_OK(internal::FromProto(pb_result, &out));
+  return out;
+}
+
+// GetSessionOptionsRequest
+
+std::string GetSessionOptionsRequest::ToString() const {
+  return "<GetSessionOptionsRequest>";
+}
+
+bool GetSessionOptionsRequest::Equals(const GetSessionOptionsRequest& other) const {
+  return true;
+}
+
+arrow::Result<std::string> GetSessionOptionsRequest::SerializeToString() const {
+  pb::GetSessionOptionsRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized GetSessionOptionsRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<GetSessionOptionsRequest> GetSessionOptionsRequest::Deserialize(
+    std::string_view serialized) {
+  pb::GetSessionOptionsRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized GetSessionOptionsRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid GetSessionOptionsRequest");
+  }
+  GetSessionOptionsRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// GetSessionOptionsResult
+
+std::string GetSessionOptionsResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<GetSessionOptionsResult session_options=" << session_options << '>';
+
+  return ss.str();
+}
+
+bool GetSessionOptionsResult::Equals(const GetSessionOptionsResult& other) const {
+  return CompareSessionOptionMaps(session_options, other.session_options);
+}
+
+arrow::Result<std::string> GetSessionOptionsResult::SerializeToString() const {
+  pb::GetSessionOptionsResult pb_result;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_result));
+
+  std::string out;
+  if (!pb_result.SerializeToString(&out)) {
+    return Status::IOError("Serialized GetSessionOptionsResult exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<GetSessionOptionsResult> GetSessionOptionsResult::Deserialize(
+    std::string_view serialized) {
+  pb::GetSessionOptionsResult pb_result;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized GetSessionOptionsResult size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_result.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid GetSessionOptionsResult");
+  }
+  GetSessionOptionsResult out;
+  RETURN_NOT_OK(internal::FromProto(pb_result, &out));
+  return out;
+}
+
+// CloseSessionRequest
+
+std::string CloseSessionRequest::ToString() const { return "<CloseSessionRequest>"; }
+
+bool CloseSessionRequest::Equals(const CloseSessionRequest& other) const { return true; }
+
+arrow::Result<std::string> CloseSessionRequest::SerializeToString() const {
+  pb::CloseSessionRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized CloseSessionRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<CloseSessionRequest> CloseSessionRequest::Deserialize(
+    std::string_view serialized) {
+  pb::CloseSessionRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid("Serialized CloseSessionRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid CloseSessionRequest");
+  }
+  CloseSessionRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// CloseSessionResult
+
+std::string CloseSessionResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<CloseSessionResult result=" << status << '>';

Review Comment:
   ```suggestion
     ss << "<CloseSessionResult status=" << status << '>';
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,74 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <functional>
+#include <optional>
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+static constexpr char const kSessionCookieName[] = "arrow_flight_session_id";
+
+class ARROW_FLIGHT_SQL_EXPORT FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+
+ public:
+  /// \brief Get session option by key
+  std::optional<SessionOptionValue> GetSessionOption(const std::string&);
+  /// \brief Set session option by key to given value
+  void SetSessionOption(const std::string& key, const SessionOptionValue v);
+  /// \brief Idempotently remove key from this call's Session, if Session & key exist
+  void EraseSessionOption(const std::string& key);

Review Comment:
   ```suggestion
     /// \brief Idempotently remove name from this call's session, if session & name exist
     void EraseSessionOption(const std::string& name);
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,74 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <functional>
+#include <optional>
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+static constexpr char const kSessionCookieName[] = "arrow_flight_session_id";
+
+class ARROW_FLIGHT_SQL_EXPORT FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+
+ public:
+  /// \brief Get session option by key
+  std::optional<SessionOptionValue> GetSessionOption(const std::string&);

Review Comment:
   ```suggestion
     /// \brief Get session option by name
     std::optional<SessionOptionValue> GetSessionOption(const std::string& name);
   ```



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -468,6 +468,320 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {
+  os << CloseSessionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+// Helpers for stringifying maps containing various types
+std::ostream& operator<<(std::ostream& os, std::vector<std::string> v) {
+  os << '[';
+  std::string sep = "";
+  for (const auto& x : v) {
+    os << sep << '"' << x << '"';
+    sep = ", ";
+  }
+  os << ']';
+
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SessionOptionValue& v) {
+  std::visit([&](const auto& x) { os << x; }, v);
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionsResult::Result& r) {
+  os << '{' << r.status << '}';
+  return os;
+}
+
+template <typename T>
+std::ostream& operator<<(std::ostream& os, std::map<std::string, T> m) {
+  os << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : m) {
+    os << sep << '[' << k << "]: '" << v;
+    sep = ", ";
+  }
+  os << '}';
+
+  return os;
+}
+
+namespace {
+static bool CompareSessionOptionMaps(const std::map<std::string, SessionOptionValue>& a,
+                                     const std::map<std::string, SessionOptionValue>& b) {
+  if (a.size() != b.size()) {
+    return false;
+  }
+  for (const auto& [k, v] : a) {
+    if (const auto it = b.find(k); it == b.end()) {
+      return false;
+    } else {
+      const auto& b_v = it->second;
+      if (v.index() != b_v.index()) {
+        return false;
+      }
+      if (v != b_v) {
+        return false;
+      }
+    }
+  }
+  return true;
+}
+}  // namespace
+
+// SetSessionOptionsRequest
+
+std::string SetSessionOptionsRequest::ToString() const {
+  std::stringstream ss;
+
+  ss << "<SetSessionOptionsRequest session_options=" << session_options << '>';
+
+  return ss.str();
+}
+
+bool SetSessionOptionsRequest::Equals(const SetSessionOptionsRequest& other) const {
+  return CompareSessionOptionMaps(session_options, other.session_options);
+}
+
+arrow::Result<std::string> SetSessionOptionsRequest::SerializeToString() const {
+  pb::SetSessionOptionsRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized SetSessionOptionsRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<SetSessionOptionsRequest> SetSessionOptionsRequest::Deserialize(
+    std::string_view serialized) {
+  // TODO these & SerializeToString should all be factored out to a superclass
+  pb::SetSessionOptionsRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized SetSessionOptionsRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid SetSessionOptionsRequest");
+  }
+  SetSessionOptionsRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// SetSessionOptionsResult
+
+std::string SetSessionOptionsResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<SetSessionOptionsResult results=" << results << '>';
+
+  return ss.str();
+}
+
+bool SetSessionOptionsResult::Equals(const SetSessionOptionsResult& other) const {
+  if (results != other.results) {
+    return false;
+  }
+  return true;
+}
+
+arrow::Result<std::string> SetSessionOptionsResult::SerializeToString() const {
+  pb::SetSessionOptionsResult pb_result;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_result));
+
+  std::string out;
+  if (!pb_result.SerializeToString(&out)) {
+    return Status::IOError("Serialized SetSessionOptionsResult exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<SetSessionOptionsResult> SetSessionOptionsResult::Deserialize(
+    std::string_view serialized) {
+  pb::SetSessionOptionsResult pb_result;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized SetSessionOptionsResult size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_result.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid SetSessionOptionsResult");
+  }
+  SetSessionOptionsResult out;
+  RETURN_NOT_OK(internal::FromProto(pb_result, &out));
+  return out;
+}
+
+// GetSessionOptionsRequest
+
+std::string GetSessionOptionsRequest::ToString() const {
+  return "<GetSessionOptionsRequest>";
+}
+
+bool GetSessionOptionsRequest::Equals(const GetSessionOptionsRequest& other) const {
+  return true;
+}
+
+arrow::Result<std::string> GetSessionOptionsRequest::SerializeToString() const {
+  pb::GetSessionOptionsRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized GetSessionOptionsRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<GetSessionOptionsRequest> GetSessionOptionsRequest::Deserialize(
+    std::string_view serialized) {
+  pb::GetSessionOptionsRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized GetSessionOptionsRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid GetSessionOptionsRequest");
+  }
+  GetSessionOptionsRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// GetSessionOptionsResult
+
+std::string GetSessionOptionsResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<GetSessionOptionsResult session_options=" << session_options << '>';
+
+  return ss.str();
+}
+
+bool GetSessionOptionsResult::Equals(const GetSessionOptionsResult& other) const {
+  return CompareSessionOptionMaps(session_options, other.session_options);
+}
+
+arrow::Result<std::string> GetSessionOptionsResult::SerializeToString() const {
+  pb::GetSessionOptionsResult pb_result;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_result));
+
+  std::string out;
+  if (!pb_result.SerializeToString(&out)) {
+    return Status::IOError("Serialized GetSessionOptionsResult exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<GetSessionOptionsResult> GetSessionOptionsResult::Deserialize(
+    std::string_view serialized) {
+  pb::GetSessionOptionsResult pb_result;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized GetSessionOptionsResult size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_result.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid GetSessionOptionsResult");
+  }
+  GetSessionOptionsResult out;
+  RETURN_NOT_OK(internal::FromProto(pb_result, &out));
+  return out;
+}
+
+// CloseSessionRequest
+
+std::string CloseSessionRequest::ToString() const { return "<CloseSessionRequest>"; }
+
+bool CloseSessionRequest::Equals(const CloseSessionRequest& other) const { return true; }
+
+arrow::Result<std::string> CloseSessionRequest::SerializeToString() const {
+  pb::CloseSessionRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized CloseSessionRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<CloseSessionRequest> CloseSessionRequest::Deserialize(
+    std::string_view serialized) {
+  pb::CloseSessionRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid("Serialized CloseSessionRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid CloseSessionRequest");
+  }
+  CloseSessionRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// CloseSessionResult
+
+std::string CloseSessionResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<CloseSessionResult result=" << status << '>';
+
+  return ss.str();
+}
+
+bool CloseSessionResult::Equals(const CloseSessionResult& other) const {
+  if (status != other.status) {
+    return false;
+  }
+  return true;

Review Comment:
   ```suggestion
     return status == other.status;
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();

Review Comment:
   How about `Create` than `Get` because `Create` is more descriptive?
   
   ```suggestion
     std::pair<std::string, std::shared_ptr<FlightSqlSession>> CreateNewSession();
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,74 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <functional>
+#include <optional>
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+static constexpr char const kSessionCookieName[] = "arrow_flight_session_id";
+
+class ARROW_FLIGHT_SQL_EXPORT FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+
+ public:
+  /// \brief Get session option by key
+  std::optional<SessionOptionValue> GetSessionOption(const std::string&);
+  /// \brief Set session option by key to given value
+  void SetSessionOption(const std::string& key, const SessionOptionValue v);

Review Comment:
   ```suggestion
     /// \brief Set session option by name to given value
     void SetSessionOption(const std::string& name, const SessionOptionValue value);
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;

Review Comment:
   I think that `mutex_` is better here because this is not `std::shared_lock`:
   
   ```suggestion
     std::shared_mutex mutex_;
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,74 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <functional>
+#include <optional>
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+static constexpr char const kSessionCookieName[] = "arrow_flight_session_id";
+
+class ARROW_FLIGHT_SQL_EXPORT FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+
+ public:
+  /// \brief Get session option by key
+  std::optional<SessionOptionValue> GetSessionOption(const std::string&);
+  /// \brief Set session option by key to given value
+  void SetSessionOption(const std::string& key, const SessionOptionValue v);
+  /// \brief Idempotently remove key from this call's Session, if Session & key exist
+  void EraseSessionOption(const std::string& key);
+};
+
+/// \brief A middleware to handle Session option persistence and related *Cookie headers.

Review Comment:
   ```suggestion
   /// \brief A middleware to handle session option persistence and related cookie headers.
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,74 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <functional>
+#include <optional>
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+static constexpr char const kSessionCookieName[] = "arrow_flight_session_id";
+
+class ARROW_FLIGHT_SQL_EXPORT FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+
+ public:
+  /// \brief Get session option by key
+  std::optional<SessionOptionValue> GetSessionOption(const std::string&);
+  /// \brief Set session option by key to given value
+  void SetSessionOption(const std::string& key, const SessionOptionValue v);
+  /// \brief Idempotently remove key from this call's Session, if Session & key exist
+  void EraseSessionOption(const std::string& key);
+};
+
+/// \brief A middleware to handle Session option persistence and related *Cookie headers.
+class ARROW_FLIGHT_SQL_EXPORT ServerSessionMiddleware : public ServerMiddleware {
+ public:
+  static constexpr char const kMiddlewareName[] =
+      "arrow::flight::sql::ServerSessionMiddleware";
+
+  std::string name() const override { return kMiddlewareName; }
+
+  /// \brief Is there an existing session (either existing or new)
+  virtual bool HasSession() const = 0;
+  /// \brief Get existing or new call-associated session
+  virtual std::shared_ptr<FlightSqlSession> GetSession() = 0;
+  /// \brief Get request headers, in lieu of a provided or created session.
+  virtual const CallHeaders& GetCallHeaders() const = 0;
+};
+
+/// \brief Returns a ServerMiddlewareFactory that handles Session option storage.
+/// \param[in] id_gen A generator function for unique session id strings.

Review Comment:
   ```suggestion
   /// \param[in] id_gen A generator function for unique session ID strings.
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);

Review Comment:
   Is `shared_lock` safe here?
   It seems that we need to use `lock_guard` instead here because `session_` may be updated.



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();

Review Comment:
   ```suggestion
     const auto pair_sep_len = pair_sep.length();
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }

Review Comment:
   Can we use `arrow::flight::internal::Cookie` instead of implementing this?



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");
+  for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+    const std::string_view& cookie_header = itr->second;
+    const std::vector<std::pair<std::string, std::string>> cookies =
+        ParseCookieString(cookie_header);
+    for (const std::pair<std::string, std::string>& cookie : cookies) {
+      if (cookie.first == kSessionCookieName) {
+        if (cookie.second.empty())
+          return Status::Invalid("Empty ", kSessionCookieName, " cookie value.");
+        session_id = std::move(cookie.second);
+      }
+    }
+    if (!session_id.empty()) break;
+  }
+
+  if (session_id.empty()) {
+    // No cookie was found
+    *middleware = std::make_shared<ServerSessionMiddlewareImpl>(this, incoming_headers);
+  } else {
+    const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+    if (auto it = session_store_.find(session_id); it == session_store_.end()) {
+      return Status::Invalid("Invalid or expired ", kSessionCookieName, " cookie.");
+    } else {
+      auto session = it->second;
+      *middleware = std::make_shared<ServerSessionMiddlewareImpl>(
+          this, incoming_headers, std::move(session), session_id);
+    }
+  }
+
+  return Status::OK();
+}
+
+/// \brief Get a new, empty session option map and its id key.
+std::pair<std::string, std::shared_ptr<FlightSqlSession>>
+ServerSessionMiddlewareFactory::GetNewSession() {
+  std::string new_id = id_generator_();
+  auto session = std::make_shared<FlightSqlSession>();
+
+  const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+  session_store_[new_id] = session;

Review Comment:
   Should we re-generate a new ID when the generated ID is already used?



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");
+  for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+    const std::string_view& cookie_header = itr->second;
+    const std::vector<std::pair<std::string, std::string>> cookies =
+        ParseCookieString(cookie_header);
+    for (const std::pair<std::string, std::string>& cookie : cookies) {
+      if (cookie.first == kSessionCookieName) {
+        if (cookie.second.empty())
+          return Status::Invalid("Empty ", kSessionCookieName, " cookie value.");
+        session_id = std::move(cookie.second);
+      }
+    }
+    if (!session_id.empty()) break;
+  }
+
+  if (session_id.empty()) {
+    // No cookie was found
+    *middleware = std::make_shared<ServerSessionMiddlewareImpl>(this, incoming_headers);
+  } else {
+    const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+    if (auto it = session_store_.find(session_id); it == session_store_.end()) {
+      return Status::Invalid("Invalid or expired ", kSessionCookieName, " cookie.");
+    } else {
+      auto session = it->second;
+      *middleware = std::make_shared<ServerSessionMiddlewareImpl>(
+          this, incoming_headers, std::move(session), session_id);
+    }
+  }
+
+  return Status::OK();
+}
+
+/// \brief Get a new, empty session option map and its id key.
+std::pair<std::string, std::shared_ptr<FlightSqlSession>>
+ServerSessionMiddlewareFactory::GetNewSession() {
+  std::string new_id = id_generator_();

Review Comment:
   ```suggestion
     auto new_id = id_generator_();
   ```



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -468,6 +468,320 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {
+  os << CloseSessionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+// Helpers for stringifying maps containing various types
+std::ostream& operator<<(std::ostream& os, std::vector<std::string> v) {
+  os << '[';
+  std::string sep = "";
+  for (const auto& x : v) {
+    os << sep << '"' << x << '"';

Review Comment:
   ```suggestion
   std::ostream& operator<<(std::ostream& os, std::vector<std::string> values) {
     os << '[';
     std::string sep = "";
     for (const auto& v : values) {
       os << sep << '"' << v << '"';
   ```



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -468,6 +468,320 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {
+  os << CloseSessionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+// Helpers for stringifying maps containing various types
+std::ostream& operator<<(std::ostream& os, std::vector<std::string> v) {
+  os << '[';
+  std::string sep = "";
+  for (const auto& x : v) {
+    os << sep << '"' << x << '"';
+    sep = ", ";
+  }
+  os << ']';
+
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SessionOptionValue& v) {
+  std::visit([&](const auto& x) { os << x; }, v);
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionsResult::Result& r) {
+  os << '{' << r.status << '}';
+  return os;
+}
+
+template <typename T>
+std::ostream& operator<<(std::ostream& os, std::map<std::string, T> m) {
+  os << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : m) {
+    os << sep << '[' << k << "]: '" << v;

Review Comment:
   Is the `'` quote balanced?



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,74 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <functional>
+#include <optional>
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+static constexpr char const kSessionCookieName[] = "arrow_flight_session_id";
+
+class ARROW_FLIGHT_SQL_EXPORT FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+
+ public:
+  /// \brief Get session option by key
+  std::optional<SessionOptionValue> GetSessionOption(const std::string&);
+  /// \brief Set session option by key to given value
+  void SetSessionOption(const std::string& key, const SessionOptionValue v);
+  /// \brief Idempotently remove key from this call's Session, if Session & key exist
+  void EraseSessionOption(const std::string& key);
+};
+
+/// \brief A middleware to handle Session option persistence and related *Cookie headers.
+class ARROW_FLIGHT_SQL_EXPORT ServerSessionMiddleware : public ServerMiddleware {
+ public:
+  static constexpr char const kMiddlewareName[] =
+      "arrow::flight::sql::ServerSessionMiddleware";
+
+  std::string name() const override { return kMiddlewareName; }
+
+  /// \brief Is there an existing session (either existing or new)
+  virtual bool HasSession() const = 0;
+  /// \brief Get existing or new call-associated session
+  virtual std::shared_ptr<FlightSqlSession> GetSession() = 0;
+  /// \brief Get request headers, in lieu of a provided or created session.
+  virtual const CallHeaders& GetCallHeaders() const = 0;
+};
+
+/// \brief Returns a ServerMiddlewareFactory that handles Session option storage.

Review Comment:
   ```suggestion
   /// \brief Returns a ServerMiddlewareFactory that handles session option storage.
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;

Review Comment:
   ```suggestion
     const bool existing_session_;
   ```



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {

Review Comment:
   Yes.  We very explicitly wanted more int/etc values for programmatic use cases of passing exact numeric parameters in their expected type container.  I had also considered that Value gets less interesting with Java and signed ints, but nonetheless as with the strict map (rather than loose k:v pairs) we wanted the constraint of the values coming in in-bounds so a server implementation doesn't need to boundscheck them and care about informative errors and so on.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########


Review Comment:
   Quite probably.  Maybe make this an extension task for now, although it won't take a whole ton of time.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   I think if you put any authentication middleware first, then it can reject unauthenticated requests and you can always create a session after that.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> CreateNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex mutex_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session_;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session_(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session_(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session_ && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::lock_guard<std::shared_mutex> l(mutex_);
+    if (!session_) {
+      auto [id, s] = factory_->CreateNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const auto pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }
+
+  return result;
+}
+
+Status ServerSessionMiddlewareFactory::StartCall(
+    const CallInfo&, const CallHeaders& incoming_headers,
+    std::shared_ptr<ServerMiddleware>* middleware) {
+  std::string session_id;
+
+  const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+      headers_it_pr = incoming_headers.equal_range("cookie");
+  for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+    const std::string_view& cookie_header = itr->second;
+    const std::vector<std::pair<std::string, std::string>> cookies =
+        ParseCookieString(cookie_header);
+    for (const std::pair<std::string, std::string>& cookie : cookies) {
+      if (cookie.first == kSessionCookieName) {
+        if (cookie.second.empty())
+          return Status::Invalid("Empty ", kSessionCookieName, " cookie value.");
+        session_id = std::move(cookie.second);
+      }
+    }
+    if (!session_id.empty()) break;
+  }
+
+  if (session_id.empty()) {
+    // No cookie was found
+    *middleware = std::make_shared<ServerSessionMiddlewareImpl>(this, incoming_headers);
+  } else {
+    const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+    if (auto it = session_store_.find(session_id); it == session_store_.end()) {
+      return Status::Invalid("Invalid or expired ", kSessionCookieName, " cookie.");
+    } else {
+      auto session = it->second;
+      *middleware = std::make_shared<ServerSessionMiddlewareImpl>(
+          this, incoming_headers, std::move(session), session_id);
+    }
+  }
+
+  return Status::OK();
+}
+
+/// \brief Get a new, empty session option map and its id key.
+std::pair<std::string, std::shared_ptr<FlightSqlSession>>
+ServerSessionMiddlewareFactory::CreateNewSession() {
+  auto new_id = id_generator_();

Review Comment:
   `/// \param[in] id_gen A thread-safe, collision-free generator for session id strings.`



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########


Review Comment:
   Agreed as elsewhere this could/should be [in future] rolled into GetSessionOptions under an optional e.g. `supported_session_options` nested message, presumably as a map to optional vectors of valid values, with some decisions to be made around whether multiple value types are accommodates over-the-wire.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }

Review Comment:
   OK. But the implementation is buggy:
   
   * `a=b` returns an empty vector instead of `[<a,b>]`
   * `a=b;c=d` returns an empty vector instead of `[<a,b>, <c,d>]`
   * `a=b; c=d` causes an infinite loop
   
   Can we add tests?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/gandiva/gdv_hash_function_stubs.cc:
##########


Review Comment:
   Inconsistent linter version :)  Should all be backed out now.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########


Review Comment:
   Yeah, I might want to think a bit harder about handling invisible options or how to communicate valid option value type(s) to the client (Proto doesn't really have a reflection facility like that, I don't think?), so I'll defer this.  Easy enough to add later and doesn't break anything for now.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.

Review Comment:
   > Hmm... I'm not sure I understand: what does cookies have to do with those protobuf declarations?
   
   Cookies are used to persist the session across otherwise stateless Flight/Flight SQL calls.



##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.

Review Comment:
   > Also, since this references HTTP, how does this translate to e.g. the UCX backend?
   
   Looking at UCX, its docs are particularly sparse.  The documentation here can be reworded:  Does UCX have its own "State Management Mechanism" as it were, to persist such things across its individual UCP "calls", which we could generalize the wording here to capture?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;

Review Comment:
   It's a local so I'll skip the bool, but yes .empty() is cleaner for all the uses here.  Fixed.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.cc:
##########
@@ -463,6 +463,300 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+// Helper for stringifying maps containing various types
+template <typename T>
+ostream& operator<<(std::map<std::string, T>) {
+  std::stringstream ss;
+
+  ss << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : session_options) {
+    std::cout << sep << '[' << k << "]: '" << v << "', ";  // PHOXME v.ToString() not implemented yet
+    sep = ", ";
+  }
+  ss << '}';
+
+  return ss.str();
+}
+static bool CompareSessionOptionMaps(

Review Comment:
   Ty, done.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOption {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  string option_name = 1;
+  oneof option_value {
+    string string_value = 2;
+    bool bool_value = 3;
+    sfixed32 int32_value = 4;
+    sfixed64 int64_value = 5;
+    float float_value = 6;
+    double double_value = 7;

Review Comment:
   I believe James Duong can provide a better explanation than I did—he was involved in this design decision.  (Closing thread.)



##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOption {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  string option_name = 1;
+  oneof option_value {
+    string string_value = 2;
+    bool bool_value = 3;
+    sfixed32 int32_value = 4;
+    sfixed64 int64_value = 5;
+    float float_value = 6;
+    double double_value = 7;

Review Comment:
   I believe James Duong can provide a better explanation than I did—he was involved in this design decision.  (Closing thread.)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;
+    }
+
+    if (!session_id.length()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers,
+                                        session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid(
+            "Invalid or expired "
+            + static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session,
+    std::string session_id)
+    : factory_(factory), headers_(headers), session_(session), existing_session(true) {}

Review Comment:
   move() when initializing fields from params



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;
+    }
+
+    if (!session_id.length()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers,
+                                        session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid(
+            "Invalid or expired "
+            + static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;

Review Comment:
   Why not assign directly to *session_id instead of having the new_id local



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/client.cc:
##########
@@ -793,6 +800,149 @@ ::arrow::Result<CancelResult> FlightSqlClient::CancelQuery(
   return Status::IOError("Server returned unknown result ", result.result());
 }
 
+::arrow::Result<std::map<std::string, SetSessionOptionResult>>
+FlightSqlClient::SetSessionOptions(
+    const FlightCallOptions& options,
+    const std::map<std::string, SessionOptionValue>& session_options) {
+  pb::ActionSetSessionOptionsRequest request;
+  auto* options_map = request.mutable_session_options();
+
+  for (const auto & [name, opt_value] : session_options) {
+    pb::SessionOptionValue pb_opt_value;
+
+    if (opt_value.index() == std::variant_npos)
+      return Status::Invalid("Undefined SessionOptionValue type ");
+
+    std::visit(overloaded{
+      // TODO move this somewhere common that can have Proto-involved code
+      [&](std::string v) { pb_opt_value.set_string_value(v); },
+      [&](bool v) { pb_opt_value.set_bool_value(v); },
+      [&](int32_t v) { pb_opt_value.set_int32_value(v); },
+      [&](int64_t v) { pb_opt_value.set_int64_value(v); },
+      [&](float v) { pb_opt_value.set_float_value(v); },
+      [&](double v) { pb_opt_value.set_double_value(v); },
+      [&](std::vector<std::string> v) {
+        auto* string_list_value = pb_opt_value.mutable_string_list_value();
+        for (const std::string& s : v)

Review Comment:
   Could use ->Add() or ->Assign() here to pass in a range instead of a loop. Note that the current code doesn't reserve() memory.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.cc:
##########
@@ -463,6 +463,300 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+// Helper for stringifying maps containing various types
+template <typename T>
+ostream& operator<<(std::map<std::string, T>) {
+  std::stringstream ss;
+
+  ss << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : session_options) {
+    std::cout << sep << '[' << k << "]: '" << v << "', ";  // PHOXME v.ToString() not implemented yet
+    sep = ", ";
+  }
+  ss << '}';
+
+  return ss.str();
+}
+static bool CompareSessionOptionMaps(

Review Comment:
   Does this have different semantics than operator== for maps?
   https://en.cppreference.com/w/cpp/container/map/operator_cmp
   Where does .session_options come from?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/client.h:
##########
@@ -383,6 +383,27 @@ class ARROW_FLIGHT_EXPORT FlightClient {
     return DoExchange({}, descriptor);
   }
 
+  /// \\brief Set server session option(s) by key/value. Sessions are generally
+  /// persisted via HTTP cookies.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The server session options to set
+  ::arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const FlightCallOptions& options, const SetSessionOptionsRequest& request);
+
+  /// \\brief Get the current server session options. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The GetSessionOptions request object.
+  ::arrow::Result<GetSessionOptionsResult> GetSessionOptions(
+      const FlightCallOptions& options, const GetSessionOptionsRequest& request);
+
+  /// \\brief Close/invalidate the current server session. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The CloseSession request object.
+  ::arrow::Result<CloseSessionResult> CloseSession(const FlightCallOptions& options,
+                                                   const CloseSessionRequest& request);

Review Comment:
   Correct.  Passing an empty message around allows for extension without breaking interfaces that currently assume no object / Proto message / etc.  In this case it might later want something like application-specific hints as to how to handle various levels of forcible closure 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValueFactory.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.SessionOptionValue;
+
+public class SessionOptionValueFactory {
+    public static SessionOptionValue makeSessionOption(String value) {
+        return new SessionOptionValueString(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(bool value) {
+        return new SessionOptionValueBool(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(int value) {
+        return new SessionOptionValueInt(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(long value) {
+        return new SessionOptionValueLong(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(float value) {
+        return new SessionOptionValueFloat(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(double value) {
+        return new SessionOptionValueDouble(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(String[] value) {
+        return new SessionOptionValueStringList(value);
+    }
+}
+
+class SessionOptionValueString {

Review Comment:
   +Added comment to Proto (already in C++ and Java types/visitors respectively)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    OK = 1;
+    // The given session option name was an alias for another option name.
+    OK_MAPPED = 2;
+    // The given session option name is invalid.
+    INVALID_NAME = 3;
+    // The session option value is invalid.
+    INVALID_VALUE = 4;
+    // The session option cannot be set.
+    ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}

Review Comment:
   > For the initial session creation, shouldn't it be possible to combine with the handshake? It is used only for auth, but it could be changed to be used also if session options and/or auth is to be configured? (otherwise to be skipped?) (I'm assuming that most people may actually set auth already)
   
   Again a server implementation detail, but any other handler that can touch the middleware stack can freely tell custom middleware to create a session which can be used for any other purpose in addition to the session option functionality.  For example with a "common" case of auth headers, a common session middleware can be used which would then trigger session creation either on (depending on client behaviour/feature support) (a) an initial SetSessionOptions call OR (b) any other initial call where the auth headers would be examined by the auth middleware.
   
   For any implementation that's set up to have an explicit auth RPC, there's not much we can do to combine that with SetSessionOptions, and that becomes more a matter of rolling Action(/etc) batching/streaming into Arrow, i.e. a "DoActions" sort of thing.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.

Review Comment:
   Updated docs to be more general.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    OK = 1;
+    // The given session option name was an alias for another option name.
+    OK_MAPPED = 2;
+    // The given session option name is invalid.
+    INVALID_NAME = 3;
+    // The session option value is invalid.
+    INVALID_VALUE = 4;
+    // The session option cannot be set.
+    ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}

Review Comment:
   > should there be a corresponding `CreateSessionRequest`?
   This is implicit in SetSessionOptions, OR the server may go ahead and create a session for its own purposes which would then also be tied into the same session.  E.g. @kou has discussed tying AuthN into the session state as well, where SetSessionOptions calls (if any) would be subsequent to the session creation.
   > More to the point, should there be a default TTL on sessions (modifiable via options) for the case of a client which doesn't close their session?
   Seems like a server implementation detail?  The client can infer this (not robustly) by looking at the cookies emitted by the server, particularly those included in a SetSessionOptions response.
   
   



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   The problem is that this is asking the client to handle something they're not interested in, and not able to act on in any way. This sounds like the Windows behavior of refusing to delete a file, when it's being used by another process.
   



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/CloseSessionResult.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class CloseSessionResult {
+  public enum Status {
+    /**
+     * The session close status is unknown. Servers should avoid using this value
+     * (send a NOT_FOUND error if the requested session is not known). Clients can
+     * retry the request.
+     */
+    UNSPECIFIED(Flight.CloseSessionResult.Status.UNSPECIFIED),
+    /**
+     * The session close request is complete.
+     */
+    CLOSED(Flight.CloseSessionResult.Status.CLOSED),
+    /**
+     * The session close request is in progress. The client may retry the request.
+     */
+    CLOSING(Flight.CloseSessionResult.Status.CLOSING),
+    /**
+     * The session is not closeable.
+     */
+    NOT_CLOSABLE(Flight.CloseSessionResult.Status.NOT_CLOSABLE),
+    ;
+
+    private static final Map<Flight.CloseSessionResult.Status, Status> mapFromProto;
+
+    static {
+      for (Status s : values()) mapFromProto.put(s.proto, s);

Review Comment:
   Sorry, confused on `s` vs `proto_source_value_parameter_name` in your initial comment; makes sense now.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   I think if I proposed this my rationale was long lived distributed operations can typically be long lived or hard to interrupt immediately.  IIUC sessions seems like they unlikely to have distributed work associated with them (unless we are hoping to do things like unclose a session?)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   Backing up though...why does the RPC handler body need to affect the middleware state before it sends headers? 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   > Though is what you want, all middleware run StartCall, then after that all middleware run SendingHeaders? That could be fixed
   
   What I'm seeing right now that's causing problems is that SendingHeaders is being called before the RPC handler, in this case DoAction, so the headers aren't being generated as they depend on the middleware instance state post-RPC-handler.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   ServerCallContext has explicit AddHeader now. So you could implement it not as a middleware, but just as a plain class that reads/writes headers from the context, and sidestep that. I think it's pretty clear that middleware are borked in Flight in C++ anyways (due to issues with gRPC itself - 'real' gRPC interceptors aren't allowed to communicate state to the RPC handler for reasons unclear to me)


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   > Could you implement an integration test?
   
   Done in C++ and Java, unfortunately with some limitations (for now) due to https://github.com/apache/arrow/issues/39791 which I'm hoping to fix at some point.  Also unit tests around the C++ cookie string parsing, as it is much more intricate than the Java impl.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   I've closed the vote. I'll merge this now.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -423,6 +494,91 @@ arrow::Result<Result> PackActionResult(ActionCreatePreparedStatementResult resul
   return PackActionResult(pb_result);
 }
 
+arrow::Result<Result> PackActionResult(ActionSetSessionOptionsResult result) {
+  pb::sql::ActionSetSessionOptionsResult pb_result;
+  for (SetSessionOptionResult& res : result.results) {
+    switch (res) {
+      case SetSessionOptionResult::kUnspecified:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_UNSPECIFIED);
+        break;
+      case SetSessionOptionResult::kOk:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_OK);
+        break;
+      case SetSessionOptionResult::kInvalidResult:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_INVALID_VALUE);
+        break;
+      case SetSessionOptionResult::kError:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_ERROR);
+        break;
+    }
+  }
+  return PackActionResult(pb_result);
+}
+
+arrow::Result<Result> PackActionResult(ActionGetSessionOptionsResult result) {
+  pb::sql::ActionGetSessionOptionsResult pb_result;
+  for (const SessionOption& in_opt : result.session_options) {
+    pb::sql::SessionOption& opt = *pb_result.add_session_options();

Review Comment:
   The version of the Google C++ guidelines that Arrow was developed with prefers `T*` to `T&`.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] github-actions[bot] commented on pull request #34817: Add Session management messages, Location URI path accessors

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34817:
URL: https://github.com/apache/arrow/pull/34817#issuecomment-1491220559

   <!--
     Licensed to the Apache Software Foundation (ASF) under one
     or more contributor license agreements.  See the NOTICE file
     distributed with this work for additional information
     regarding copyright ownership.  The ASF licenses this file
     to you under the Apache License, Version 2.0 (the
     "License"); you may not use this file except in compliance
     with the License.  You may obtain a copy of the License at
   
       http://www.apache.org/licenses/LICENSE-2.0
   
     Unless required by applicable law or agreed to in writing,
     software distributed under the License is distributed on an
     "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
     KIND, either express or implied.  See the License for the
     specific language governing permissions and limitations
     under the License.
   -->
   
   Thanks for opening a pull request!
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/main/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on GitHub? https://github.com/apache/arrow/issues/new/choose
   
   Opening GitHub issues ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename the pull request title in the following format?
   
       GH-${GITHUB_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   In the case of PARQUET issues on JIRA the title also supports:
   
       PARQUET-${JIRA_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/types.h:
##########
@@ -409,6 +409,15 @@ struct ARROW_FLIGHT_EXPORT Location {
   /// \brief Get the scheme of this URI.
   std::string scheme() const;
 
+  /// \brief Get the path of this URI.
+  std::string path() const;
+
+  /// \brief Get the query parameters of this URI.
+  arrow::Result<std::vector<std::pair<std::string, std::string>>> QueryItems() const;
+
+  /// \brief Convert URI path and parameters to headers.
+  arrow::Result<std::vector<std::pair<std::string, std::string>>> AsHeaders() const;

Review Comment:
   This is still very specific to Flight SQL (and the docstring isn't very clear on what the transformation is). Could it be implemented as a helper function in the `arrow::flight::sql` namespace?



##########
cpp/src/arrow/flight/sql/types.h:
##########
@@ -44,6 +44,23 @@ using SqlInfoResult =
 /// \brief Map SQL info identifier to its value.
 using SqlInfoResultMap = std::unordered_map<int32_t, SqlInfoResult>;
 
+/// \brief Variant supporting all possible types for SetSessionOption
+using SessionOptionValue =
+    std::variant<std::string, bool, int32_t, int64_t, float, double, std::vector<std::string>>;
+
+enum struct SessionOptionValueType : size_t {
+  kString, kBool, kInt32, kInt64, kFloat, kDouble, kStringList
+};
+
+struct ARROW_FLIGHT_SQL_EXPORT SessionOption {
+  std::string option_name;
+  SessionOptionValue option_value;
+
+  SessionOption(std::string name, SessionOptionValue val)

Review Comment:
   Constructors with parameters must be marked `explicit`



##########
cpp/src/arrow/flight/sql/types.h:
##########
@@ -44,6 +44,23 @@ using SqlInfoResult =
 /// \brief Map SQL info identifier to its value.
 using SqlInfoResultMap = std::unordered_map<int32_t, SqlInfoResult>;
 
+/// \brief Variant supporting all possible types for SetSessionOption
+using SessionOptionValue =
+    std::variant<std::string, bool, int32_t, int64_t, float, double, std::vector<std::string>>;
+
+enum struct SessionOptionValueType : size_t {
+  kString, kBool, kInt32, kInt64, kFloat, kDouble, kStringList
+};
+
+struct ARROW_FLIGHT_SQL_EXPORT SessionOption {
+  std::string option_name;
+  SessionOptionValue option_value;
+
+  SessionOption(std::string name, SessionOptionValue val)
+      : option_name{ name }, option_value{ val } {}

Review Comment:
   Move when initializing



##########
cpp/src/arrow/flight/sql/types.h:
##########
@@ -44,6 +44,23 @@ using SqlInfoResult =
 /// \brief Map SQL info identifier to its value.
 using SqlInfoResultMap = std::unordered_map<int32_t, SqlInfoResult>;
 
+/// \brief Variant supporting all possible types for SetSessionOption
+using SessionOptionValue =
+    std::variant<std::string, bool, int32_t, int64_t, float, double, std::vector<std::string>>;

Review Comment:
   nit, but do we really need both 32 and 64 bit int and float?



##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -1072,6 +1260,18 @@ Status FlightSqlServerBase::EndTransaction(const ServerCallContext& context,
   return Status::NotImplemented("EndTransaction not implemented");
 }
 
+arrow::Result<ActionSetSessionOptionsResult> FlightSqlServerBase::SetSessionOptions (
+    const ServerCallContext& context,
+    const ActionSetSessionOptionsRequest& request) {
+  return Status::NotImplemented("SetSessionOptions not implemented");
+}

Review Comment:
   Put another way: how do I actually _implement_ this? I don't get anything identifying the session here.



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -430,6 +430,34 @@ std::string Location::scheme() const {
   return scheme;
 }
 
+std::string Location::path() const { return uri_->path(); }
+arrow::Result<std::vector<std::pair<std::string, std::string>>> Location::QueryItems()
+    const {
+  return uri_->query_items();
+}
+
+arrow::Result<std::vector<std::pair<std::string, std::string>>> Location::AsHeaders()
+    const {
+  std::string catalog = path();
+  if (catalog.empty()) {
+    return QueryItems();
+  }
+
+  std::vector<std::pair<std::string, std::string>> headers;
+
+  auto query_items_result = QueryItems();
+  if (!query_items_result.ok()) {

Review Comment:
   Use ARROW_ASSIGN_OR_RAISE?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   The concern was simply minimizing how much more stuff we're jamming into the Arrow package (and if it's not really necessary etc), however I think this is both important enough (to clarify the flow described above so we're not giving users/developers a puzzle to put together) and not more bulk added to the core code (Middleware being an optional helper) so it's probably worth adding an interface and impl to the package.
   
   Do the coding guidelines being used discourage MI?—by default I'm looking at implementing the core Middleware interface plus the session object getter via another interface that the new server session middleware would also inherit.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   The concern was simply minimizing how much more stuff we're jamming into the Arrow package (and if it's not really necessary etc), however I think this is both important enough (to clarify the flow described above so we're not giving users/developers a puzzle to put together) and not more bulk added to the core code (Middleware being an optional helper) so it's probably worth adding an interface and impl to the package.  Do the guidelines being used discourage MI?—by default I'm looking at implementing the core Middleware interface plus the session object getter via another interface that the new server session middleware would also inherit.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   ServerSessionMiddleware (builds but I am still getting tests sorted) has been added to the PR as discussed.  Please let me know if you have any thoughts on ways you'd prefer the mechanics to be different.  (ServerSessionMiddleware also needs a bit more documentation possibly, again comments welcome but I will be reviewing that anyways.)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOption {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  string option_name = 1;
+  oneof option_value {
+    string string_value = 2;
+    bool bool_value = 3;
+    sfixed32 int32_value = 4;
+    sfixed64 int64_value = 5;
+    float float_value = 6;
+    double double_value = 7;

Review Comment:
   Out of curiosity are there examples where having the smaller type is important?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOption {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  string option_name = 1;
+  oneof option_value {
+    string string_value = 2;
+    bool bool_value = 3;
+    sfixed32 int32_value = 4;
+    sfixed64 int64_value = 5;
+    float float_value = 6;
+    double double_value = 7;

Review Comment:
   Not right offhand—my understanding is that this is more or less expected of client/server configuration channels by most classical DB drivers.  James is unavailable presently but this is a valid question and I can follow up on that if desired.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,88 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+class ServerSessionMiddlewareFactory;
+
+static constexpr char const kSessionCookieName[] =
+    "flight_sql_session_id";
+
+class FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;

Review Comment:
   It doesn't but the decision was (consciously) made here (and I believe made by the STL designers when choosing what the more obvious 'map' type would be) on the basis that most of the tree-based std::map impls are faster than unordered_map (hash-based) for "up to around 10,000 entries".  (Performance issues with dense keys are also avoided by the former making the thing without unexpected degenerate performance cases the 'sane' default.)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;

Review Comment:
   Do we need a specific ok status if a session option is overwritten with a new value?
   It might be intentional to overwrite but can be one of those gotchas if unintentional and appeared to succeed.



##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with

Review Comment:
   2nd sentence doesn't quite make sense. Does the following convey the correct action:
   Send a NOT_FOUND error on subsequent requests.



##########
cpp/src/arrow/flight/client.h:
##########
@@ -383,6 +383,27 @@ class ARROW_FLIGHT_EXPORT FlightClient {
     return DoExchange({}, descriptor);
   }
 
+  /// \\brief Set server session option(s) by key/value. Sessions are generally
+  /// persisted via HTTP cookies.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The server session options to set
+  ::arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const FlightCallOptions& options, const SetSessionOptionsRequest& request);
+
+  /// \\brief Get the current server session options. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The GetSessionOptions request object.
+  ::arrow::Result<GetSessionOptionsResult> GetSessionOptions(
+      const FlightCallOptions& options, const GetSessionOptionsRequest& request);

Review Comment:
   GetSessionOptionsRequest is empty in the proto definition. Is it there for future expansion?



##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session cannot be set to the given value.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;

Review Comment:
   Minor wording correction: "The session *option* cannot be set."



##########
cpp/src/arrow/flight/client.h:
##########
@@ -383,6 +383,27 @@ class ARROW_FLIGHT_EXPORT FlightClient {
     return DoExchange({}, descriptor);
   }
 
+  /// \\brief Set server session option(s) by key/value. Sessions are generally
+  /// persisted via HTTP cookies.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The server session options to set
+  ::arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const FlightCallOptions& options, const SetSessionOptionsRequest& request);
+
+  /// \\brief Get the current server session options. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The GetSessionOptions request object.

Review Comment:
   Since this is an empty object, shouldn't the comment reflect its purpose, which is to do nothing?



##########
cpp/src/arrow/flight/client.h:
##########
@@ -383,6 +383,27 @@ class ARROW_FLIGHT_EXPORT FlightClient {
     return DoExchange({}, descriptor);
   }
 
+  /// \\brief Set server session option(s) by key/value. Sessions are generally
+  /// persisted via HTTP cookies.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The server session options to set
+  ::arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const FlightCallOptions& options, const SetSessionOptionsRequest& request);
+
+  /// \\brief Get the current server session options. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The GetSessionOptions request object.
+  ::arrow::Result<GetSessionOptionsResult> GetSessionOptions(
+      const FlightCallOptions& options, const GetSessionOptionsRequest& request);
+
+  /// \\brief Close/invalidate the current server session. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The CloseSession request object.
+  ::arrow::Result<CloseSessionResult> CloseSession(const FlightCallOptions& options,
+                                                   const CloseSessionRequest& request);

Review Comment:
   CloseSessionRequest is empty in the proto definition. Is it there for future expansion?



##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session cannot be set to the given value.

Review Comment:
   Minor wording correction: "The session *option* cannot be set to the given value."



##########
cpp/src/arrow/flight/client.h:
##########
@@ -383,6 +383,27 @@ class ARROW_FLIGHT_EXPORT FlightClient {
     return DoExchange({}, descriptor);
   }
 
+  /// \\brief Set server session option(s) by key/value. Sessions are generally
+  /// persisted via HTTP cookies.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The server session options to set
+  ::arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const FlightCallOptions& options, const SetSessionOptionsRequest& request);
+
+  /// \\brief Get the current server session options. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The GetSessionOptions request object.
+  ::arrow::Result<GetSessionOptionsResult> GetSessionOptions(
+      const FlightCallOptions& options, const GetSessionOptionsRequest& request);
+
+  /// \\brief Close/invalidate the current server session. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The CloseSession request object.

Review Comment:
   Should this be clearer that this has no functional impact?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/serialization_internal.cc:
##########
@@ -372,6 +372,174 @@ Status ToPayload(const FlightDescriptor& descr, std::shared_ptr<Buffer>* out) {
   return Status::OK();
 }
 
+// SessionOptionValue
+
+Status FromProto(const pb::SessionOptionValue& pb_val
+                                     SessionOptionValue* val) {
+  switch (pb_opt_val.option_value_case()) {
+    case pb::SessionOptionValue::OPTION_VALUE_NOT_SET:
+      return Status::Invalid("Unset option_value for name '" +
+                             pb_opt_name + "'");
+    case pb::SessionOptionValue::kStringValue:
+      val = pb_opt_val.string_value();
+      break;
+    case pb::SessionOptionValue::kBoolValue:
+      val = pb_opt_val.bool_value();
+      break;
+    case pb::SessionOptionValue::kInt32Value:
+      val = pb_opt_val.int32_value();
+      break;
+    case pb::SessionOptionValue::kInt64Value:
+      val = pb_opt_val.int64_value();
+      break;
+    case pb::SessionOptionValue::kFloatValue:
+      val = pb_opt_val.float_value();
+      break;
+    case pb::SessionOptionValue::kDoubleValue:
+      val = pb_opt_val.double_value();
+      break;
+    case pb::SessionOptionValue::kStringListValue:
+      val.emplace<std::vector<std::string>>();
+      std::get<std::vector<std::string>>(val)
+          .reserve(pb_opt_val.string_list_value().values_size());
+      for (const std::string& s : pb_opt_val.string_list_value().values())
+        std::get<std::vector<std::string>>(val).push_back(s);
+      break;
+  }
+  return Status::OK();
+}
+
+Status ToProto(const SessionOptionValue& val
+                                       pb::SessionOptionValue pb_val) {
+  std::visit(overloaded{
+      [&](std::string v) { pb_val.set_string_value(v); },
+      [&](bool v) { pb_val.set_bool_value(v); },
+      [&](int32_t v) { pb_val.set_int32_value(v); },
+      [&](int64_t v) { pb_val.set_int64_value(v); },
+      [&](float v) { pb_val.set_float_value(v); },
+      [&](double v) { pb_val.set_double_value(v); },
+      [&](std::vector<std::string> v) {
+        auto* string_list_value = pb_val.mutable_string_list_value();
+        for (const std::string& s : v)
+          string_list_value->add_values(s);
+      }
+    }, opt_value);
+  return Status::OK();
+}
+
+// map<string, SessionOptionValue>
+
+Status FromProto(const google::protobuf::map<string,
+                                             pb::SessionOptionValue> pb_map,   //PHOXME maybe need to include google/protobuf/map.h ?  shouldn't this be brought in by other headers?
+                 std::map<std::string, SessionOptionValue>* map) {
+  if (pb_map.size() == 0) {

Review Comment:
   Yes.  Fixed.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.cc:
##########
@@ -463,6 +463,300 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+// Helper for stringifying maps containing various types
+template <typename T>
+ostream& operator<<(std::map<std::string, T>) {
+  std::stringstream ss;
+
+  ss << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : session_options) {
+    std::cout << sep << '[' << k << "]: '" << v << "', ";  // PHOXME v.ToString() not implemented yet
+    sep = ", ";
+  }
+  ss << '}';
+
+  return ss.str();
+}
+static bool CompareSessionOptionMaps(
+    const std::map<std::string, SessionOptionValue>& a,
+    const std::map<std::string, SessionOptionValue>& b) {
+  if (a.session_options.size() != b.session_options.size()) {
+    return false;
+  }
+  for (const auto & [k, v] : a.session_options) {
+    if (!b.session_options.contains(k)) {
+      return false;
+    }
+    const auto& b_v = b.session_options[k];
+    if (v.index() != b_v.index()) {
+      return false;
+    }
+    if (v != b_v) {
+      return false;
+    }
+  }
+  return true;
+}
+
+// SetSessionOptionsRequest
+
+std::string SetSessionOptionsRequest::ToString() const {
+  std::stringstream ss;
+
+  ss << "<SetSessionOptionsRequest session_options="
+     << SessionOptionMapToString(session_options); << '>';
+
+  return ss.str();
+}
+
+bool SetSessionOptionsRequest::Equals(const SetSessionOptionsRequest& other) const {
+  return CompareSessionOptionMaps(session_options, other.session_options);
+}
+
+arrow::Result<std::string>
+SetSessionOptionsRequest::SerializeToString() const {
+  pb::SetSessionOptionsRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized SetSessionOptionsRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<SetSessionOptionsRequest>
+SetSessionOptionsRequest::Deserialize(std::string_view serialized) {
+  // TODO these & SerializeToString should all be factored out to a superclass
+  pb::SetSessionOptionsRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized SetSessionOptionsRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid SetSessionOptionsRequest");
+  }
+  SetSessionOptionsRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// SetSessionOptionsResult
+
+std::string SetSessionOptionsResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<SetSessionOptionsResult results=" << statuses << '>';
+
+  return ss.str();
+}
+
+bool SetSessionOptionsResult::Equals(const SetSessionOptionsResult& other) const {
+  if (statuses != other.statuses) {
+    return false;
+  }
+  return true;
+}
+
+arrow::Result<std::string>
+SetSessionOptionsResult::SerializeToString() const {
+  pb::SetSessionOptionsResult pb_result;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_result));
+
+  std::string out;
+  if (!pb_result.SerializeToString(&out)) {
+    return Status::IOError("Serialized SetSessionOptionsResult exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<SetSessionOptionsResult>
+SetSessionOptionsResult::Deserialize(std::string_view serialized) {
+  pb::SetSessionOptionsResult pb_result;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized SetSessionOptionsResult size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_result.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid SetSessionOptionsResult");
+  }
+  SetSessionOptionsResult out;
+  RETURN_NOT_OK(internal::FromProto(pb_result, &out));
+  return out;
+}
+
+// GetSessionOptionsRequest
+
+std::string GetSessionOptionsRequest::ToString() const {
+  return "<GetSessionOptionsRequest>";
+}
+
+bool GetSessionOptionsRequest::Equals(const GetSessionOptionsRequest& other) const {
+  return true;
+}
+
+arrow::Result<std::string>
+GetSessionOptionsRequest::SerializeToString() const {
+  pb::GetSessionOptionsRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized GetSessionOptionsRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<GetSessionOptionsRequest>
+GetSessionOptionsRequest::Deserialize(std::string_view serialized) {
+  pb::GetSessionOptionsRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized GetSessionOptionsRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid GetSessionOptionsRequest");
+  }
+  GetSessionOptionsRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// GetSessionOptionsResult
+
+std::string GetSessionOptionsResult::ToString() const {
+  std::stringstream ss;
+  
+  ss << "<GetSessionOptionsResult session_options=" << session_options << '>';
+
+  return ss.str();
+}
+
+bool GetSessionOptionsResult::Equals(const GetSessionOptionsResult& other) const {
+  return CompareSessionOptionMaps(session_options, other.session_options);
+}
+
+arrow::Result<std::string>
+GetSessionOptionsResult::SerializeToString() const {
+  pb::GetSessionOptionsResult pb_result;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_result));
+
+  std::string out;
+  if (!pb_result.SerializeToString(&out)) {
+    return Status::IOError("Serialized GetSessionOptionsResult exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<GetSessionOptionsResult>
+GetSessionOptionsResult::Deserialize(std::string_view serialized) {
+  pb::GetSessionOptionsResult pb_result;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized GetSessionOptionsResult size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_result.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid GetSessionOptionsResult");
+  }
+  GetSessionOptionsResult out;
+  RETURN_NOT_OK(internal::FromProto(pb_result, &out));
+  return out;
+}
+
+// CloseSessionRequest
+
+std::string CloseSessionRequest::ToString() const {
+  return "<CloseSessionRequest>";
+}
+
+bool CloseSessionRequest::Equals(const CloseSessionRequest& other) const {
+  return true;
+}
+
+arrow::Result<std::string>
+CloseSessionRequest::SerializeToString() const {
+  pb::CloseSessionRequest pb_request;
+  RETURN_NOT_OK(internal::ToProto(*this, &pb_request));
+
+  std::string out;
+  if (!pb_request.SerializeToString(&out)) {
+    return Status::IOError("Serialized CloseSessionRequest exceeded 2GiB limit");
+  }
+  return out;
+}
+
+arrow::Result<CloseSessionRequest>
+CloseSessionRequest::Deserialize(std::string_view serialized) {
+  pb::CloseSessionRequest pb_request;
+  if (serialized.size() > static_cast<size_t>(std::numeric_limits<int>::max())) {
+    return Status::Invalid(
+        "Serialized CloseSessionRequest size should not exceed 2 GiB");
+  }
+  google::protobuf::io::ArrayInputStream input(serialized.data(),
+                                               static_cast<int>(serialized.size()));
+  if (!pb_request.ParseFromZeroCopyStream(&input)) {
+    return Status::Invalid("Not a valid CloseSessionRequest");
+  }
+  CloseSessionRequest out;
+  RETURN_NOT_OK(internal::FromProto(pb_request, &out));
+  return out;
+}
+
+// CloseSessionResult
+
+std::string CloseSessionResult::ToString() const {
+  std::stringstream ss;
+
+  ss << "<CloseSessionResult result=" << status << '>';
+
+  return ss.str();
+}
+
+bool CloseSessionResult::Equals(const CloseSessionResult& other) const {

Review Comment:
   There should be a superclass for a bunch of these and templates for others; refactoring the existing code plus this PR's code is probably a separate task to get alignment on proper design there but agreed lots of things should be factored out, especially Deserialize/SerializeToString.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;
+    }
+
+    if (!session_id.length()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers,
+                                        session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid(
+            "Invalid or expired "
+            + static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session,
+    std::string session_id)
+    : factory_(factory), headers_(headers), session_(session), existing_session(true) {}

Review Comment:
   Fixed.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session cannot be set to the given value.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  map<string, Status> statuses = 1;

Review Comment:
   Thought about it but it probably makes the API more cumbersome; on the other hand I kind-of want this for things like SET_SESSION_OPTION_RESULT_OK_MAPPED where the remapped key could be provided in the response (if anyone would actually have a programmatic use case for that, but I guess IDE cases and so on—e.g. R, spark, etc.—would apply).



##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session cannot be set to the given value.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  map<string, Status> statuses = 1;

Review Comment:
   Thought about it but it probably makes the API more cumbersome; on the other hand I kind-of want this for things like SET_SESSION_OPTION_RESULT_OK_MAPPED where the remapped key could be provided in the response (if anyone would actually have a programmatic use case for that, but I guess IDE cases and so on—e.g. R, spark, etc.—would apply).



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");

Review Comment:
   Not valid though: both are const char[].  Probably more relevant to static_cast the first literal though, what I'm doing is a bit weird too (fixed build error late at night and didn't think very hard about the types other than that of kSessionCookieName).



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   > One other thought: we could standardize a few names of common session options here, in particular 'current catalog' and 'current schema', since I believe that was the original goal way back when
   
   Thoughts on where to put this—Proto defs maybe?
   
   Related, (a) we were also discussing maybe adding (later) a verb to list available option keys, and (b) possibly (I think maybe not—your opinion?) documenting the recommended practice of setting options before other (e.g. getFI) calls and all-at-once, but I think that's pretty implementation-dependant and probably doesn't belong in Arrow unlike the suggested option key conventions...


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   > As a https://github.com/apache/arrow-flight-sql-postgresql developer, I want to create a new session only when authentication is succeeded. Can we use this feature for this use case?
   
   Not 100% clear if you want to trigger session creation on successful auth (it is "normally" triggered on DoAction/SetSessionOptions calls but with a lot of flexibility), or gate allowing a client to explicitly trigger session creation on their AuthNZ status, but both of these should be trivially feasible.  You may end up needing a shim around the RPC handlers (we have similar for similar reasons) to relate the call-local middleware instances (IIRC they can't look one another up?), at which point either of them could be authoritative in taking action to deliberately create or allow creation of a session (depending on your meaning above).


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   Session scope and lifetime is now documented as being implementation-defined.  Clients need not care about the close session status if they don't want to, as the default is indeed best-effort; however, in cases where e.g. session share auth state, clients are also free to care and continue to retry until success or NOT_CLOSABLE in which case they might notify the user (this is effectively a warning not an error).



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   C++ middleware are "fake" due to gRPC design issues in C++ so there might not be a way to work around this. https://github.com/apache/arrow/blob/3b73f438bf0a1abc76009f017c1df9c244854aca/cpp/src/arrow/flight/transport/grpc/grpc_server.cc#L321-L344


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   Not thinking about the DoExchange case yet, but as far as DoAction, I believe that (I'm assuming we're talking about also eliminating the implicit callback at :515 for sanity?) moving the SendingHeaders call to after the handler call at :522 is exactly the solution needed.  I looked into the call path down from :515 and there's a lot of fanout and so far I'm not following the semantics perfectly—I can spend some more time on that unless you have some minutes to do the chainsaw work there and see if it flies?  I can rebase against an experimental commit of the and confirm whether it makes the middleware callback order correct (as I fully expect it would, but just to be sure..).   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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   Also not sure if there's a clean solution to the race around triggering that callback exactly once and prior to the first call passing through to the transport-layer listener, without making them all blocking... or maybe have an explicit (new) call to the stream listener interposer to disable the blocking behaviour which would also trigger the middleware callback.  But that's ~breaking... hm.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   Basically
   
   https://github.com/apache/arrow/blob/3fe598ae4dfd7805ab05452dd5ed4b0d6c97d8d5/cpp/src/arrow/flight/transport/grpc/grpc_server.cc#L513-L544
   
   Here I think it would be fine to manually call SendingHeaders at L528 (where currently it's implicitly called at L515). I think that solves your problem?
   
   https://github.com/apache/arrow/blob/3fe598ae4dfd7805ab05452dd5ed4b0d6c97d8d5/cpp/src/arrow/flight/transport/grpc/grpc_server.cc#L490-L499
   
   Here I think it must still be called before L496.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   These have mostly been mirrored from earlier similar definitions, I believe originally suggested by @emkornfield 



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   > Thanks for the suggestions. I haven't try this yet but can I reuse `ServerSessionMiddleware` for my case? Or should I implement a similar middleware from scratch?
   
   There are arguments either way, but I would think for a more core component of another project it would make sense to write something anew even if you borrow from what's been done for ServerSessionMiddleware:
   
   * It would make more sense to use a more polished (library) implementation for the server-side cookie stuff
   * At some point you're going to have to relate the auth and cookie middleware if you want to use a common session cookie (no reason to separate them if the lifespans are indeed identical), which means one of (a) a shim around the handlers, (b) a wrapper around both middlewares that relate them, or (c) combining the two middlewares to have a common session middleware that handles auth as well as options.
   * At present there is no callback facility for e.g. session option validation, and a richer use case probably needs things like that tied into the app implementation rather than just an open-ended K/V store that the app can interpret async from the SetSessionOptions/etc. calls.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;

Review Comment:
   What is the point of this? Why would the client retry closing if it's currently closing?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;

Review Comment:
   Why all these alternatives? Why not simply `int_value` and `double_value`?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   Which earlier similar definitions? I think closing should always be successful. What is the use case for a session that can't be closed?
   



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;

Review Comment:
   https://protobuf.dev/programming-guides/style/#enums
   https://buf.build/docs/best-practices/style-guide#enums
   https://github.com/uber/prototool/blob/dev/style/README.md#enums



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.

Review Comment:
   > The documentation here can be reworded: Does UCX have its own "State Management Mechanism" as it were, to persist such things across its individual UCP "calls", which we could generalize the wording here to capture?
   
   I have no idea :-)
   
   If this snippet of doc is gRPC-specific, it should be mentioned. Also mention whether the client is supposed to send the cookie for each request intended to fall into the same session.
   
   Also should mention whether the server is supposed to re-send the client the same cookie, or can send another cookie as a reply, or can send no cookie at all.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   One other thought: we could standardize a few names of common session options here, in particular 'current catalog' and 'current schema', since I believe that was the original goal way back when


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/serialization_internal.cc:
##########
@@ -376,6 +384,161 @@ Status ToPayload(const FlightDescriptor& descr, std::shared_ptr<Buffer>* out) {
   return Status::OK();
 }
 
+// SessionOptionValue
+
+Status FromProto(const pb::SessionOptionValue& pb_val, SessionOptionValue* val) {
+  switch (pb_val.option_value_case()) {
+    case pb::SessionOptionValue::OPTION_VALUE_NOT_SET:
+      return Status::Invalid("Unset SessionOptionValue found");
+    case pb::SessionOptionValue::kStringValue:
+      *val = pb_val.string_value();
+      break;
+    case pb::SessionOptionValue::kBoolValue:
+      *val = pb_val.bool_value();
+      break;
+    case pb::SessionOptionValue::kInt32Value:
+      *val = pb_val.int32_value();
+      break;
+    case pb::SessionOptionValue::kInt64Value:
+      *val = pb_val.int64_value();
+      break;
+    case pb::SessionOptionValue::kFloatValue:
+      *val = pb_val.float_value();
+      break;
+    case pb::SessionOptionValue::kDoubleValue:
+      *val = pb_val.double_value();
+      break;
+    case pb::SessionOptionValue::kStringListValue:
+      (*val).emplace<std::vector<std::string>>();
+      std::get<std::vector<std::string>>(*val).reserve(
+          pb_val.string_list_value().values_size());
+      for (const std::string& s : pb_val.string_list_value().values())
+        std::get<std::vector<std::string>>(*val).push_back(s);

Review Comment:
   Done and done.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.cc:
##########
@@ -463,6 +463,318 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {
+  os << CloseSessionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+// Helpers for stringifying maps containing various types
+std::ostream& operator<<(std::ostream& os, std::vector<std::string> v) {
+  os << '[';
+  std::string sep = "";
+  for (const auto& x : v) {
+    os << sep << '"' << x << '"';
+    sep = ", ";
+  }
+  os << ']';
+
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SessionOptionValue& v) {
+  std::visit([&](const auto& x) { os << x; }, v);
+  return os;
+}
+
+template <typename T>
+std::ostream& operator<<(std::ostream& os, std::map<std::string, T> m) {
+  os << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : m) {
+    os << sep << '[' << k << "]: '" << v;
+    sep = ", ";
+  }
+  os << '}';
+
+  return os;
+}
+
+namespace {
+static bool CompareSessionOptionMaps(const std::map<std::string, SessionOptionValue>& a,
+                                     const std::map<std::string, SessionOptionValue>& b) {
+  if (a.size() != b.size()) {
+    return false;
+  }
+  for (const auto& [k, v] : a) {
+    if (!b.count(k)) {
+      return false;
+    }
+    try {
+      const auto& b_v = b.at(k);
+      if (v.index() != b_v.index()) {
+        return false;
+      }
+      if (v != b_v) {
+        return false;
+      }
+    } catch (const std::out_of_range& e) {
+      return false;
+    }

Review Comment:
   Done.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/client.h:
##########
@@ -383,6 +383,27 @@ class ARROW_FLIGHT_EXPORT FlightClient {
     return DoExchange({}, descriptor);
   }
 
+  /// \\brief Set server session option(s) by key/value. Sessions are generally
+  /// persisted via HTTP cookies.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The server session options to set
+  ::arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const FlightCallOptions& options, const SetSessionOptionsRequest& request);
+
+  /// \\brief Get the current server session options. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The GetSessionOptions request object.
+  ::arrow::Result<GetSessionOptionsResult> GetSessionOptions(
+      const FlightCallOptions& options, const GetSessionOptionsRequest& request);
+
+  /// \\brief Close/invalidate the current server session. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The CloseSession request object.

Review Comment:
   Seems somewhat redundant but it won't hurt, so added to both calls in flight/client.h and flight/sql/client.h.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/client.h:
##########
@@ -383,6 +383,27 @@ class ARROW_FLIGHT_EXPORT FlightClient {
     return DoExchange({}, descriptor);
   }
 
+  /// \\brief Set server session option(s) by key/value. Sessions are generally
+  /// persisted via HTTP cookies.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The server session options to set
+  ::arrow::Result<SetSessionOptionsResult> SetSessionOptions(
+      const FlightCallOptions& options, const SetSessionOptionsRequest& request);
+
+  /// \\brief Get the current server session options. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The GetSessionOptions request object.
+  ::arrow::Result<GetSessionOptionsResult> GetSessionOptions(
+      const FlightCallOptions& options, const GetSessionOptionsRequest& request);
+
+  /// \\brief Close/invalidate the current server session. The session is generally
+  /// accessed via an HTTP cookie.
+  /// \param[in] options Per-RPC options
+  /// \param[in] request The CloseSession request object.

Review Comment:
   Then wouldn't a comment indicating this be appropriate?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;

Review Comment:
   The pairs in the vector as well as the internals of the pair members are already (internally) mutable so we can do this.  Kind of breaks the "set once const" paradigm but it's a very small block of code so not too dangerous—also reminds me that I should be using const more than once for loop/argument refs ;)  Moved it to the end of the block (clearer move context & saves the wasteful construction in the error case!), too.



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;

Review Comment:
   The pairs in the vector as well as the internals of the pair members are already (internally) mutable so we can do this.  Kind of breaks the "set once const" paradigm but it's a very small block of code so not too dangerous—also reminds me that I should be using const more than once for loop/argument refs ;)  Moved it to the end of the block (clearer move context & saves the wasteful construction in the error case!), too.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid("Invalid or expired " +
+                               static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session, std::string session_id)
+    : factory_(factory),
+      headers_(headers),
+      session_(std::move(session)),
+      session_id_(std::move(session_id)),
+      existing_session(true) {}
+
+void ServerSessionMiddleware::SendingHeaders(AddCallHeaders* addCallHeaders) {
+  if (!existing_session && session_) {
+    addCallHeaders->AddHeader(
+        "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+  }
+}
+
+void ServerSessionMiddleware::CallCompleted(const Status&) {}
+
+bool ServerSessionMiddleware::HasSession() const { return static_cast<bool>(session_); }
+
+std::shared_ptr<FlightSqlSession> ServerSessionMiddleware::GetSession() {
+  if (!session_) session_ = factory_->GetNewSession(&session_id_);
+  return session_;
+}
+
+const CallHeaders& ServerSessionMiddleware::GetCallHeaders() const { return headers_; }
+
+std::shared_ptr<ServerMiddlewareFactory> MakeServerSessionMiddlewareFactory() {
+  return std::shared_ptr<ServerSessionMiddlewareFactory>(
+      new ServerSessionMiddlewareFactory());

Review Comment:
   Fixed



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   Hey @indigophox , my comment was just to make the Visitor have a generic return type. I wasn't thinking of making SessionOptionValue use generics.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // the same session produce a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the

Review Comment:
   > I think if I proposed this my rationale was long lived distributed operations can typically be long lived or hard to interrupt immediately. IIUC sessions seems like they unlikely to have distributed work associated with them (unless we are hoping to do things like unclose a session?)
   
   If for any reason someone uses sessions to persist client state to doGet, or for AuthN persistence, etc., or uses them in a distributed context in general then closing them is also possibly a distributed operation depending how they're backed.  If they're used for AuthN in particular (as @kou has suggested doing for his purposes) a client/user may want to confirm their (distributed) invalidation.  Also not very expensive to have a couple if enum values in-case here.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   OTOH Java still doesn't really support specialization to an array type (that I know of?) so I'm not sure we can build this on top of Java generics.  All ears if there's a way to make Java do the thing.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);
+
+    Flight.SessionOptionValue toProtocol() {
+        Flight.SessionOptionValue.Builder b = Flight.SessionOptionValue.newBuilder();
+        SessionOptionValueToProtocolVisitor visitor = new SessionOptionValueToProtocolVisitor(b);
+        this.visit(visitor);
+        return b.build();
+    }
+}
+
+class SessionOptionValueToProtocolVisitor implements SessionOptionValueVisitor {
+    final Flight.SessionOptionValue.Builder b;
+
+    SessionOptionValueVisitor(Flight.SessionOptionValue.Builder b) { this.b = b; }
+
+    Flight.SessionOptionValue visit(String value) {
+        b.setStringValue(value);

Review Comment:
   Or not.  Whoops: leftover from a different thought on how to design that interaction.  Now they're void :)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;

Review Comment:
   Originally these types were chosen for consistency with APIs such as ODBC and JDBC. It'd be reasonable to prune these down to 64-bit integers and 64-bit doubles 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;

Review Comment:
   > I wonder about the strong typing and how it would actually be used between a client and server without having both of them know in advance what the type is?
   > 
   > In the context of URL parsing, if we allow for passing extra options as query parameters, all of them would be untyped so how the client would be able to provide them to the server with the correct type?
   
   Very good point here.  I think the best behaviour is perhaps to have servers allow preferred-type-OR-string, and native Flight clients get the benefit of strongly-typed options, while other e.g. *DBC clients that have an interface that precludes passing in typed options (e.g. connection string parameters) can defer that to the server.  This should probably also be documented in the Protocol specification as the preferred practice.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -423,6 +494,91 @@ arrow::Result<Result> PackActionResult(ActionCreatePreparedStatementResult resul
   return PackActionResult(pb_result);
 }
 
+arrow::Result<Result> PackActionResult(ActionSetSessionOptionsResult result) {
+  pb::sql::ActionSetSessionOptionsResult pb_result;
+  for (SetSessionOptionResult& res : result.results) {
+    switch (res) {
+      case SetSessionOptionResult::kUnspecified:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_UNSPECIFIED);
+        break;
+      case SetSessionOptionResult::kOk:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_OK);
+        break;
+      case SetSessionOptionResult::kInvalidResult:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_INVALID_VALUE);
+        break;
+      case SetSessionOptionResult::kError:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_ERROR);
+        break;
+    }
+  }
+  return PackActionResult(pb_result);
+}
+
+arrow::Result<Result> PackActionResult(ActionGetSessionOptionsResult result) {
+  pb::sql::ActionGetSessionOptionsResult pb_result;
+  for (const SessionOption& in_opt : result.session_options) {
+    pb::sql::SessionOption& opt = *pb_result.add_session_options();

Review Comment:
   > The version of the Google C++ guidelines that Arrow was developed with prefers `T*` to `T&`.
   
   Can do.  Always believed the consistency of `.` vs `->` was useful, but I will update accordingly, 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server.cc:
##########
@@ -423,6 +494,91 @@ arrow::Result<Result> PackActionResult(ActionCreatePreparedStatementResult resul
   return PackActionResult(pb_result);
 }
 
+arrow::Result<Result> PackActionResult(ActionSetSessionOptionsResult result) {
+  pb::sql::ActionSetSessionOptionsResult pb_result;
+  for (SetSessionOptionResult& res : result.results) {
+    switch (res) {
+      case SetSessionOptionResult::kUnspecified:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_UNSPECIFIED);
+        break;
+      case SetSessionOptionResult::kOk:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_OK);
+        break;
+      case SetSessionOptionResult::kInvalidResult:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_INVALID_VALUE);
+        break;
+      case SetSessionOptionResult::kError:
+        pb_result.add_results(
+            pb::sql::ActionSetSessionOptionsResult::SET_SESSION_OPTION_RESULT_ERROR);
+        break;
+    }
+  }
+  return PackActionResult(pb_result);
+}
+
+arrow::Result<Result> PackActionResult(ActionGetSessionOptionsResult result) {
+  pb::sql::ActionGetSessionOptionsResult pb_result;
+  for (const SessionOption& in_opt : result.session_options) {
+    pb::sql::SessionOption& opt = *pb_result.add_session_options();

Review Comment:
   > Can do. Always believed the consistency of . vs -> was useful, but I will update accordingly, thanks.
   
   Yeah, it would be nice, but intra-codebase consistency is more valuable for now. (The Google C++ guidelines have since updated to favor the reference in all cases.) But changing this for us would mean hard compatibility breaks in many, many APIs (and it would likely be possible to fix as we've been trying to favor arrow::Result over out parameters _anyways_, but not sure if anyone is driving that refactoring.)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/types.h:
##########
@@ -409,6 +409,15 @@ struct ARROW_FLIGHT_EXPORT Location {
   /// \brief Get the scheme of this URI.
   std::string scheme() const;
 
+  /// \brief Get the path of this URI.
+  std::string path() const;
+
+  /// \brief Get the query parameters of this URI.
+  arrow::Result<std::vector<std::pair<std::string, std::string>>> QueryItems() const;
+
+  /// \brief Convert URI path and parameters to headers.
+  arrow::Result<std::vector<std::pair<std::string, std::string>>> AsHeaders() const;

Review Comment:
   Circling back around on this one, given it's part of the Location state/semantics I'm really inclined to have it kept here.  Additionally if anyone wanted to use the consistent Location behaviour for a Flight non-SQL impl it's best not to have them "borrowing" static stuff from fSQL, reimplementing it, or some other hacky means of getting at something that's intrinsically part of the Location object.  (If that's still an issue perhaps we can discuss briefly as I'm also sending an e-mail OOB somewhat related to 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] indigophox commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages, Location URI path accessors

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(

Review Comment:
   I think having it included as non-test code is really useful for "get up and running" use, so I'll have a look at how much refactoring of the cookie_internals stuff would be needed to make this shared.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on pull request #34817: Add Session management messages, Location URI path accessors

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

   I believe we discussed using headers as the base/fallback implementation; how does this integrate into that?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
format/FlightSql.proto:
##########
@@ -1842,6 +1842,94 @@ message ActionCancelQueryResult {
   CancelResult result = 1;
 }
 
+/*
+ * Request message for the "Close Session" action.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOption {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  string option_name = 1;
+  oneof option_value {
+    string string_value = 2;
+    bool bool_value = 3;
+    sfixed32 int32_value = 4;
+    sfixed64 int64_value = 5;
+    float float_value = 6;
+    double double_value = 7;

Review Comment:
   Ok, interesting. I'll accept it, but I'd be curious to know the answer eventually if 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
cpp/src/arrow/flight/sql/types.h:
##########


Review Comment:
   I suppose these types can be moved to flight/types.h?



##########
format/Flight.proto:
##########
@@ -360,3 +361,96 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message ActionCloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message ActionCloseSessionResult {
+  option (experimental) = true;
+
+  enum CloseSessionResult {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  CloseSessionResult result = 1;
+}
+
+message SessionOptionValue {

Review Comment:
   Missing docstrings?



##########
format/Flight.proto:
##########
@@ -360,3 +361,96 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message ActionCloseSessionRequest {

Review Comment:
   Can we be consistent with the existing messages? https://github.com/apache/arrow/blob/main/format/Flight.proto#L191



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] lidavidm commented on a diff in pull request #34817: GH-34865: [C++][Flight RPC] Add Session management messages

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


##########
cpp/src/arrow/flight/sql/types.h:
##########


Review Comment:
   Look at what's been done for other types in Flight.proto.
   
   We cannot expose Protobuf so there is no "touch the Proto types anyways".



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;

Review Comment:
   I think there are various aspects of set-only-if-unset requests or set-once values or whatever that could be done as extension work here, but I'm not sure we need this richer as-is and I don't think we're boxed out of extending the request and response semantics to provide this later (correct me if you see something we might not be able to accomplish as later work).



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/types.cc:
##########
@@ -468,6 +468,323 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {
+  os << CloseSessionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+// Helpers for stringifying maps containing various types
+std::ostream& operator<<(std::ostream& os, std::vector<std::string> v) {
+  os << '[';
+  std::string sep = "";
+  for (const auto& x : v) {
+    os << sep << '"' << x << '"';
+    sep = ", ";
+  }
+  os << ']';
+
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SessionOptionValue& v) {
+  std::visit([&](const auto& x) { os << x; }, v);
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionsResult::Result& r) {
+  os << '{' << r.status << '}';
+  return os;
+}
+
+template <typename T>
+std::ostream& operator<<(std::ostream& os, std::map<std::string, T> m) {
+  os << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : m) {
+    os << sep << '[' << k << "]: '" << v;
+    sep = ", ";
+  }
+  os << '}';
+
+  return os;
+}
+
+namespace {
+static bool CompareSessionOptionMaps(const std::map<std::string, SessionOptionValue>& a,
+                                     const std::map<std::string, SessionOptionValue>& b) {
+  if (a.size() != b.size()) {
+    return false;
+  }
+  for (const auto& [k, v] : a) {
+    if (!b.count(k)) {

Review Comment:
   Actually you should be able to just delete this whole if block entirely since line 527 makes it redundant.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));

Review Comment:
   Done.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,87 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+class ServerSessionMiddlewareFactory;
+
+static constexpr char const kSessionCookieName[] = "flight_sql_session_id";
+
+class FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+
+ public:
+  /// \brief Get session option by key
+  ::arrow::Result<SessionOptionValue> GetSessionOption(const std::string&);
+  /// \brief Set session option by key to given value
+  void SetSessionOption(const std::string&, const SessionOptionValue&);

Review Comment:
   Fixed.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));

Review Comment:
   Slightly bigger than that, going to need to have a public interface class and pimpl ServerSessionMiddlewareImpl so the constructor is public to make_shared :)



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));

Review Comment:
   This will require turning the class in the header into an abstract class as you can't friend shared_ptr's internals.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,207 @@
+// 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 <mutex>
+
+#include "arrow/flight/sql/server_session_middleware.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  std::function<std::string()> id_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s);
+
+ public:
+  explicit ServerSessionMiddlewareFactory(std::function<std::string()> id_gen)
+      : id_generator_(id_gen) {}
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware);
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::pair<std::string, std::shared_ptr<FlightSqlSession>> GetNewSession();
+};
+
+class ServerSessionMiddlewareImpl : public ServerSessionMiddleware {
+ protected:
+  std::shared_mutex lock_;
+  ServerSessionMiddlewareFactory* factory_;
+  const CallHeaders& headers_;
+  std::shared_ptr<FlightSqlSession> session_;
+  std::string session_id_;
+  const bool existing_session;
+
+ public:
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers)
+      : factory_(factory), headers_(headers), existing_session(false) {}
+
+  ServerSessionMiddlewareImpl(ServerSessionMiddlewareFactory* factory,
+                              const CallHeaders& headers,
+                              std::shared_ptr<FlightSqlSession> session,
+                              std::string session_id)
+      : factory_(factory),
+        headers_(headers),
+        session_(std::move(session)),
+        session_id_(std::move(session_id)),
+        existing_session(true) {}
+
+  void SendingHeaders(AddCallHeaders* add_call_headers) override {
+    if (!existing_session && session_) {
+      add_call_headers->AddHeader(
+          "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+    }
+  }
+
+  void CallCompleted(const Status&) override {}
+
+  bool HasSession() const override { return static_cast<bool>(session_); }
+
+  std::shared_ptr<FlightSqlSession> GetSession() override {
+    const std::shared_lock<std::shared_mutex> l(lock_);
+    if (!session_) {
+      auto [id, s] = factory_->GetNewSession();
+      session_ = std::move(s);
+      session_id_ = std::move(id);
+    }
+    return session_;
+  }
+
+  const CallHeaders& GetCallHeaders() const override { return headers_; }
+};
+
+std::vector<std::pair<std::string, std::string>>
+ServerSessionMiddlewareFactory::ParseCookieString(const std::string_view& s) {
+  const std::string list_sep = "; ";
+  const std::string pair_sep = "=";
+  const size_t pair_sep_len = pair_sep.length();
+
+  std::vector<std::pair<std::string, std::string>> result;
+
+  size_t cur = 0;
+  while (cur < s.length()) {
+    const size_t end = s.find(list_sep, cur);
+    size_t len;
+    if (end == std::string::npos) {
+      // No (further) list delimiters
+      len = std::string::npos;
+      cur = s.length();
+    } else {
+      len = end - cur;
+      cur = end;
+    }
+    const std::string_view tok = s.substr(cur, len);
+
+    const size_t val_pos = tok.find(pair_sep);
+    if (val_pos == std::string::npos) {
+      // The cookie header is somewhat malformed; ignore the key and continue parsing
+      continue;
+    }
+    result.emplace_back(tok.substr(0, val_pos),
+                        tok.substr(val_pos + pair_sep_len, std::string::npos));
+  }

Review Comment:
   I do now see the error directive when building:
   ```
   [44/400] Building CXX object _deps/googletest-build/googletest/CMakeFiles/gtest.dir/src/gtest-all.cc.o
   ```
   
   However just the above-pasted errors on the step for _deps/googletest-build/googlemock/CMakeFiles/gmock.dir/src/gmock-all.cc.o, so it's included in some places, but apparently not there as you figured.



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   > I think if you put any authentication middleware first, then it can reject unauthenticated requests and you can always create a session after that.
   
   As long as the auth & session interaction is happening within a gRPC call where there can be a shim that relates the two middlewares (or just integrate it all into one compound middleware) it would be simple enough to extend the provided middleware to have a function to tell it to (from the shim/other middleware) create a session whether or not one was otherwise triggered by the client, and do this based on the authNZ state of the other middlware.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;

Review Comment:
   More clear if this is !session_id.empty()



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   @github-actions autotune


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   @lidavidm are we ok on the AMD64 Windows MinGW build workflow falling over on what looks like an unrelated issue?  Maybe preexisting(?)
   
   `CMake Error at C:/msys64/ucrt64/share/cmake/Modules/FindPackageHandleStandardArgs.cmake:230 (message):
     Could NOT find LLVMAlt (missing: LLVM_PACKAGE_VERSION CLANG_EXECUTABLE
     LLVM_FOUND LLVM_LINK_EXECUTABLE)
   Call Stack (most recent call first):
     C:/msys64/ucrt64/share/cmake/Modules/FindPackageHandleStandardArgs.cmake:600 (_FPHSA_FAILURE_MESSAGE)
     cmake_modules/FindLLVMAlt.cmake:118 (find_package_handle_standard_args)
     src/gandiva/CMakeLists.txt:28 (find_package)
   
   
   -- Configuring incomplete, errors occurred!
   Error: Process completed with exit code 1.`
   
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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

   I would be OK removing the `experimental` tag (and possibly just documenting it in the docstring instead). The Protobuf setup we have is quite fragile and it would be some significant effort to get that working, I believe.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;

Review Comment:
   What's the use of this?



##########
format/Flight.proto:
##########
@@ -503,3 +504,100 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+  option (experimental) = true;
+}
+
+/*
+ * The result of closing a session.
+ *
+ * The result should be wrapped in a google.protobuf.Any message.
+ */
+message CloseSessionResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;
+    // The session close request is complete. Subsequent requests with
+    // a NOT_FOUND error.
+    CLOSE_RESULT_CLOSED = 1;
+    // The session close request is in progress. The client may retry
+    // the close request.
+    CLOSE_RESULT_CLOSING = 2;
+    // The session is not closeable. The client should not retry the
+    // close request.
+    CLOSE_RESULT_NOT_CLOSEABLE = 3;
+  }
+
+  Status status = 1;
+}
+
+message SessionOptionValue {
+  option (experimental) = true;
+
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+message SetSessionOptionsRequest {
+  option (experimental) = true;
+
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+message SetSessionOptionsResult {
+  option (experimental) = true;
+
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session cannot be set to the given value.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  map<string, Status> statuses = 1;

Review Comment:
   Would it make sense to provide a full message as the value type so we can also insert an error message?



##########
cpp/src/gandiva/gdv_hash_function_stubs.cc:
##########


Review Comment:
   Did you mean to touch these files?



##########
cpp/src/arrow/flight/types.h:
##########
@@ -742,6 +746,164 @@ struct ARROW_FLIGHT_EXPORT CancelFlightInfoRequest {
   static arrow::Result<CancelFlightInfoRequest> Deserialize(std::string_view serialized);
 };
 
+/// \brief Variant supporting all possible value types for {Set,Get}SessionOptions
+using SessionOptionValue = std::variant<std::string, bool, int32_t, int64_t, float,
+                                        double, std::vector<std::string>>;
+
+/// \brief The result of setting a session option.
+enum class SetSessionOptionStatus : int8_t {
+  kUnspecified,

Review Comment:
   It would be good to have the enum cases documented, too



##########
cpp/src/arrow/flight/types.cc:
##########
@@ -463,6 +463,318 @@ arrow::Result<CancelFlightInfoRequest> CancelFlightInfoRequest::Deserialize(
   return out;
 }
 
+std::ostream& operator<<(std::ostream& os, const SetSessionOptionStatus& r) {
+  os << SetSessionOptionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const CloseSessionStatus& r) {
+  os << CloseSessionStatusNames[static_cast<int>(r)];
+  return os;
+}
+
+// Helpers for stringifying maps containing various types
+std::ostream& operator<<(std::ostream& os, std::vector<std::string> v) {
+  os << '[';
+  std::string sep = "";
+  for (const auto& x : v) {
+    os << sep << '"' << x << '"';
+    sep = ", ";
+  }
+  os << ']';
+
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const SessionOptionValue& v) {
+  std::visit([&](const auto& x) { os << x; }, v);
+  return os;
+}
+
+template <typename T>
+std::ostream& operator<<(std::ostream& os, std::map<std::string, T> m) {
+  os << '{';
+  std::string sep = "";
+  for (const auto& [k, v] : m) {
+    os << sep << '[' << k << "]: '" << v;
+    sep = ", ";
+  }
+  os << '}';
+
+  return os;
+}
+
+namespace {
+static bool CompareSessionOptionMaps(const std::map<std::string, SessionOptionValue>& a,
+                                     const std::map<std::string, SessionOptionValue>& b) {
+  if (a.size() != b.size()) {
+    return false;
+  }
+  for (const auto& [k, v] : a) {
+    if (!b.count(k)) {
+      return false;
+    }
+    try {
+      const auto& b_v = b.at(k);
+      if (v.index() != b_v.index()) {
+        return false;
+      }
+      if (v != b_v) {
+        return false;
+      }
+    } catch (const std::out_of_range& e) {
+      return false;
+    }

Review Comment:
   You could use `find` to confirm whether the element is in the map, and then access it if so, without having to use a try-catch with `at` (which should be redundant anyways?)



##########
cpp/src/arrow/flight/client.cc:
##########
@@ -713,6 +713,49 @@ arrow::Result<FlightClient::DoExchangeResult> FlightClient::DoExchange(
   return result;
 }
 
+::arrow::Result<SetSessionOptionsResult> FlightClient::SetSessionOptions(
+    const FlightCallOptions& options, const SetSessionOptionsRequest& request) {
+  RETURN_NOT_OK(CheckOpen());
+  RETURN_NOT_OK(CheckOpen());

Review Comment:
   duplicated (here and below)



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {

Review Comment:
   In general Arrow avoids using exceptions. You can always use `find` to check whether an item exists and then access it.



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");

Review Comment:
   ```suggestion
               return Status::Invalid("Empty ", kSessionCookieName, " cookie value");
   ```



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid("Invalid or expired " +
+                               static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session, std::string session_id)
+    : factory_(factory),
+      headers_(headers),
+      session_(std::move(session)),
+      session_id_(std::move(session_id)),
+      existing_session(true) {}
+
+void ServerSessionMiddleware::SendingHeaders(AddCallHeaders* addCallHeaders) {
+  if (!existing_session && session_) {
+    addCallHeaders->AddHeader(
+        "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+  }
+}
+
+void ServerSessionMiddleware::CallCompleted(const Status&) {}
+
+bool ServerSessionMiddleware::HasSession() const { return static_cast<bool>(session_); }
+
+std::shared_ptr<FlightSqlSession> ServerSessionMiddleware::GetSession() {
+  if (!session_) session_ = factory_->GetNewSession(&session_id_);
+  return session_;
+}
+
+const CallHeaders& ServerSessionMiddleware::GetCallHeaders() const { return headers_; }
+
+std::shared_ptr<ServerMiddlewareFactory> MakeServerSessionMiddlewareFactory() {
+  return std::shared_ptr<ServerSessionMiddlewareFactory>(
+      new ServerSessionMiddlewareFactory());

Review Comment:
   Use make_shared



##########
cpp/src/arrow/flight/sql/client.h:
##########
@@ -18,6 +18,7 @@
 #pragma once
 
 #include <cstdint>
+#include <map>

Review Comment:
   nit: the header isn't directly used here



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid("Invalid or expired " +
+                               static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session, std::string session_id)
+    : factory_(factory),
+      headers_(headers),
+      session_(std::move(session)),
+      session_id_(std::move(session_id)),
+      existing_session(true) {}
+
+void ServerSessionMiddleware::SendingHeaders(AddCallHeaders* addCallHeaders) {
+  if (!existing_session && session_) {
+    addCallHeaders->AddHeader(
+        "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+  }
+}
+
+void ServerSessionMiddleware::CallCompleted(const Status&) {}
+
+bool ServerSessionMiddleware::HasSession() const { return static_cast<bool>(session_); }
+
+std::shared_ptr<FlightSqlSession> ServerSessionMiddleware::GetSession() {
+  if (!session_) session_ = factory_->GetNewSession(&session_id_);
+  return session_;
+}
+
+const CallHeaders& ServerSessionMiddleware::GetCallHeaders() const { return headers_; }
+
+std::shared_ptr<ServerMiddlewareFactory> MakeServerSessionMiddlewareFactory() {
+  return std::shared_ptr<ServerSessionMiddlewareFactory>(
+      new ServerSessionMiddlewareFactory());
+}
+
+::arrow::Result<SessionOptionValue> FlightSqlSession::GetSessionOption(

Review Comment:
   Might be semantically clearer to use `std::optional` if the only possible error is that the key might not be found



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);

Review Comment:
   We should handle the case that `pair_sep` is not found



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid("Invalid or expired " +
+                               static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session, std::string session_id)
+    : factory_(factory),
+      headers_(headers),
+      session_(std::move(session)),
+      session_id_(std::move(session_id)),
+      existing_session(true) {}
+
+void ServerSessionMiddleware::SendingHeaders(AddCallHeaders* addCallHeaders) {

Review Comment:
   nit: Arrow uses snake_case for variable/parameter names



##########
cpp/src/arrow/flight/sql/server_session_middleware.h:
##########
@@ -0,0 +1,87 @@
+// 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 for handling Flight SQL Sessions including session cookie handling.
+// Currently experimental.
+
+#pragma once
+
+#include <shared_mutex>
+#include <string_view>
+
+#include "arrow/flight/server_middleware.h"
+#include "arrow/flight/sql/types.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+class ServerSessionMiddlewareFactory;
+
+static constexpr char const kSessionCookieName[] = "flight_sql_session_id";
+
+class FlightSqlSession {
+ protected:
+  std::map<std::string, SessionOptionValue> map_;
+  std::shared_mutex map_lock_;
+
+ public:
+  /// \brief Get session option by key
+  ::arrow::Result<SessionOptionValue> GetSessionOption(const std::string&);
+  /// \brief Set session option by key to given value
+  void SetSessionOption(const std::string&, const SessionOptionValue&);

Review Comment:
   Take the value by value so it can be moved into the map?



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));

Review Comment:
   make_shared



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid("Invalid or expired " +
+                               static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;
+    auto session = std::make_shared<FlightSqlSession>();
+
+    const std::unique_lock<std::shared_mutex> l(session_store_lock_);
+    session_store_[new_id] = session;
+
+    return session;
+  }
+};
+
+ServerSessionMiddleware::ServerSessionMiddleware(ServerSessionMiddlewareFactory* factory,
+                                                 const CallHeaders& headers)
+    : factory_(factory), headers_(headers), existing_session(false) {}
+
+ServerSessionMiddleware::ServerSessionMiddleware(
+    ServerSessionMiddlewareFactory* factory, const CallHeaders& headers,
+    std::shared_ptr<FlightSqlSession> session, std::string session_id)
+    : factory_(factory),
+      headers_(headers),
+      session_(std::move(session)),
+      session_id_(std::move(session_id)),
+      existing_session(true) {}
+
+void ServerSessionMiddleware::SendingHeaders(AddCallHeaders* addCallHeaders) {
+  if (!existing_session && session_) {
+    addCallHeaders->AddHeader(
+        "set-cookie", static_cast<std::string>(kSessionCookieName) + "=" + session_id_);
+  }
+}
+
+void ServerSessionMiddleware::CallCompleted(const Status&) {}
+
+bool ServerSessionMiddleware::HasSession() const { return static_cast<bool>(session_); }
+
+std::shared_ptr<FlightSqlSession> ServerSessionMiddleware::GetSession() {
+  if (!session_) session_ = factory_->GetNewSession(&session_id_);
+  return session_;
+}
+
+const CallHeaders& ServerSessionMiddleware::GetCallHeaders() const { return headers_; }
+
+std::shared_ptr<ServerMiddlewareFactory> MakeServerSessionMiddlewareFactory() {
+  return std::shared_ptr<ServerSessionMiddlewareFactory>(
+      new ServerSessionMiddlewareFactory());
+}
+
+::arrow::Result<SessionOptionValue> FlightSqlSession::GetSessionOption(
+    const std::string& k) {
+  const std::shared_lock<std::shared_mutex> l(map_lock_);
+  try {
+    return map_.at(k);
+  } catch (const std::out_of_range& e) {
+    return ::arrow::Status::KeyError("Session option key '" + k + "' not found.");
+  }

Review Comment:
   Same here, use a non-exception version



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>

Review Comment:
   I don't believe Boost is always available. If you want to use it, we'll have to adjust the CMakeLists to make sure it's found, and adjust ThirdpartyToolchain.cmake to reflect the new dependency.
   
   If there's a small/vendorable UUID implementation available, that might be preferable. CC @pitrou 



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(tok.substr(0, val_pos),
+                          tok.substr(val_pos + pair_sep_len, std::string::npos));
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo&, const CallHeaders& incoming_headers,
+                   std::shared_ptr<ServerMiddleware>* middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (session_id.empty())
+            return Status::Invalid("Empty " +
+                                   static_cast<std::string>(kSessionCookieName) +
+                                   " cookie value.");
+        }
+      }
+      if (!session_id.empty()) break;
+    }
+
+    if (session_id.empty()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers, session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid("Invalid or expired " +
+                               static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());

Review Comment:
   Is the generator thread-safe?



##########
cpp/src/arrow/flight/sql/server_session_middleware.cc:
##########
@@ -0,0 +1,179 @@
+// 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 <shared_mutex>
+#include "arrow/flight/sql/server_session_middleware.h"
+#include <boost/lexical_cast.hpp>
+#include <boost/uuid/uuid.hpp>
+#include <boost/uuid/uuid_generators.hpp>
+#include <boost/uuid/uuid_io.hpp>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief A factory for ServerSessionMiddleware, itself storing session data.
+class ServerSessionMiddlewareFactory : public ServerMiddlewareFactory {
+ protected:
+  std::map<std::string, std::shared_ptr<FlightSqlSession>> session_store_;
+  std::shared_mutex session_store_lock_;
+  boost::uuids::random_generator uuid_generator_;
+
+  std::vector<std::pair<std::string, std::string>> ParseCookieString(
+      const std::string_view& s) {
+    const std::string list_sep = "; ";
+    const std::string pair_sep = "=";
+    const size_t pair_sep_len = pair_sep.length();
+
+    std::vector<std::pair<std::string, std::string>> result;
+
+    size_t cur = 0;
+    while (cur < s.length()) {
+      const size_t end = s.find(list_sep, cur);
+      size_t len;
+      if (end == std::string::npos) {
+        // No (further) list delimiters
+        len = std::string::npos;
+        cur = s.length();
+      } else {
+        len = end - cur;
+        cur = end;
+      }
+      const std::string_view tok = s.substr(cur, len);
+
+      const size_t val_pos = tok.find(pair_sep);
+      result.emplace_back(
+          tok.substr(0, val_pos),
+          tok.substr(val_pos + pair_sep_len, std::string::npos)
+      );
+    }
+
+    return result;
+  }
+
+ public:
+  Status StartCall(const CallInfo &, const CallHeaders &incoming_headers,
+                   std::shared_ptr<ServerMiddleware> *middleware) {
+    std::string session_id;
+
+    const std::pair<CallHeaders::const_iterator, CallHeaders::const_iterator>&
+        headers_it_pr = incoming_headers.equal_range("cookie");
+    for (auto itr = headers_it_pr.first; itr != headers_it_pr.second; ++itr) {
+      const std::string_view& cookie_header = itr->second;
+      const std::vector<std::pair<std::string, std::string>> cookies =
+          ParseCookieString(cookie_header);
+      for (const std::pair<std::string, std::string>& cookie : cookies) {
+        if (cookie.first == kSessionCookieName) {
+          session_id = cookie.second;
+          if (!session_id.length())
+            return Status::Invalid(
+                "Empty " + static_cast<std::string>(kSessionCookieName)
+                + " cookie value.");
+        }
+      }
+      if (session_id.length()) break;
+    }
+
+    if (!session_id.length()) {
+      // No cookie was found
+      *middleware = std::shared_ptr<ServerSessionMiddleware>(
+          new ServerSessionMiddleware(this, incoming_headers));
+    } else {
+      try {
+        const std::shared_lock<std::shared_mutex> l(session_store_lock_);
+        auto session = session_store_.at(session_id);
+        *middleware = std::shared_ptr<ServerSessionMiddleware>(
+            new ServerSessionMiddleware(this, incoming_headers,
+                                        session, session_id));
+      } catch (std::out_of_range& e) {
+        return Status::Invalid(
+            "Invalid or expired "
+            + static_cast<std::string>(kSessionCookieName) + " cookie.");
+      }
+    }
+
+    return Status::OK();
+  }
+
+  /// \brief Get a new, empty session option map and its id key.
+  std::shared_ptr<FlightSqlSession> GetNewSession(std::string* session_id) {
+    std::string new_id = boost::lexical_cast<std::string>(uuid_generator_());
+    *session_id = new_id;

Review Comment:
   Better yet, return `std::pair` instead of using an out parameter



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,108 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed32 int32_value = 3;
+    sfixed64 int64_value = 4;
+    float float_value = 5;
+    double double_value = 6;
+    StringListValue string_list_value = 7;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via RFC 6265 HTTP cookies, canonically
+ * 'arrow_flight_session_id', although implementations may freely choose their own name.
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ */
+message SetSessionOptionsResult {
+  enum Status {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.
+    SET_SESSION_OPTION_RESULT_UNSPECIFIED = 0;
+    // The session option setting completed successfully.
+    SET_SESSION_OPTION_RESULT_OK = 1;
+    // The given session option name was an alias for another option name.
+    SET_SESSION_OPTION_RESULT_OK_MAPPED = 2;
+    // The given session option name is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_NAME = 3;
+    // The session option value is invalid.
+    SET_SESSION_OPTION_RESULT_INVALID_VALUE = 4;
+    // The session option cannot be set.
+    SET_SESSION_OPTION_RESULT_ERROR = 5;
+  }
+
+  message Result {
+    Status status = 1;
+  }
+  
+  map<string, Result> results = 1;
+}
+
+/*
+ * EXPERIMENTAL: A request to access the session options for the current server session.
+ *
+ * The existing session is referenced via a cookie header; it is an error to make this
+ * request with a missing, invalid, or expired session cookie header.
+ */
+message GetSessionOptionsRequest {
+}
+
+/*
+ * EXPERIMENTAL: The result containing the current server session options.
+ */
+message GetSessionOptionsResult {
+    map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * Request message for the "Close Session" action.
+ *
+ * The exiting session is referenced via a cookie header.
+ */
+message CloseSessionRequest {
+}
+
+/*
+ * The result of closing a session.
+ */
+message CloseSessionResult {
+  enum Status {
+    // The session close status is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested session is
+    // not known). Clients can retry the request.
+    CLOSE_RESULT_UNSPECIFIED = 0;

Review Comment:
   Why 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
format/Flight.proto:
##########
@@ -525,3 +525,117 @@ message FlightData {
 message PutResult {
   bytes app_metadata = 1;
 }
+
+/*
+ * EXPERIMENTAL: Union of possible value types for a Session Option to be set to.
+ *
+ * By convention, an attempt to set a valueless SessionOptionValue should
+ * attempt to unset or clear the named option value on the server.
+ */
+message SessionOptionValue {
+  message StringListValue {
+    repeated string values = 1;
+  }
+
+  oneof option_value {
+    string string_value = 1;
+    bool bool_value = 2;
+    sfixed64 int64_value = 3;
+    double double_value = 4;
+    StringListValue string_list_value = 5;
+  }
+}
+
+/*
+ * EXPERIMENTAL: A request to set session options for an existing or new (implicit)
+ * server session.
+ *
+ * Sessions are persisted and referenced via a transport-level state management, typically
+ * RFC 6265 HTTP cookies when using an HTTP transport.  The suggested cookie name or state
+ * context key is 'arrow_flight_session_id', although implementations may freely choose their
+ * own name.
+ *
+ * Session creation (if one does not already exist) is implied by this RPC request, however
+ * server implementations may choose to initiate a session that also contains client-provided
+ * session options at any other time, e.g. on authentication, or when any other call is made
+ * and the server wishes to use a session to persist any state (or lack thereof).
+ */
+message SetSessionOptionsRequest {
+  map<string, SessionOptionValue> session_options = 1;
+}
+
+/*
+ * EXPERIMENTAL: The results (individually) of setting a set of session options.
+ *
+ * Option names should only be present in the response if they were not successfully
+ * set on the server; that is, a response without an Error for a name provided in the
+ * SetSessionOptionsRequest implies that the named option value was set successfully.
+ */
+message SetSessionOptionsResult {
+  enum ErrorValue {
+    // The status of setting the option is unknown. Servers should avoid using
+    // this value (send a NOT_FOUND error if the requested query is
+    // not known). Clients can retry the request.

Review Comment:
   I think it was originally added to help reviewers who aren't familiar with Protobuf convention/who don't work with it often enough to care to remember (since nearly every proposal gets a question along the lines of "why do we have UNSPECIFIED").



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   The problem with say, DoAction, is that it's a streaming response, so in theory the RPC handler can start returning results, and then you can't call SendingHeaders anymore (it needs to be before you send any headers). The one saving grace is that our DoAction doesn't let you directly send results, instead you return an iterator. So then we can implement the potentially very confusing API where the body of DoAction() itself can affect what SendingHeaders sees, but NOT the iterator's body. Probably this is OK in general because most people will return a dummy iterator (and you shouldn't expect to affect headers after the iterator has started running), but it is odd. 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   > I am lost. You would stop calling SendingHeaders where it was previously called, and start calling it explicitly in each RPC handler at an appropriate time. I don't see why empty streams need special treatment.
   
   Just looking at where you said to do it around L528 here https://github.com/apache/arrow/blob/3fe598ae4dfd7805ab05452dd5ed4b0d6c97d8d5/cpp/src/arrow/flight/transport/grpc/grpc_server.cc#L513-L544
   
   
   That creates a special case for DoAction getting an empty stream back from the Action handler, so I'm assuming it would be L524 instead...


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   > There's no session unless session creation is triggered by putting stuff into the session, thus no session cookie until then.
   
   That's an arbitrary choice, 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   > That gets us back to needing middleware to access the session store, which just moves the problem around in order to solve it... I'm thinking about doing the other solution very locally, but the header handling needs to be triggered from within the handler itself (not even DoAction() which can't intercede at the right point), so I will need to add the stream interposer to defaut-trigger the header handling on stream onNext & onError & onCompleted for compatibility with existing implementations... objections?
   
   No: just do everything inline in the RPC handler/a helper, no middleware involved at all. You can read and write headers inside the handler.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   > @lidavidm mostly unrelated but also, I'm not seeing how I get a CallStatus.NOT_FOUND equivalent in C++, am I missing something? (TIA and sorry for the additional bother)
   
   KeyError, IIRC (yes, using arrow::Status for this was likely stupid)
   
   > @lidavidm ok for purposes of the PR merge I'm going to defang CloseSession in the C++ MW, do you have a preference between these two moderately ugly temporary solutions:
   
   No particular preference, I guess CLOSED + invalid cookie is fine


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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

   I pinged the mailing list. 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   Done, 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Java][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/CloseSessionResult.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class CloseSessionResult {
+  public enum Status {
+    /**
+     * The session close status is unknown. Servers should avoid using this value
+     * (send a NOT_FOUND error if the requested session is not known). Clients can
+     * retry the request.
+     */
+    UNSPECIFIED(Flight.CloseSessionResult.Status.UNSPECIFIED),
+    /**
+     * The session close request is complete.
+     */
+    CLOSED(Flight.CloseSessionResult.Status.CLOSED),
+    /**
+     * The session close request is in progress. The client may retry the request.
+     */
+    CLOSING(Flight.CloseSessionResult.Status.CLOSING),
+    /**
+     * The session is not closeable.
+     */
+    NOT_CLOSABLE(Flight.CloseSessionResult.Status.NOT_CLOSABLE),
+    ;
+
+    private static final Map<Flight.CloseSessionResult.Status, Status> mapFromProto;
+
+    static {
+      for (Status s : values()) mapFromProto.put(s.proto, s);

Review Comment:
   Understood & done



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   > SessionOptionValueVisitor should probably be a generic and acceptVisitor() should return its result.
   
   I didn't because Java can't constrain the templated type to a specific set of values; SessionOptionValue could possibly—I think that's checked at runtime so the SessionOptionValueVisitor::visit overloads would constrain that correctly?



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   > SessionOptionValueVisitor should probably be a generic and acceptVisitor() should return its result.
   
   I didn't because Java can't constrain the templated type to a specific set of values; SessionOptionValue could possibly—I think that's checked at runtime so the SessionOptionValueVisitor::visit overloads would constrain that correctly?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   The callback is required to avoid the caller to acceptVisitor also having to be generic (in order to assign a templated type), no?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


Re: [PR] GH-34865: [C++][Flight RPC] Add Session management messages [arrow]

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


##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/CloseSessionResult.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class CloseSessionResult {
+  public enum Status {
+    /**
+     * The session close status is unknown. Servers should avoid using this value
+     * (send a NOT_FOUND error if the requested session is not known). Clients can
+     * retry the request.
+     */
+    UNSPECIFIED(Flight.CloseSessionResult.Status.UNSPECIFIED),
+    /**
+     * The session close request is complete.
+     */
+    CLOSED(Flight.CloseSessionResult.Status.CLOSED),
+    /**
+     * The session close request is in progress. The client may retry the request.
+     */
+    CLOSING(Flight.CloseSessionResult.Status.CLOSING),
+    /**
+     * The session is not closeable.
+     */
+    NOT_CLOSABLE(Flight.CloseSessionResult.Status.NOT_CLOSABLE),
+    ;
+
+    private static final Map<Flight.CloseSessionResult.Status, Status> mapFromProto;
+
+    static {
+      for (Status s : values()) mapFromProto.put(s.proto, s);

Review Comment:
   Isn't this just accessing Status.values()[s.ordinal]?



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SetSessionOptionsResult.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class SetSessionOptionsResult {
+  public enum Status {
+    /**
+     * The status of setting the option is unknown. Servers should avoid using this value
+     * (send a NOT_FOUND error if the requested session is not known). Clients can retry
+     * the request.
+      */
+    UNSPECIFIED(Flight.SetSessionOptionsResult.Status.UNSPECIFIED),
+    /**
+     * The session option setting completed successfully.
+     */
+    OK(Flight.SetSessionOptionsResult.Status.OK),
+    /**
+     * The given session option name was an alias for another option name.
+     */
+    OK_MAPPED(Flight.SetSessionOptionsResult.Status.OK_MAPPED),
+    /**
+     * The given session option name is invalid.
+     */
+    INVALID_NAME(Flight.SetSessionOptionsResult.Status.INVALID_NAME),
+    /**
+     * The session option value is invalid.
+     */
+    INVALID_VALUE(Flight.SetSessionOptionsResult.Status.INVALID_VALUE),
+    /**
+     * The session option cannot be set.
+     */
+    ERROR(Flight.SetSessionOptionsResult.Status.ERROR),
+    ;
+
+    private static final Map<Flight.SetSessionOptionsResult.Status, SetSessionOptionsResult.Status> mapFromProto;
+
+    static {
+      for (Status s : values()) mapFromProto.put(s.proto, s);
+    }
+
+    private final Flight.SetSessionOptionsResult.Status proto;
+
+    private Status(Flight.SetSessionOptionsResult.Status s) {
+      proto = s;
+    }
+
+    public static Status fromProtocol(Flight.SetSessionOptionsResult.Status s) {
+      return mapFromProto.get(s);
+    }
+
+    public Flight.SetSessionOptionsResult.Status toProtocol() {
+      return proto;
+    }
+  }
+
+  private final Map<String, Status> results;
+
+  public SetSessionOptionsResult(Map<String, Status> results) {
+    this.results = HashMap<String, Status>(results);
+  }
+
+  SetSessionOptionsResult(Flight.SetSessionOptionsResult proto) {
+    results = proto.getResults().entrySet().stream().collect(Collectors.toMap(
+        Map.Entry::getKey, e -> Status.fromProtocol(e.getValue())));
+  }
+
+  Map<String, Status> getResults() {
+    return Collections.unmodifiableMap(results);
+  }
+
+  Flight.SetSessionOptionsResult toProtocol() {
+    Flight.SetSessionOptionsResult.Builder b = Flight.SetSessionOptionsResult.newBuilder();
+    b.putAllResults(results.entrySet().stream().collect(Collectors.toMap(
+        Map.Entry::getKey, e->getValue().toProtocol())));

Review Comment:
   e -> getValue()



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/CloseSessionRequest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;

Review Comment:
   Let's change the title of this pr to include [Java]



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/CloseSessionRequest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class CloseSessionRequest {

Review Comment:
   A class comment is needed.



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);
+
+    Flight.SessionOptionValue toProtocol() {
+        Flight.SessionOptionValue.Builder b = Flight.SessionOptionValue.newBuilder();
+        SessionOptionValueToProtocolVisitor visitor = new SessionOptionValueToProtocolVisitor(b);
+        this.visit(visitor);
+        return b.build();
+    }
+}
+
+class SessionOptionValueToProtocolVisitor implements SessionOptionValueVisitor {
+    final Flight.SessionOptionValue.Builder b;
+
+    SessionOptionValueVisitor(Flight.SessionOptionValue.Builder b) { this.b = b; }
+
+    Flight.SessionOptionValue visit(String value) {
+        b.setStringValue(value);

Review Comment:
   These functions need to return a value



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValueFactory.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.SessionOptionValue;
+
+public class SessionOptionValueFactory {
+    public static SessionOptionValue makeSessionOption(String value) {
+        return new SessionOptionValueString(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(bool value) {
+        return new SessionOptionValueBool(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(int value) {
+        return new SessionOptionValueInt(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(long value) {
+        return new SessionOptionValueLong(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(float value) {
+        return new SessionOptionValueFloat(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(double value) {
+        return new SessionOptionValueDouble(value);
+    }
+
+    public static SessionOptionValue makeSessionOption(String[] value) {
+        return new SessionOptionValueStringList(value);
+    }
+}
+
+class SessionOptionValueString {

Review Comment:
   Are options nullable? Is there any way for a user to "unset" an option? For example they might want to do this to revert to a server-side default.



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);
+
+    Flight.SessionOptionValue toProtocol() {
+        Flight.SessionOptionValue.Builder b = Flight.SessionOptionValue.newBuilder();
+        SessionOptionValueToProtocolVisitor visitor = new SessionOptionValueToProtocolVisitor(b);
+        this.visit(visitor);
+        return b.build();
+    }
+}
+
+class SessionOptionValueToProtocolVisitor implements SessionOptionValueVisitor {
+    final Flight.SessionOptionValue.Builder b;
+
+    SessionOptionValueVisitor(Flight.SessionOptionValue.Builder b) { this.b = b; }
+
+    Flight.SessionOptionValue visit(String value) {
+        b.setStringValue(value);
+    }
+
+    Flight.SessionOptionValue visit(boolean value) {
+        b.setBoolValue(value);
+    }
+
+    Flight.SessionOptionValue visit(int value) {
+        b.setIn32Value(value);
+    }
+
+    Flight.SessionOptionValue visit(long value) {
+        b.setInt64Value(value);
+    }
+
+    Flight.SessionOptionValue visit(float value) {
+        b.setFloatValue(value);
+    }
+
+    Flight.SessionOptionValue visit(double value) {
+        b.setDoubleValue(value);
+    }
+
+    Flight.SessionOptionValue visit(String[] value) {
+        Flight.SessionOptionValue.StringListValue pb_value;

Review Comment:
   Use java naming conventions



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/CloseSessionRequest.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class CloseSessionRequest {
+  public CloseSessionRequest() {}
+
+  CloseSessionRequest(Flight.CloseSessionRequest proto) {}
+
+  Flight.CloseSessionRequest toProtocol() {
+    Flight.CloseSessionRequest.Builder b = Flight.CloseSessionRequest.newBuilder();
+    return b.build();
+  }
+
+  /**
+   * Get the serialized form of this protocol message.
+   *
+   * <p>Intended to help interoperability by allowing non-Flight services to still return Flight types.
+   */
+  public ByteBuffer serialize() {
+    return ByteBuffer.wrap(toProtocol().toByteArray());

Review Comment:
   Should this return a singleton? There's no state here.



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   This should be "acceptVisitor".



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);
+
+    Flight.SessionOptionValue toProtocol() {
+        Flight.SessionOptionValue.Builder b = Flight.SessionOptionValue.newBuilder();
+        SessionOptionValueToProtocolVisitor visitor = new SessionOptionValueToProtocolVisitor(b);
+        this.visit(visitor);
+        return b.build();
+    }
+}
+
+class SessionOptionValueToProtocolVisitor implements SessionOptionValueVisitor {
+    final Flight.SessionOptionValue.Builder b;
+
+    SessionOptionValueVisitor(Flight.SessionOptionValue.Builder b) { this.b = b; }
+
+    Flight.SessionOptionValue visit(String value) {
+        b.setStringValue(value);
+    }
+
+    Flight.SessionOptionValue visit(boolean value) {
+        b.setBoolValue(value);
+    }
+
+    Flight.SessionOptionValue visit(int value) {
+        b.setIn32Value(value);
+    }
+
+    Flight.SessionOptionValue visit(long value) {
+        b.setInt64Value(value);
+    }
+
+    Flight.SessionOptionValue visit(float value) {
+        b.setFloatValue(value);
+    }
+
+    Flight.SessionOptionValue visit(double value) {
+        b.setDoubleValue(value);
+    }
+
+    Flight.SessionOptionValue visit(String[] value) {
+        Flight.SessionOptionValue.StringListValue pb_value;
+        pb_value.addAllStringValues(value);
+        b.setValue(pb_value);
+    }
+}

Review Comment:
   Please add a newline to the end of the file.



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SetSessionOptionsRequest.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class SetSessionOptionsRequest {
+  private final Map<String, SessionOptionValue> session_options;
+
+  public SetSessionOptionsRequest(Map<String, SessionOptionValue> session_options) {
+    this.session_options = ConcurrentHashMap(session_options);

Review Comment:
   new ConcurrentHashMap.



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/GetSessionOptionsResult.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class GetSessionOptionsResult {
+  private final Map<String, SessionOptionValue> session_options;
+
+  public GetSessionOptionsResult(Map<String, SessionOptionValue> session_options) {
+    this.session_options = new HashMap(session_options);
+  }
+
+  GetSessionOptionsResult(Flight.GetSessionOptionsResult proto) {
+    session_options = proto.getSessionOptions().entrySet().stream().collect(Collectors.toMap(Map

Review Comment:
   Use java variable naming convention (eg sessionOptions)



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SessionOptionValue.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import org.apache.arrow.flight.impl.Flight;
+
+/**
+ * A union-like container interface for supported session option value types.
+ */
+public abstract class SessionOptionValue {
+    static SessionOptionValue fromProto(Flight.SessionOptionValue proto); {
+        switch(proto.getOptionValueCase()) {
+            case STRING_VALUE:
+                return new SessionOptionValueString(proto.getStringValue());
+                break;
+            case BOOL_VALUE:
+                return new SessionOptionValueBoolean(proto.getValue());
+                break;
+            case INT32_VALUE:
+                return new SessionOptionValueInt(proto.getInt32Value());
+                break;
+            case INT64_VALUE:
+                return new SessionOptionValueLong(proto.getInt64Value());
+                break;
+            case FLOAT_VALUE:
+                return new SessionOptionValueFloat(proto.getFloatValue());
+                break;
+            case DOUBLE_VALUE:
+                return new SessionOptionValueDouble(proto.getDoubleValue());
+                break;
+            case STRING_LIST_VALUE:
+                // FIXME PHOXME is this what's in the ProtocolStringList?
+                return new SessionOptionValueStringList(proto.getValueStringList().stream().collect(
+                    Collectors.toList(e -> google.protocol.StringValue.parseFrom(e).getValue())));
+                break;
+            default:
+                // Unreachable
+                 throw new IllegalArgumentException("");
+        }
+    }
+
+    /**
+     * Value access via a caller-provided visitor/functor.
+     */
+    abstract void visit(SessionOptionValueVisitor);

Review Comment:
   SessionOptionValueVisitor should probably be a generic and acceptVisitor() should return its result.



##########
java/flight/flight-core/src/main/java/org/apache/arrow/flight/SetSessionOptionsRequest.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class SetSessionOptionsRequest {
+  private final Map<String, SessionOptionValue> session_options;
+
+  public SetSessionOptionsRequest(Map<String, SessionOptionValue> session_options) {
+    this.session_options = ConcurrentHashMap(session_options);

Review Comment:
   Why use a ConcurrentHashMap? It doesn't look like the map is even mutable after being passed to the constructor?



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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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