You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/09/17 00:08:17 UTC

[GitHub] [iceberg] danielcweeks opened a new pull request, #5781: Add REST Servlet/Server Implementations

danielcweeks opened a new pull request, #5781:
URL: https://github.com/apache/iceberg/pull/5781

   This PR provides a proxy implementation for a HttpServlet backed by a RESTCatalogAdaptor to expose the REST spec for any backend catalog implementation.


-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5781: Add REST Servlet/Server Implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5781:
URL: https://github.com/apache/iceberg/pull/5781#discussion_r978157553


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.iceberg.rest;
+
+import static java.lang.String.format;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hc.core5.http.ContentType;
+import org.apache.hc.core5.http.HttpHeaders;
+import org.apache.iceberg.exceptions.RESTException;
+import org.apache.iceberg.relocated.com.google.common.base.Splitter;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.io.CharStreams;
+import org.apache.iceberg.rest.RESTCatalogAdapter.HTTPMethod;
+import org.apache.iceberg.rest.RESTCatalogAdapter.Route;
+import org.apache.iceberg.rest.responses.ErrorResponse;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The RESTCatalogServlet provides a servlet implementation used in combination with a
+ * RESTCatalogAdaptor to proxy the REST Spec to any Catalog implementation.
+ */
+public class RESTCatalogServlet extends HttpServlet {
+  private static final Logger LOG = LoggerFactory.getLogger(RESTCatalogServlet.class);
+
+  private final RESTCatalogAdapter restCatalogAdapter;
+  private final Map<String, String> responseHeaders =
+      ImmutableMap.of(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType());
+
+  public RESTCatalogServlet(RESTCatalogAdapter restCatalogAdapter) {
+    this.restCatalogAdapter = restCatalogAdapter;
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doHead(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doDelete(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  protected void execute(ServletRequestContext context, HttpServletResponse response)
+      throws IOException {
+    response.setStatus(HttpServletResponse.SC_OK);
+    responseHeaders.forEach(response::setHeader);
+
+    if (context.error().isPresent()) {
+      response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+      RESTObjectMapper.mapper().writeValue(response.getWriter(), context.error().get());
+      return;
+    }
+
+    try {
+      Object responseBody =
+          restCatalogAdapter.execute(
+              context.method(),
+              context.path(),
+              context.queryParams(),
+              context.body(),
+              context.route().getResponseClass(),
+              context.headers(),
+              handle(response));
+
+      if (responseBody != null) {
+        RESTObjectMapper.mapper().writeValue(response.getWriter(), responseBody);
+      }
+    } catch (RESTException e) {
+      LOG.error("Error processing REST request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } catch (Exception e) {
+      LOG.error("Unexpected exception when processing REST request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    }
+  }
+
+  protected Consumer<ErrorResponse> handle(HttpServletResponse response) {
+    return (errorResponse) -> {
+      response.setStatus(errorResponse.code());
+      try {
+        RESTObjectMapper.mapper().writeValue(response.getWriter(), errorResponse);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    };
+  }
+
+  public static class ServletRequestContext {
+    private HTTPMethod method;
+    private Route route;
+    private String path;
+    private Map<String, String> headers;
+    private Map<String, String> queryParams;
+    private Object body;
+
+    private ErrorResponse errorResponse;
+
+    private ServletRequestContext(ErrorResponse errorResponse) {
+      this.errorResponse = errorResponse;
+    }
+
+    private ServletRequestContext(
+        HTTPMethod method,
+        Route route,
+        String path,
+        Map<String, String> headers,
+        Map<String, String> queryParams,
+        Object body) {
+      this.method = method;
+      this.route = route;
+      this.path = path;
+      this.headers = headers;
+      this.queryParams = queryParams;
+      this.body = body;
+    }
+
+    static ServletRequestContext from(HttpServletRequest request) throws IOException {
+      HTTPMethod method = HTTPMethod.valueOf(request.getMethod());
+      String path = request.getRequestURI().substring(1);
+      Pair<Route, Map<String, String>> routeContext = Route.from(method, path);
+
+      if (routeContext == null) {
+        return new ServletRequestContext(
+            ErrorResponse.builder()
+                .responseCode(400)
+                .withType("BadRequestException")
+                .withMessage(format("No route for request: %s %s", method, path))
+                .build());
+      }
+
+      Route route = routeContext.first();
+      Object requestBody = null;
+      if (route.getRequestClass() != null) {
+        requestBody =
+            RESTObjectMapper.mapper().readValue(request.getReader(), route.getRequestClass());
+      } else if (route == Route.TOKENS) {
+        try (Reader reader = new InputStreamReader(request.getInputStream())) {
+          Splitter.MapSplitter formSplitter = Splitter.on("&").withKeyValueSeparator("=");

Review Comment:
   This can be a constant. Does it also make sense to put this into `RESTUtil` rather than embedding it here?



##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.iceberg.rest;
+
+import static java.lang.String.format;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hc.core5.http.ContentType;
+import org.apache.hc.core5.http.HttpHeaders;
+import org.apache.iceberg.exceptions.RESTException;
+import org.apache.iceberg.relocated.com.google.common.base.Splitter;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.io.CharStreams;
+import org.apache.iceberg.rest.RESTCatalogAdapter.HTTPMethod;
+import org.apache.iceberg.rest.RESTCatalogAdapter.Route;
+import org.apache.iceberg.rest.responses.ErrorResponse;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The RESTCatalogServlet provides a servlet implementation used in combination with a
+ * RESTCatalogAdaptor to proxy the REST Spec to any Catalog implementation.
+ */
+public class RESTCatalogServlet extends HttpServlet {
+  private static final Logger LOG = LoggerFactory.getLogger(RESTCatalogServlet.class);
+
+  private final RESTCatalogAdapter restCatalogAdapter;
+  private final Map<String, String> responseHeaders =
+      ImmutableMap.of(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType());
+
+  public RESTCatalogServlet(RESTCatalogAdapter restCatalogAdapter) {
+    this.restCatalogAdapter = restCatalogAdapter;
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doHead(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doDelete(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  protected void execute(ServletRequestContext context, HttpServletResponse response)
+      throws IOException {
+    response.setStatus(HttpServletResponse.SC_OK);
+    responseHeaders.forEach(response::setHeader);
+
+    if (context.error().isPresent()) {
+      response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+      RESTObjectMapper.mapper().writeValue(response.getWriter(), context.error().get());
+      return;
+    }
+
+    try {
+      Object responseBody =
+          restCatalogAdapter.execute(
+              context.method(),
+              context.path(),
+              context.queryParams(),
+              context.body(),
+              context.route().getResponseClass(),
+              context.headers(),
+              handle(response));
+
+      if (responseBody != null) {
+        RESTObjectMapper.mapper().writeValue(response.getWriter(), responseBody);
+      }
+    } catch (RESTException e) {
+      LOG.error("Error processing REST request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } catch (Exception e) {
+      LOG.error("Unexpected exception when processing REST request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    }
+  }
+
+  protected Consumer<ErrorResponse> handle(HttpServletResponse response) {
+    return (errorResponse) -> {
+      response.setStatus(errorResponse.code());
+      try {
+        RESTObjectMapper.mapper().writeValue(response.getWriter(), errorResponse);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    };
+  }
+
+  public static class ServletRequestContext {
+    private HTTPMethod method;
+    private Route route;
+    private String path;
+    private Map<String, String> headers;
+    private Map<String, String> queryParams;
+    private Object body;
+
+    private ErrorResponse errorResponse;
+
+    private ServletRequestContext(ErrorResponse errorResponse) {
+      this.errorResponse = errorResponse;
+    }
+
+    private ServletRequestContext(
+        HTTPMethod method,
+        Route route,
+        String path,
+        Map<String, String> headers,
+        Map<String, String> queryParams,
+        Object body) {
+      this.method = method;
+      this.route = route;
+      this.path = path;
+      this.headers = headers;
+      this.queryParams = queryParams;
+      this.body = body;
+    }
+
+    static ServletRequestContext from(HttpServletRequest request) throws IOException {
+      HTTPMethod method = HTTPMethod.valueOf(request.getMethod());
+      String path = request.getRequestURI().substring(1);
+      Pair<Route, Map<String, String>> routeContext = Route.from(method, path);
+
+      if (routeContext == null) {
+        return new ServletRequestContext(
+            ErrorResponse.builder()
+                .responseCode(400)
+                .withType("BadRequestException")
+                .withMessage(format("No route for request: %s %s", method, path))
+                .build());
+      }
+
+      Route route = routeContext.first();
+      Object requestBody = null;
+      if (route.getRequestClass() != null) {
+        requestBody =
+            RESTObjectMapper.mapper().readValue(request.getReader(), route.getRequestClass());
+      } else if (route == Route.TOKENS) {
+        try (Reader reader = new InputStreamReader(request.getInputStream())) {
+          Splitter.MapSplitter formSplitter = Splitter.on("&").withKeyValueSeparator("=");

Review Comment:
   This can be a constant. Does it also make sense to put this payload parsing into `RESTUtil` rather than embedding it 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5781: Add REST Servlet/Server Implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5781:
URL: https://github.com/apache/iceberg/pull/5781#discussion_r978154163


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java:
##########
@@ -155,6 +191,14 @@ public static Pair<Route, Map<String, String>> from(HTTPMethod method, String pa
 
       return null;
     }
+
+    public Class<? extends RESTRequest> getRequestClass() {

Review Comment:
   Minor: we don't use `get` in methods names since it isn't adding anything. These should be `requestClass` and `responseClass`



-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5781: Add REST Servlet/Server Implementations

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5781:
URL: https://github.com/apache/iceberg/pull/5781#discussion_r978156104


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.iceberg.rest;
+
+import static java.lang.String.format;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hc.core5.http.ContentType;
+import org.apache.hc.core5.http.HttpHeaders;
+import org.apache.iceberg.exceptions.RESTException;
+import org.apache.iceberg.relocated.com.google.common.base.Splitter;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.io.CharStreams;
+import org.apache.iceberg.rest.RESTCatalogAdapter.HTTPMethod;
+import org.apache.iceberg.rest.RESTCatalogAdapter.Route;
+import org.apache.iceberg.rest.responses.ErrorResponse;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The RESTCatalogServlet provides a servlet implementation used in combination with a
+ * RESTCatalogAdaptor to proxy the REST Spec to any Catalog implementation.
+ */
+public class RESTCatalogServlet extends HttpServlet {
+  private static final Logger LOG = LoggerFactory.getLogger(RESTCatalogServlet.class);
+
+  private final RESTCatalogAdapter restCatalogAdapter;
+  private final Map<String, String> responseHeaders =
+      ImmutableMap.of(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType());
+
+  public RESTCatalogServlet(RESTCatalogAdapter restCatalogAdapter) {
+    this.restCatalogAdapter = restCatalogAdapter;
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doHead(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doDelete(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  protected void execute(ServletRequestContext context, HttpServletResponse response)
+      throws IOException {
+    response.setStatus(HttpServletResponse.SC_OK);
+    responseHeaders.forEach(response::setHeader);
+
+    if (context.error().isPresent()) {
+      response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+      RESTObjectMapper.mapper().writeValue(response.getWriter(), context.error().get());
+      return;
+    }
+
+    try {
+      Object responseBody =
+          restCatalogAdapter.execute(
+              context.method(),
+              context.path(),
+              context.queryParams(),
+              context.body(),
+              context.route().getResponseClass(),
+              context.headers(),
+              handle(response));
+
+      if (responseBody != null) {
+        RESTObjectMapper.mapper().writeValue(response.getWriter(), responseBody);
+      }
+    } catch (RESTException e) {
+      LOG.error("Error processing REST request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } catch (Exception e) {

Review Comment:
   Where is this thrown? The only exception I see thrown from `execute` is `RESTException`



-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks merged pull request #5781: Add REST Servlet/Server Implementations

Posted by GitBox <gi...@apache.org>.
danielcweeks merged PR #5781:
URL: https://github.com/apache/iceberg/pull/5781


-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #5781: Add REST Servlet/Server Implementations

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5781:
URL: https://github.com/apache/iceberg/pull/5781#discussion_r976070831


##########
versions.props:
##########
@@ -27,6 +27,7 @@ com.google.cloud:libraries-bom = 24.1.0
 org.scala-lang.modules:scala-collection-compat_2.12 = 2.6.0
 org.scala-lang.modules:scala-collection-compat_2.13 = 2.6.0
 com.emc.ecs:object-client-bundle = 3.3.2
+org.eclipse.jetty:* = 9.4.43.v20210629

Review Comment:
   nit: should probably go further down in this file under `#test deps`



-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #5781: Add REST Servlet/Server Implementations

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #5781:
URL: https://github.com/apache/iceberg/pull/5781#discussion_r978173563


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.iceberg.rest;
+
+import static java.lang.String.format;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hc.core5.http.ContentType;
+import org.apache.hc.core5.http.HttpHeaders;
+import org.apache.iceberg.exceptions.RESTException;
+import org.apache.iceberg.relocated.com.google.common.base.Splitter;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.io.CharStreams;
+import org.apache.iceberg.rest.RESTCatalogAdapter.HTTPMethod;
+import org.apache.iceberg.rest.RESTCatalogAdapter.Route;
+import org.apache.iceberg.rest.responses.ErrorResponse;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The RESTCatalogServlet provides a servlet implementation used in combination with a
+ * RESTCatalogAdaptor to proxy the REST Spec to any Catalog implementation.
+ */
+public class RESTCatalogServlet extends HttpServlet {
+  private static final Logger LOG = LoggerFactory.getLogger(RESTCatalogServlet.class);
+
+  private final RESTCatalogAdapter restCatalogAdapter;
+  private final Map<String, String> responseHeaders =
+      ImmutableMap.of(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType());
+
+  public RESTCatalogServlet(RESTCatalogAdapter restCatalogAdapter) {
+    this.restCatalogAdapter = restCatalogAdapter;
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doHead(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doDelete(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  protected void execute(ServletRequestContext context, HttpServletResponse response)
+      throws IOException {
+    response.setStatus(HttpServletResponse.SC_OK);
+    responseHeaders.forEach(response::setHeader);
+
+    if (context.error().isPresent()) {
+      response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+      RESTObjectMapper.mapper().writeValue(response.getWriter(), context.error().get());
+      return;
+    }
+
+    try {
+      Object responseBody =
+          restCatalogAdapter.execute(
+              context.method(),
+              context.path(),
+              context.queryParams(),
+              context.body(),
+              context.route().getResponseClass(),
+              context.headers(),
+              handle(response));
+
+      if (responseBody != null) {
+        RESTObjectMapper.mapper().writeValue(response.getWriter(), responseBody);
+      }
+    } catch (RESTException e) {
+      LOG.error("Error processing REST request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } catch (Exception e) {
+      LOG.error("Unexpected exception when processing REST request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    }
+  }
+
+  protected Consumer<ErrorResponse> handle(HttpServletResponse response) {
+    return (errorResponse) -> {
+      response.setStatus(errorResponse.code());
+      try {
+        RESTObjectMapper.mapper().writeValue(response.getWriter(), errorResponse);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    };
+  }
+
+  public static class ServletRequestContext {
+    private HTTPMethod method;
+    private Route route;
+    private String path;
+    private Map<String, String> headers;
+    private Map<String, String> queryParams;
+    private Object body;
+
+    private ErrorResponse errorResponse;
+
+    private ServletRequestContext(ErrorResponse errorResponse) {
+      this.errorResponse = errorResponse;
+    }
+
+    private ServletRequestContext(
+        HTTPMethod method,
+        Route route,
+        String path,
+        Map<String, String> headers,
+        Map<String, String> queryParams,
+        Object body) {
+      this.method = method;
+      this.route = route;
+      this.path = path;
+      this.headers = headers;
+      this.queryParams = queryParams;
+      this.body = body;
+    }
+
+    static ServletRequestContext from(HttpServletRequest request) throws IOException {
+      HTTPMethod method = HTTPMethod.valueOf(request.getMethod());
+      String path = request.getRequestURI().substring(1);
+      Pair<Route, Map<String, String>> routeContext = Route.from(method, path);
+
+      if (routeContext == null) {
+        return new ServletRequestContext(
+            ErrorResponse.builder()
+                .responseCode(400)
+                .withType("BadRequestException")
+                .withMessage(format("No route for request: %s %s", method, path))
+                .build());
+      }
+
+      Route route = routeContext.first();
+      Object requestBody = null;
+      if (route.getRequestClass() != null) {
+        requestBody =
+            RESTObjectMapper.mapper().readValue(request.getReader(), route.getRequestClass());
+      } else if (route == Route.TOKENS) {
+        try (Reader reader = new InputStreamReader(request.getInputStream())) {
+          Splitter.MapSplitter formSplitter = Splitter.on("&").withKeyValueSeparator("=");

Review Comment:
   Added decode (equivalent to existing encode) to `RESTUtil` along with corresponding test.



-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #5781: Add REST Servlet/Server Implementations

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #5781:
URL: https://github.com/apache/iceberg/pull/5781#discussion_r978163228


##########
core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.iceberg.rest;
+
+import static java.lang.String.format;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hc.core5.http.ContentType;
+import org.apache.hc.core5.http.HttpHeaders;
+import org.apache.iceberg.exceptions.RESTException;
+import org.apache.iceberg.relocated.com.google.common.base.Splitter;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.io.CharStreams;
+import org.apache.iceberg.rest.RESTCatalogAdapter.HTTPMethod;
+import org.apache.iceberg.rest.RESTCatalogAdapter.Route;
+import org.apache.iceberg.rest.responses.ErrorResponse;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The RESTCatalogServlet provides a servlet implementation used in combination with a
+ * RESTCatalogAdaptor to proxy the REST Spec to any Catalog implementation.
+ */
+public class RESTCatalogServlet extends HttpServlet {
+  private static final Logger LOG = LoggerFactory.getLogger(RESTCatalogServlet.class);
+
+  private final RESTCatalogAdapter restCatalogAdapter;
+  private final Map<String, String> responseHeaders =
+      ImmutableMap.of(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType());
+
+  public RESTCatalogServlet(RESTCatalogAdapter restCatalogAdapter) {
+    this.restCatalogAdapter = restCatalogAdapter;
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doHead(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  @Override
+  protected void doDelete(HttpServletRequest request, HttpServletResponse response)
+      throws IOException {
+    execute(ServletRequestContext.from(request), response);
+  }
+
+  protected void execute(ServletRequestContext context, HttpServletResponse response)
+      throws IOException {
+    response.setStatus(HttpServletResponse.SC_OK);
+    responseHeaders.forEach(response::setHeader);
+
+    if (context.error().isPresent()) {
+      response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+      RESTObjectMapper.mapper().writeValue(response.getWriter(), context.error().get());
+      return;
+    }
+
+    try {
+      Object responseBody =
+          restCatalogAdapter.execute(
+              context.method(),
+              context.path(),
+              context.queryParams(),
+              context.body(),
+              context.route().getResponseClass(),
+              context.headers(),
+              handle(response));
+
+      if (responseBody != null) {
+        RESTObjectMapper.mapper().writeValue(response.getWriter(), responseBody);
+      }
+    } catch (RESTException e) {
+      LOG.error("Error processing REST request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } catch (Exception e) {

Review Comment:
   That is true, but since this is a servlet implementation, this is a safeguard to make sure we return a response (instead of having an empty response) if anything goes wrong during server-side processing.



-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org