You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/12/14 08:22:33 UTC

[GitHub] [druid] jihoonson commented on a change in pull request #12026: Fix vulnerabilities in some HTTP endpoints

jihoonson commented on a change in pull request #12026:
URL: https://github.com/apache/druid/pull/12026#discussion_r768368220



##########
File path: indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
##########
@@ -1297,7 +1298,7 @@ public Response getPhaseName(@Context final HttpServletRequest req)
         return Response.ok(runner.getName()).build();
       }
     } else {
-      return Response.status(Status.BAD_REQUEST).entity("task is running in the sequential mode").build();
+      return ResponseStatusExceptionMapper.toResponse(Response.Status.BAD_REQUEST, "task is running in the sequential mode");

Review comment:
       This changes the response format of an user-facing API (https://druid.apache.org/docs/latest/ingestion/native-batch.html#http-status-endpoints). Why do we need this change for this PR?

##########
File path: indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
##########
@@ -587,11 +574,7 @@ public Response getTasks(
           authorizerMapper
       );
       if (!authResult.isAllowed()) {
-        throw new WebApplicationException(
-            Response.status(Response.Status.FORBIDDEN)
-                    .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString()))
-                    .build()
-        );
+        throw new ForbiddenException(StringUtils.format(authResult.toString()));

Review comment:
       Is this change accidental? This changes the response format.

##########
File path: indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java
##########
@@ -520,15 +521,16 @@ private Response specSuspendOrResume(final String id, boolean suspend)
             Response.Status status;
             String errMsg;
             if (spec.isPresent()) {
-              status = Response.Status.BAD_REQUEST;
-              errMsg = StringUtils.format("[%s] is already %s", id, suspend ? "suspended" : "running");
+              return ResponseStatusExceptionMapper.toResponse(Response.Status.BAD_REQUEST,
+                                                              StringUtils.format("[%s] is already %s", id, suspend ? "suspended" : "running"));
             } else {
               status = Response.Status.NOT_FOUND;
               errMsg = StringUtils.format("[%s] does not exist", id);
-            }
-            return Response.status(status)
+              return Response.status(status)
+                           .type(MediaType.APPLICATION_JSON_TYPE)
                            .entity(ImmutableMap.of("error", errMsg))
                            .build();
+            }

Review comment:
       Why not 
   
   ```suggestion
                 return ResponseStatusExceptionMapper.toResponse(Response.Status.NOT_FOUND, errMsg);
               }
   ```

##########
File path: indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java
##########
@@ -1447,6 +1452,28 @@ public void testDisableWorkerWhenWorkerAPIRaisesError()
     Assert.assertEquals(ImmutableMap.of("error", "Worker API returns error!"), response.getEntity());
   }
 
