You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2023/07/21 21:04:21 UTC

[pinot] branch master updated: Returning 403 status code in case of authorization failures (#11136)

This is an automated email from the ASF dual-hosted git repository.

jackie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 6a10bd8aca Returning 403 status code in case of authorization failures (#11136)
6a10bd8aca is described below

commit 6a10bd8acaf59c7d62af40873ad022c4858b69df
Author: soumitra-st <12...@users.noreply.github.com>
AuthorDate: Fri Jul 21 14:04:14 2023 -0700

    Returning 403 status code in case of authorization failures (#11136)
---
 .../broker/api/resources/PinotClientRequest.java   |  4 ++++
 .../requesthandler/BaseBrokerRequestHandler.java   |  6 ++++--
 .../MultiStageBrokerRequestHandler.java            |  8 ++++---
 .../api/resources/PinotQueryResource.java          | 14 ++++++++++--
 .../tests/BasicAuthBatchIntegrationTest.java       | 25 +++++++++++++---------
 5 files changed, 40 insertions(+), 17 deletions(-)

diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotClientRequest.java b/pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotClientRequest.java
index 30c8cb6d7b..5792579c3a 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotClientRequest.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotClientRequest.java
@@ -121,6 +121,8 @@ public class PinotClientRequest {
       }
       BrokerResponse brokerResponse = executeSqlQuery(requestJson, makeHttpIdentity(requestContext), true);
       asyncResponse.resume(brokerResponse.toJsonString());
+    } catch (WebApplicationException wae) {
+      asyncResponse.resume(wae);
     } catch (Exception e) {
       LOGGER.error("Caught exception while processing GET request", e);
       _brokerMetrics.addMeteredGlobalValue(BrokerMeter.UNCAUGHT_GET_EXCEPTIONS, 1L);
@@ -148,6 +150,8 @@ public class PinotClientRequest {
       BrokerResponse brokerResponse =
           executeSqlQuery((ObjectNode) requestJson, makeHttpIdentity(requestContext), false);
       asyncResponse.resume(brokerResponse.toJsonString());
+    } catch (WebApplicationException wae) {
+      asyncResponse.resume(wae);
     } catch (Exception e) {
       LOGGER.error("Caught exception while processing POST request", e);
       _brokerMetrics.addMeteredGlobalValue(BrokerMeter.UNCAUGHT_POST_EXCEPTIONS, 1L);
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
index e64251493d..5921435aab 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
@@ -37,6 +37,8 @@ import java.util.concurrent.TimeoutException;
 import java.util.stream.Collectors;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.httpclient.HttpConnectionManager;
 import org.apache.commons.httpclient.URI;
@@ -242,7 +244,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
       _brokerMetrics.addMeteredGlobalValue(BrokerMeter.REQUEST_DROPPED_DUE_TO_ACCESS_ERROR, 1);
       LOGGER.info("Access denied for requestId {}", requestId);
       requestContext.setErrorCode(QueryException.ACCESS_DENIED_ERROR_CODE);
-      return new BrokerResponseNative(QueryException.ACCESS_DENIED_ERROR);
+      throw new WebApplicationException("Permission denied", Response.Status.FORBIDDEN);
     }
 
     JsonNode sql = request.get(Broker.Request.SQL);
@@ -386,7 +388,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
         _brokerMetrics.addMeteredTableValue(tableName, BrokerMeter.REQUEST_DROPPED_DUE_TO_ACCESS_ERROR, 1);
         LOGGER.info("Access denied for request {}: {}, table: {}", requestId, query, tableName);
         requestContext.setErrorCode(QueryException.ACCESS_DENIED_ERROR_CODE);
-        return new BrokerResponseNative(QueryException.ACCESS_DENIED_ERROR);
+        throw new WebApplicationException("Permission denied", Response.Status.FORBIDDEN);
       }
       _brokerMetrics.addPhaseTiming(rawTableName, BrokerQueryPhase.AUTHORIZATION,
           System.nanoTime() - compilationEndTimeNs);
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java
index 7328dacb8c..2f10ed7e37 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java
@@ -27,6 +27,8 @@ import java.util.Set;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response;
 import org.apache.calcite.jdbc.CalciteSchemaBuilder;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.pinot.broker.api.RequesterIdentity;
@@ -134,7 +136,7 @@ public class MultiStageBrokerRequestHandler extends BaseBrokerRequestHandler {
       _brokerMetrics.addMeteredGlobalValue(BrokerMeter.REQUEST_DROPPED_DUE_TO_ACCESS_ERROR, 1);
       LOGGER.info("Access denied for requestId {}", requestId);
       requestContext.setErrorCode(QueryException.ACCESS_DENIED_ERROR_CODE);
-      return new BrokerResponseNative(QueryException.ACCESS_DENIED_ERROR);
+      throw new WebApplicationException("Permission denied", Response.Status.FORBIDDEN);
     }
 
     JsonNode sql = request.get(CommonConstants.Broker.Request.SQL);
@@ -167,7 +169,7 @@ public class MultiStageBrokerRequestHandler extends BaseBrokerRequestHandler {
           String plan = queryPlanResult.getExplainPlan();
           Set<String> tableNames = queryPlanResult.getTableNames();
           if (!hasTableAccess(requesterIdentity, tableNames, requestId, requestContext)) {
-            return new BrokerResponseNative(QueryException.ACCESS_DENIED_ERROR);
+            throw new WebApplicationException("Permission denied", Response.Status.FORBIDDEN);
           }
 
           return constructMultistageExplainPlan(query, plan);
@@ -193,7 +195,7 @@ public class MultiStageBrokerRequestHandler extends BaseBrokerRequestHandler {
 
     // Validate table access.
     if (!hasTableAccess(requesterIdentity, tableNames, requestId, requestContext)) {
-      return new BrokerResponseNative(QueryException.ACCESS_DENIED_ERROR);
+      throw new WebApplicationException("Permission denied", Response.Status.FORBIDDEN);
     }
     updatePhaseTimingForTables(tableNames, BrokerQueryPhase.AUTHORIZATION, System.nanoTime() - compilationEndTimeNs);
 
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
index a8fb80f421..8cddd62e3d 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
@@ -44,8 +44,10 @@ import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.Path;
 import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.Response;
 import org.apache.calcite.jdbc.CalciteSchemaBuilder;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.commons.io.IOUtils;
@@ -117,6 +119,9 @@ public class PinotQueryResource {
     } catch (ProcessingException pe) {
       LOGGER.error("Caught exception while processing post request {}", pe.getMessage());
       return pe.getMessage();
+    } catch (WebApplicationException wae) {
+      LOGGER.error("Caught exception while processing post request", wae);
+      throw wae;
     } catch (Exception e) {
       LOGGER.error("Caught exception while processing post request", e);
       return QueryException.getException(QueryException.INTERNAL_ERROR, e).toString();
@@ -133,6 +138,9 @@ public class PinotQueryResource {
     } catch (ProcessingException pe) {
       LOGGER.error("Caught exception while processing get request {}", pe.getMessage());
       return pe.getMessage();
+    } catch (WebApplicationException wae) {
+      LOGGER.error("Caught exception while processing get request", wae);
+      throw wae;
     } catch (Exception e) {
       LOGGER.error("Caught exception while processing get request", e);
       return QueryException.getException(QueryException.INTERNAL_ERROR, e).toString();
@@ -187,7 +195,7 @@ public class PinotQueryResource {
     // we don't have a cross table access control rule so only ADMIN can make request to multi-stage engine.
     AccessControl accessControl = _accessControlFactory.create();
     if (!accessControl.hasAccess(null, AccessType.READ, httpHeaders, endpointUrl)) {
-      return QueryException.ACCESS_DENIED_ERROR.toString();
+      throw new WebApplicationException("Permission denied", Response.Status.FORBIDDEN);
     }
 
     QueryEnvironment queryEnvironment = new QueryEnvironment(new TypeFactory(new TypeSystem()),
@@ -390,7 +398,9 @@ public class PinotQueryResource {
       /*if (LOG.isInfoEnabled()){
         LOGGER.info("The http response code is " + responseCode);
       }*/
-      if (responseCode != HttpURLConnection.HTTP_OK) {
+      if (responseCode == HttpURLConnection.HTTP_FORBIDDEN) {
+        throw new WebApplicationException("Permission denied", Response.Status.FORBIDDEN);
+      } else if (responseCode != HttpURLConnection.HTTP_OK) {
         throw new IOException("Failed : HTTP error code : " + responseCode + ". Root Cause: "
             + IOUtils.toString(conn.getErrorStream(), StandardCharsets.UTF_8));
       }
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasicAuthBatchIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasicAuthBatchIntegrationTest.java
index 39ad829fda..fb209db1f4 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasicAuthBatchIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BasicAuthBatchIntegrationTest.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.pinot.common.auth.AuthProviderUtils;
+import org.apache.pinot.common.exception.HttpErrorStatusException;
 import org.apache.pinot.controller.helix.core.minion.generator.PinotTaskGenerator;
 import org.apache.pinot.minion.executor.PinotTaskExecutor;
 import org.apache.pinot.spi.env.PinotConfiguration;
@@ -99,10 +100,12 @@ public class BasicAuthBatchIntegrationTest extends ClusterTest {
   @Test
   public void testBrokerNoAuth()
       throws Exception {
-    JsonNode response = JsonUtils.stringToJsonNode(
-        sendPostRequest("http://localhost:" + getRandomBrokerPort() + "/query/sql", "{\"sql\":\"SELECT now()\"}"));
-    Assert.assertFalse(response.has("resultTable"), "must not return result table");
-    Assert.assertTrue(response.get("exceptions").get(0).get("errorCode").asInt() != 0, "must return error code");
+    try {
+        sendPostRequest("http://localhost:" + getRandomBrokerPort() + "/query/sql", "{\"sql\":\"SELECT now()\"}");
+    } catch (IOException e) {
+      HttpErrorStatusException httpErrorStatusException = (HttpErrorStatusException) e.getCause();
+      Assert.assertEquals(httpErrorStatusException.getStatusCode(), 403, "must return 403");
+    }
   }
 
   @Test
@@ -176,11 +179,13 @@ public class BasicAuthBatchIntegrationTest extends ClusterTest {
         "must return row count 97889");
     Assert.assertTrue(response.get("exceptions").isEmpty(), "must not return exception");
 
-    // user with valid auth but no table access
-    JsonNode responseUser = JsonUtils.stringToJsonNode(
-        sendPostRequest("http://localhost:" + getRandomBrokerPort() + "/query/sql",
-            "{\"sql\":\"SELECT count(*) FROM baseballStats\"}", AUTH_HEADER_USER));
-    Assert.assertFalse(responseUser.has("resultTable"), "must not return result table");
-    Assert.assertTrue(responseUser.get("exceptions").get(0).get("errorCode").asInt() != 0, "must return error code");
+    // user with valid auth but no table access - must return 403
+    try {
+      sendPostRequest("http://localhost:" + getRandomBrokerPort() + "/query/sql",
+          "{\"sql\":\"SELECT count(*) FROM baseballStats\"}", AUTH_HEADER_USER);
+    } catch (IOException e) {
+      HttpErrorStatusException httpErrorStatusException = (HttpErrorStatusException) e.getCause();
+      Assert.assertEquals(httpErrorStatusException.getStatusCode(), 403, "must return 403");
+    }
   }
 }


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