You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/11/03 20:07:43 UTC

[GitHub] [arrow] tifflhl opened a new pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

tifflhl opened a new pull request #8583:
URL: https://github.com/apache/arrow/pull/8583


   - Add ClientSessionMiddleware and ClientSessionWriter to support session ID re-use.
   - Add ServerSessionMiddleware and ServerSessionHandler API to enable sessions.
   - Unit tests for ClientSessionMiddleware and ClientSessionWriter.
   - TODO: Unit tests for ServerSessionMiddleware and ServerSessionHandler API.
   - TODO: End-to-end 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.

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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


   <!--
     Licensed to the Apache Software Foundation (ASF) under one
     or more contributor license agreements.  See the NOTICE file
     distributed with this work for additional information
     regarding copyright ownership.  The ASF licenses this file
     to you under the Apache License, Version 2.0 (the
     "License"); you may not use this file except in compliance
     with the License.  You may obtain a copy of the License at
   
       http://www.apache.org/licenses/LICENSE-2.0
   
     Unless required by applicable law or agreed to in writing,
     software distributed under the License is distributed on an
     "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
     KIND, either express or implied.  See the License for the
     specific language governing permissions and limitations
     under the License.
   -->
   
   Thanks for opening a pull request!
   
   Could you open an issue for this pull request on JIRA?
   https://issues.apache.org/jira/browse/ARROW
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/client/ClientSessionMiddleware.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.client;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightClientMiddleware;
+import org.apache.arrow.flight.FlightConstants;
+
+/**
+ * Client middleware to re-use the existing session provided by the server.
+ */
+public class ClientSessionMiddleware implements FlightClientMiddleware {
+  /**
+   * Factory to create instances of ClientSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightClientMiddleware.Factory {
+    private final ClientSessionWriter sessionWriter;
+
+    public Factory() {
+      sessionWriter = new ClientSessionWriter();
+    }
+
+    @Override
+    public FlightClientMiddleware onCallStarted(CallInfo info) {
+      return new ClientSessionMiddleware(sessionWriter);
+    }
+  }
+
+  private final ClientSessionWriter sessionWriter;
+
+  public ClientSessionMiddleware(ClientSessionWriter sessionWriter) {

Review comment:
       This shouldn't be public. The factory should be the only thing that can create these, outside of testing.




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

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



[GitHub] [arrow] tifflhl commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightConstants.java
##########
@@ -23,5 +23,6 @@
 public interface FlightConstants {
 
   String SERVICE = "arrow.flight.protocol.FlightService";
+  String SESSION_HEADER = "SET-SESSION";

Review comment:
       Addressed.




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

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/test/java/org/apache/arrow/flight/TestServerSessionHandling.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.arrow.flight.grpc.RequestContextAdapter;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Tests for ServerSessionHandler and ServerSessionMiddleware.
+ */
+public class TestServerSessionHandling {
+  private static final String SESSION_HEADER_KEY = "Session";
+  private static final String SET_SESSION_HEADER_KEY = "Set-Session";
+  private static final String TEST_SESSION = "name=test-session;id=session-id;max-age=100";
+  private static final List<String> VALID_PROPERTY_KEYS = ImmutableList.of(
+          "name",
+          "id",
+          "max-age"
+  );
+
+  private class SimpleServerSessionHandler implements ServerSessionHandler {
+    private String session;
+
+    SimpleServerSessionHandler() {
+      this.session = null;
+    }
+
+    @Override
+    public String beginSession(CallHeaders headers) {
+      session = TEST_SESSION;
+      return session;
+    }
+
+    @Override
+    public String getSession(CallHeaders headers) {
+      // No session has been set yet
+      if (session == null) {
+        return null;
+      }
+
+      // Expected session header but none is found
+      if (!headers.containsKey(SESSION_HEADER_KEY)) {
+        throw CallStatus.NOT_FOUND.toRuntimeException();
+      }
+
+      // Validate that session header does not contain invalid properties
+      final Map<String, String> propertiesMap = new HashMap<>();
+      final String[] inSessionProperties = headers.get(SESSION_HEADER_KEY).split(";");
+      for (String pair : inSessionProperties) {
+        final String[] keyVal = pair.split("=");
+        if (!VALID_PROPERTY_KEYS.contains(keyVal[0])) {
+          throw CallStatus.INVALID_ARGUMENT.toRuntimeException();
+        }
+        propertiesMap.put(keyVal[0], keyVal[1]);
+      }
+
+      // Validate that session has not expired
+      final int maxAge = Integer.parseInt(propertiesMap.get("max-age"));
+      if (maxAge == 0) {

Review comment:
       For cookies, max-age < 0 also means it is expired.




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

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add an optional set-session/session header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+
+      String session = sessionHandler.getSession(incomingHeaders);
+
+      if (session == null) {
+        // No existing session provided, establishing a new session.
+        // Insert SET-SESSION header if ServerSessionHandler returns a non-null session.
+        session = sessionHandler.beginSession(incomingHeaders);
+        incomingHeaders.insert(FlightConstants.SET_SESSION_HEADER, session);
+      }
+
+      return new ServerSessionMiddleware(session);
+    }
+  }
+
+  private String session;

Review comment:
       final




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

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add an optional set-session/session header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+
+      String session = sessionHandler.getSession(incomingHeaders);
+
+      if (session == null) {
+        // No existing session provided, establishing a new session.
+        // Insert SET-SESSION header if ServerSessionHandler returns a non-null session.
+        session = sessionHandler.beginSession(incomingHeaders);
+        incomingHeaders.insert(FlightConstants.SET_SESSION_HEADER, session);
+      }
+
+      return new ServerSessionMiddleware(session);
+    }
+  }
+
+  private String session;
+
+  private ServerSessionMiddleware(String session) {
+    this.session = session;
+  }
+
+  @Override
+  public void onBeforeSendingHeaders(CallHeaders outgoingHeaders) {
+    outgoingHeaders.insert(FlightConstants.SET_SESSION_HEADER, session);

Review comment:
       Do we have to send the session token back on every 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.

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



[GitHub] [arrow] tifflhl commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add a SET-SESSION header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+      if (incomingHeaders.containsKey(FlightConstants.SESSION_HEADER)) {
+        // Client is re-using existing session ID.
+        // ServerSessionHandler validates client session ID before proceeding.
+        final String existingSessionId = incomingHeaders.get(FlightConstants.SESSION_HEADER);
+        if (!sessionHandler.isValid(existingSessionId)) {
+          throw CallStatus.UNAUTHENTICATED.toRuntimeException();
+        }
+      } else {
+        // No existing session ID provided, establishing a new session.
+        // Insert SET-SESSION header if ServerSessionHandler returns a non-null session ID.
+        final String sessionId = sessionHandler.getSessionId();

Review comment:
       The API has since been modified, please 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.

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add an optional set-session/session header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+
+      String session = sessionHandler.getSession(incomingHeaders);
+
+      if (session == null) {
+        // No existing session provided, establishing a new session.
+        // Insert SET-SESSION header if ServerSessionHandler returns a non-null session.
+        session = sessionHandler.beginSession(incomingHeaders);
+        incomingHeaders.insert(FlightConstants.SET_SESSION_HEADER, session);

Review comment:
       Why write this to incomingHeaders?




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

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/test/java/org/apache/arrow/flight/TestServerSessionHandling.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.arrow.flight.grpc.RequestContextAdapter;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Tests for ServerSessionHandler and ServerSessionMiddleware.
+ */
+public class TestServerSessionHandling {
+  private static final String SESSION_HEADER_KEY = "Session";
+  private static final String SET_SESSION_HEADER_KEY = "Set-Session";
+  private static final String TEST_SESSION = "name=test-session;id=session-id;max-age=100";
+  private static final List<String> VALID_PROPERTY_KEYS = ImmutableList.of(
+          "name",
+          "id",
+          "max-age"
+  );
+
+  private class SimpleServerSessionHandler implements ServerSessionHandler {
+    private String session;
+
+    SimpleServerSessionHandler() {
+      this.session = null;
+    }
+
+    @Override
+    public String beginSession(CallHeaders headers) {
+      session = TEST_SESSION;
+      return session;
+    }
+
+    @Override
+    public String getSession(CallHeaders headers) {
+      // No session has been set yet
+      if (session == null) {
+        return null;
+      }
+
+      // Expected session header but none is found
+      if (!headers.containsKey(SESSION_HEADER_KEY)) {
+        throw CallStatus.NOT_FOUND.toRuntimeException();
+      }
+
+      // Validate that session header does not contain invalid properties
+      final Map<String, String> propertiesMap = new HashMap<>();
+      final String[] inSessionProperties = headers.get(SESSION_HEADER_KEY).split(";");
+      for (String pair : inSessionProperties) {
+        final String[] keyVal = pair.split("=");
+        if (!VALID_PROPERTY_KEYS.contains(keyVal[0])) {
+          throw CallStatus.INVALID_ARGUMENT.toRuntimeException();
+        }
+        propertiesMap.put(keyVal[0], keyVal[1]);
+      }
+
+      // Validate that session has not expired
+      final int maxAge = Integer.parseInt(propertiesMap.get("max-age"));

Review comment:
       This is a number format exception if max-age isn't a number. 




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

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add an optional set-session/session header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+
+      String session = sessionHandler.getSession(incomingHeaders);

Review comment:
       I feel getSession should take in the session token instead of all headers, since we explicitly have this middleware send the token.




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

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add a SET-SESSION header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+      if (incomingHeaders.containsKey(FlightConstants.SESSION_HEADER)) {
+        // Client is re-using pre-existing session ID.
+        // ServerSessionHandler validates client session ID before proceeding.
+        final String sessionId = incomingHeaders.get(FlightConstants.SESSION_HEADER);
+        if (!sessionHandler.isValid(sessionId)) {
+          throw CallStatus.UNAUTHENTICATED.toRuntimeException();
+        }
+      } else {
+        // Insert SET-SESSION header if ServerSessionHandler returns a non-null session ID.
+        final String sessionId = sessionHandler.getSessionID();

Review comment:
       Let's rename getSessionID() to beginSession() and have it take in the input headers.
   Let's have ServerSessionMiddleware take in the session id.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add a SET-SESSION header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+      if (incomingHeaders.containsKey(FlightConstants.SESSION_HEADER)) {
+        // Client is re-using existing session ID.
+        // ServerSessionHandler validates client session ID before proceeding.
+        final String existingSessionId = incomingHeaders.get(FlightConstants.SESSION_HEADER);
+        if (!sessionHandler.isValid(existingSessionId)) {
+          throw CallStatus.UNAUTHENTICATED.toRuntimeException();
+        }
+      } else {
+        // No existing session ID provided, establishing a new session.
+        // Insert SET-SESSION header if ServerSessionHandler returns a non-null session ID.
+        final String sessionId = sessionHandler.getSessionId();
+        if (sessionId != null) {
+          incomingHeaders.insert(FlightConstants.SESSION_HEADER, sessionId);
+        }
+      }
+
+      return new ServerSessionMiddleware(sessionHandler);
+    }
+  }
+
+  private ServerSessionHandler sessionHandler;
+
+  private ServerSessionMiddleware(ServerSessionHandler sessionHandler) {
+    this.sessionHandler = sessionHandler;
+  }
+
+  @Override
+  public void onBeforeSendingHeaders(CallHeaders outgoingHeaders) {
+    String sessionId = sessionHandler.getSessionId();

Review comment:
       This should not call getSessionId(), because you can't guarantee that this sessionId is different from the one from onCallStarted(). Also, getSessionId() might trigger the backend starting a new session, so you'd double-start it.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add a SET-SESSION header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+      if (incomingHeaders.containsKey(FlightConstants.SESSION_HEADER)) {
+        // Client is re-using existing session ID.
+        // ServerSessionHandler validates client session ID before proceeding.
+        final String existingSessionId = incomingHeaders.get(FlightConstants.SESSION_HEADER);
+        if (!sessionHandler.isValid(existingSessionId)) {
+          throw CallStatus.UNAUTHENTICATED.toRuntimeException();
+        }
+      } else {
+        // No existing session ID provided, establishing a new session.
+        // Insert SET-SESSION header if ServerSessionHandler returns a non-null session ID.
+        final String sessionId = sessionHandler.getSessionId();

Review comment:
       Rename getSessionId() to beginSession, and cache it instead of the SessionHandler in ServerSessionMiddleware.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionHandler.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/**
+ * ServerSessionHandler interface to retrieve the current session ID.
+ */
+public interface ServerSessionHandler {
+  /**
+   * A session handler that does not support sessions.
+   * It is used as the default handler for a Flight server with no session capabilities.
+   */
+  ServerSessionHandler NO_OP = new ServerSessionHandler() {
+    @Override
+    public String getSessionId() {
+          return null;
+      }
+
+    @Override
+    public boolean isValid(String sessionId) {
+          return false;
+      }
+  };
+
+  /**
+   * Retrieves the current session ID. Generates a new session ID if there is no pre-existing

Review comment:
       We need a way to generate a session ID based off headers. There is no 'current' session because this ServerSessionHandler can be servicing multiple different sessions at once.




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

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/test/java/org/apache/arrow/flight/TestServerSessionHandling.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.arrow.flight.grpc.RequestContextAdapter;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Tests for ServerSessionHandler and ServerSessionMiddleware.
+ */
+public class TestServerSessionHandling {
+  private static final String SESSION_HEADER_KEY = "Session";
+  private static final String SET_SESSION_HEADER_KEY = "Set-Session";
+  private static final String TEST_SESSION = "name=test-session;id=session-id;max-age=100";
+  private static final List<String> VALID_PROPERTY_KEYS = ImmutableList.of(
+          "name",
+          "id",
+          "max-age"
+  );
+
+  private class SimpleServerSessionHandler implements ServerSessionHandler {
+    private String session;
+
+    SimpleServerSessionHandler() {
+      this.session = null;
+    }
+
+    @Override
+    public String beginSession(CallHeaders headers) {
+      session = TEST_SESSION;
+      return session;
+    }
+
+    @Override
+    public String getSession(CallHeaders headers) {
+      // No session has been set yet
+      if (session == null) {
+        return null;
+      }
+
+      // Expected session header but none is found
+      if (!headers.containsKey(SESSION_HEADER_KEY)) {
+        throw CallStatus.NOT_FOUND.toRuntimeException();
+      }
+
+      // Validate that session header does not contain invalid properties
+      final Map<String, String> propertiesMap = new HashMap<>();
+      final String[] inSessionProperties = headers.get(SESSION_HEADER_KEY).split(";");
+      for (String pair : inSessionProperties) {
+        final String[] keyVal = pair.split("=");

Review comment:
       Is '=' a legal character for a 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.

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



[GitHub] [arrow] tifflhl closed pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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


   


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

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



[GitHub] [arrow] tifflhl commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add a SET-SESSION header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+      if (incomingHeaders.containsKey(FlightConstants.SESSION_HEADER)) {
+        // Client is re-using pre-existing session ID.
+        // ServerSessionHandler validates client session ID before proceeding.
+        final String sessionId = incomingHeaders.get(FlightConstants.SESSION_HEADER);
+        if (!sessionHandler.isValid(sessionId)) {
+          throw CallStatus.UNAUTHENTICATED.toRuntimeException();
+        }
+      } else {
+        // Insert SET-SESSION header if ServerSessionHandler returns a non-null session ID.
+        final String sessionId = sessionHandler.getSessionID();

Review comment:
       The API has since been modified, please 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.

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



[GitHub] [arrow] tifflhl commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionHandler.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/**
+ * ServerSessionHandler interface to retrieve the current session ID.
+ */
+public interface ServerSessionHandler {
+  /**
+   * A session handler that does not support sessions.
+   * It is used as the default handler for a Flight server with no session capabilities.
+   */
+  ServerSessionHandler NO_OP = new ServerSessionHandler() {
+    @Override
+    public String getSessionId() {
+          return null;
+      }
+
+    @Override
+    public boolean isValid(String sessionId) {
+          return false;
+      }
+  };
+
+  /**
+   * Retrieves the current session ID. Generates a new session ID if there is no pre-existing

Review comment:
       The API has since been modified, please check.

##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/ServerSessionMiddleware.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * Middleware to add a SET-SESSION header.
+ */
+public class ServerSessionMiddleware implements FlightServerMiddleware {
+
+  /**
+   * Factory class to provide instances of ServerSessionMiddleware for each call.
+   */
+  public static class Factory implements FlightServerMiddleware.Factory<ServerSessionMiddleware> {
+    private final ServerSessionHandler sessionHandler;
+
+    public Factory(ServerSessionHandler sessionHandler) {
+      this.sessionHandler = sessionHandler;
+    }
+
+    @Override
+    public ServerSessionMiddleware onCallStarted(CallInfo info, CallHeaders incomingHeaders,
+                                                 RequestContext context) {
+      if (incomingHeaders.containsKey(FlightConstants.SESSION_HEADER)) {
+        // Client is re-using existing session ID.
+        // ServerSessionHandler validates client session ID before proceeding.
+        final String existingSessionId = incomingHeaders.get(FlightConstants.SESSION_HEADER);
+        if (!sessionHandler.isValid(existingSessionId)) {
+          throw CallStatus.UNAUTHENTICATED.toRuntimeException();
+        }
+      } else {
+        // No existing session ID provided, establishing a new session.
+        // Insert SET-SESSION header if ServerSessionHandler returns a non-null session ID.
+        final String sessionId = sessionHandler.getSessionId();
+        if (sessionId != null) {
+          incomingHeaders.insert(FlightConstants.SESSION_HEADER, sessionId);
+        }
+      }
+
+      return new ServerSessionMiddleware(sessionHandler);
+    }
+  }
+
+  private ServerSessionHandler sessionHandler;
+
+  private ServerSessionMiddleware(ServerSessionHandler sessionHandler) {
+    this.sessionHandler = sessionHandler;
+  }
+
+  @Override
+  public void onBeforeSendingHeaders(CallHeaders outgoingHeaders) {
+    String sessionId = sessionHandler.getSessionId();

Review comment:
       The API has since been modified, please 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.

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/client/ClientSessionMiddleware.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.client;
+
+import org.apache.arrow.flight.CallHeaders;
+import org.apache.arrow.flight.CallInfo;
+import org.apache.arrow.flight.CallStatus;
+import org.apache.arrow.flight.FlightClientMiddleware;
+import org.apache.arrow.flight.FlightConstants;
+
+/**
+ * Client middleware to re-use the existing session provided by the server.
+ */
+public class ClientSessionMiddleware implements FlightClientMiddleware {

Review comment:
       I think the ClientSessionMiddleware should be responsible for sending the header back rather than sending the header as part of a CallOption.
   
   The reason is that you are leaving it up to the application code to check the expiration of the session. (Note that a positive max-age can still expire after some 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.

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



[GitHub] [arrow] keeratsingh commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/main/java/org/apache/arrow/flight/FlightConstants.java
##########
@@ -23,5 +23,6 @@
 public interface FlightConstants {
 
   String SERVICE = "arrow.flight.protocol.FlightService";
+  String SESSION_HEADER = "SET-SESSION";

Review comment:
       Is there a particular reason why this is `SET-SESSION` as opposed to `Set-Session`? `Set-Session` would be more in line with other headers being used, such as `Set-Cookie, Authorization or Arrow-Properties`




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

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



[GitHub] [arrow] jduo commented on a change in pull request #8583: Arrow 10427: [Flight][Java] Add optional session header

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



##########
File path: java/flight/flight-core/src/test/java/org/apache/arrow/flight/TestServerSessionHandling.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.arrow.flight.grpc.RequestContextAdapter;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Tests for ServerSessionHandler and ServerSessionMiddleware.
+ */
+public class TestServerSessionHandling {
+  private static final String SESSION_HEADER_KEY = "Session";
+  private static final String SET_SESSION_HEADER_KEY = "Set-Session";
+  private static final String TEST_SESSION = "name=test-session;id=session-id;max-age=100";
+  private static final List<String> VALID_PROPERTY_KEYS = ImmutableList.of(
+          "name",
+          "id",
+          "max-age"
+  );
+
+  private class SimpleServerSessionHandler implements ServerSessionHandler {
+    private String session;
+
+    SimpleServerSessionHandler() {
+      this.session = null;
+    }
+
+    @Override
+    public String beginSession(CallHeaders headers) {
+      session = TEST_SESSION;
+      return session;
+    }
+
+    @Override
+    public String getSession(CallHeaders headers) {
+      // No session has been set yet
+      if (session == null) {
+        return null;
+      }
+
+      // Expected session header but none is found
+      if (!headers.containsKey(SESSION_HEADER_KEY)) {

Review comment:
       This code to parse the header should not be done by the implementation of the interface since we are generating and transmitting this header in the Flight layer (so this logic isn't FlightProducer-implementation-specific).




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

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