+  @Test
+  public void testGetMultipleTaskStatusesWithBadRequest()
+  {
+    // even though in this case methods on following object won't be trigger
+    // the tearDown of this test file verifies these objects.
+    // So, we need this function call
+    EasyMock.replay(
+        taskRunner,
+        taskMaster,
+        taskStorageQueryAdapter,
+        indexerMetadataStorageAdapter,
+        req,
+        workerTaskRunnerQueryAdapter
+    );
+
+    Response response = this.overlordResource.getMultipleTaskStatuses(Collections.emptySet());
+
+    Assert.assertEquals(response.getStatus(), HttpResponseStatus.BAD_REQUEST.getCode());
+    Assert.assertEquals(MediaType.valueOf(MediaType.APPLICATION_JSON), response.getMetadata().get("Content-Type").get(0));
+    Assert.assertTrue(((Map) response.getEntity()).containsKey("error"));

Review comment:
       Please verify the error message.

##########
File path: server/src/main/java/org/apache/druid/server/initialization/jetty/ResponseStatusExceptionMapper.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.druid.server.initialization.jetty;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+@Provider
+public class ResponseStatusExceptionMapper implements ExceptionMapper<ResponseStatusException>
+{
+  /**
+   * For Response.Status.BAD_REQUEST or Response.Status.NOT_FOUND,
+   * it's suggested to throw exception {@link BadRequestException} or {@link NotFoundException} respectively

Review comment:
       Why should we do this?

##########
File path: integration-tests/src/main/java/org/apache/druid/testing/clients/ResponseException.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.druid.testing.clients;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
+
+/**
+ * This class holds those responses whose HTTP status is > 200
+ * <p>
+ * This class inherits from {@link ISE} so that it's compatible with old test code which catches the ISE class
+ */
+public class ResponseException extends ISE
+{
+  private final StatusResponseHolder response;
+
+  public ResponseException(StatusResponseHolder response, String formatText, Object... arguments)
+  {
+    super(formatText, arguments);
+    this.response = response;
+  }
+
+  public StatusResponseHolder getResponse()
+  {
+    return response;
+  }
+
+  public <T> T responseBodyToObject(Class<T> clazz)
+  {
+    String type = response.getHeaders().get("Content-Type");
+    if (!"application/json".equals(type)) {
+      throw new ISE("Content-Type is [{%s], expected [application/json]", type);
+    }
+
+    ObjectMapper om = new ObjectMapper();

Review comment:
       You should inject `ObjectMapper` in the caller and then pass it into this method instead of creating a new one here. The injected objectMapper should have all required settings configured.

##########
File path: indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
##########
@@ -468,9 +458,7 @@ public Response getWorkerConfigHistory(
         return Response.ok(workerEntryList).build();
       }
       catch (IllegalArgumentException e) {
-        return Response.status(Response.Status.BAD_REQUEST)
-                       .entity(ImmutableMap.<String, Object>of("error", e.getMessage()))
-                       .build();
+        throw new BadRequestException(e.getMessage());

Review comment:
       This change is for another user-facing API. Why do we need this change for this PR? `IllegalArgumentException` looks too broad and I'm not sure all `IllegalArgumentExceptions` that can be thrown here contain an user-familiar error message.

##########
File path: indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java
##########
@@ -1010,13 +1011,17 @@ public void testGetTasksNegativeState()
         req,
         workerTaskRunnerQueryAdapter
     );
-    Object responseObject = overlordResource
-        .getTasks("blah", "ds_test", null, null, null, req)
-        .getEntity();
-    Assert.assertEquals(
-        "Invalid state : blah, valid values are: [pending, waiting, running, complete]",
-        responseObject.toString()
-    );
+    try {
+      overlordResource.getTasks("blah", "ds_test", null, null, null, req)
+                      .getEntity();
+      Assert.fail("Should throw BadRequestExceptoin");
+    }
+    catch (BadRequestException e) {

Review comment:
       nit: suggest using `ExpectedException` or `Assert.assertThrows`.

##########
File path: server/src/main/java/org/apache/druid/server/initialization/jetty/ResponseStatusExceptionMapper.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.druid.server.initialization.jetty;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+@Provider
+public class ResponseStatusExceptionMapper implements ExceptionMapper<ResponseStatusException>
+{
+  /**
+   * For Response.Status.BAD_REQUEST or Response.Status.NOT_FOUND,
+   * it's suggested to throw exception {@link BadRequestException} or {@link NotFoundException} respectively
+   * instead of returning a response object
+   */
+  public static Response toResponse(Response.Status statusCode, Exception e)
+  {
+    return toResponse(statusCode, e == null ? "null" : (e.getMessage() == null ? e.toString() : e.getMessage()));

Review comment:
       How about extracting sanitizing message as a new method in `ServletResourceUtils` and using it here, such as
   
   ```java
     public static String sanitizeException(@Nullable Throwable t)
     {
       return t == null ? "null" : (t.getMessage() == null ? t.toString() : t.getMessage());
     }
   ```

##########
File path: indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
##########
@@ -569,9 +557,7 @@ public Response getTasks(
     //check for valid state
     if (state != null) {
       if (!API_TASK_STATES.contains(StringUtils.toLowerCase(state))) {
-        return Response.status(Status.BAD_REQUEST)
-                       .entity(StringUtils.format("Invalid state : %s, valid values are: %s", state, API_TASK_STATES))
-                       .build();
+        throw new BadRequestException(StringUtils.format("Invalid state : %s, valid values are: %s", state, API_TASK_STATES));

Review comment:
       nit: this is an internal API which is not documented in user docs. The new error message could be useful when you debug this internal API, but wouldn't be that useful in general.




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org