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 2022/04/19 08:28:19 UTC

[GitHub] [druid] kfaraz commented on a diff in pull request #12396: Add support for authorizing query context params

kfaraz commented on code in PR #12396:
URL: https://github.com/apache/druid/pull/12396#discussion_r852592971


##########
processing/src/main/java/org/apache/druid/query/Query.java:
##########
@@ -95,8 +94,11 @@
 
   DateTimeZone getTimezone();
 
+  @Deprecated

Review Comment:
   Nit: Please add a comment/javadoc about the deprecation and the alternative.



##########
integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java:
##########
@@ -612,9 +696,20 @@ protected StatusResponseHolder makeSQLQueryRequest(
       String query,
       HttpResponseStatus expectedStatus
   ) throws Exception
+  {
+    return makeSQLQueryRequest(httpClient, query, ImmutableMap.of(), expectedStatus);
+  }
+
+  protected StatusResponseHolder makeSQLQueryRequest(

Review Comment:
   Suggestion: Maybe rename to something like `makeSqlRequestAndVerifyStatus`.



##########
server/src/main/java/org/apache/druid/server/QueryResource.java:
##########
@@ -364,7 +341,12 @@ public void write(OutputStream outputStream) throws WebApplicationException
 
       log.noStackTrace()
          .makeAlert(e, "Exception handling request")
-         .addData("query", query != null ? jsonMapper.writeValueAsString(query) : "unparseable query")

Review Comment:
   Nit: The original version seemed more concise. As we are using `queryLifecycle.getQuery()` in several places, assigning to a final variable seems reasonable.



##########
sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidJoinRule.java:
##########
@@ -74,7 +73,7 @@ private DruidJoinRule(final PlannerContext plannerContext)
             operand(DruidRel.class, any())
         )
     );
-    this.enableLeftScanDirect = QueryContexts.getEnableJoinLeftScanDirect(plannerContext.getQueryContext());

Review Comment:
   `isEnableJoinLeftScanDirect` seems too specific a use case to be a part of the base `QueryContext` class itself. Why have we moved it from `QueryContexts`?



##########
sql/src/main/java/org/apache/druid/sql/SqlLifecycle.java:
##########
@@ -135,32 +139,29 @@ public SqlLifecycle(
    *
    * If successful (it will be), it will transition the lifecycle to {@link State#INITIALIZED}.
    */
-  public String initialize(String sql, Map<String, Object> queryContext)
+  public String initialize(String sql, QueryContext queryAndContext)

Review Comment:
   Nit:
   ```suggestion
     public String initialize(String sql, QueryContext queryContext)
   ```



##########
sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java:
##########
@@ -113,8 +114,13 @@ public void tearDown() throws Exception
   public void testSignature()
   {
     final String sql = "SELECT * FROM druid.foo";
-    final DruidStatement statement = new DruidStatement("", 0, null, sqlLifecycleFactory.factorize(), () -> {
-    }).prepare(sql, -1, AllowAllAuthenticator.ALLOW_ALL_RESULT);
+    final DruidStatement statement = new DruidStatement(
+        "",
+        0,
+        new QueryContext(),
+        sqlLifecycleFactory.factorize(),
+        () -> {}
+    ).prepare(sql, -1, AllowAllAuthenticator.ALLOW_ALL_RESULT);

Review Comment:
   Nit: Maybe add a `createStatement(sql)` private util method for this to avoid code repetition.



##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java:
##########
@@ -158,43 +156,37 @@ public boolean isUseNativeQueryExplain()
     return useNativeQueryExplain;
   }
 
-  public PlannerConfig withOverrides(final Map<String, Object> context)
+  public PlannerConfig withOverrides(final QueryContext queryAndContext)

Review Comment:
   Suggestion: Rename to `queryContext`



##########
server/src/main/java/org/apache/druid/server/QueryLifecycle.java:
##########
@@ -173,19 +189,10 @@ public void initialize(final Query baseQuery)
   {
     transition(State.NEW, State.INITIALIZED);
 
-    String queryId = baseQuery.getId();
-    if (Strings.isNullOrEmpty(queryId)) {
-      queryId = UUID.randomUUID().toString();
-    }
-
-    Map<String, Object> mergedUserAndConfigContext;
-    if (baseQuery.getContext() != null) {
-      mergedUserAndConfigContext = BaseQuery.computeOverriddenContext(defaultQueryConfig.getContext(), baseQuery.getContext());
-    } else {
-      mergedUserAndConfigContext = defaultQueryConfig.getContext();
-    }
+    baseQuery.getQueryContext().addDefaultParam(BaseQuery.QUERY_ID, UUID.randomUUID().toString());

Review Comment:
   Much cleaner now!



##########
processing/src/main/java/org/apache/druid/query/QueryContext.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.query;
+
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Numbers;
+
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+
+/**
+ * Holder for query context parameters. There are 3 ways to set context params today.
+ *
+ * - Default parameters. These are set mostly via {@link DefaultQueryConfig#context}.
+ *   Auto-generated queryId or sqlQueryId are also set as default parameters. These default parameters can
+ *   be overridden by user or system parameters.
+ * - User parameters. These are the params set by the user. User params override default parameters but
+ *   are overridden by system paramters.
+ * - System parameters. These are the params set by the Druid query engine for internal use only.
+ *
+ * You can use {@code getX} methods or {@link #getMergedParams()} to compute the context params
+ * merging 3 types of params above.
+ *
+ * Currently, this class is mainly used for query context parameter authorization in query entires,
+ * such as HTTP query endpoints or JDBC endpoint. Its usage can be expanded in the future if we
+ * want to track user parameters and separate them from others during query processing.
+ */
+public class QueryContext
+{
+  private final Map<String, Object> defaultParams;
+  private final Map<String, Object> userParams;
+  private final Map<String, Object> systemParams;
+
+  /**
+   * Cache of params merged.
+   */
+  @Nullable
+  private Map<String, Object> mergedParams;
+
+  public QueryContext()
+  {
+    this(null);
+  }
+
+  public QueryContext(@Nullable Map<String, Object> userParams)
+  {
+    this.defaultParams = new TreeMap<>();
+    this.userParams = userParams == null ? new TreeMap<>() : new TreeMap<>(userParams);
+    this.systemParams = new TreeMap<>();
+    invalidateMergedParams();
+  }
+
+  private void invalidateMergedParams()
+  {
+    this.mergedParams = null;
+  }
+
+  public boolean isEmpty()
+  {
+    return defaultParams.isEmpty() && userParams.isEmpty() && systemParams.isEmpty();
+  }
+
+  public void addDefaultParam(String key, Object val)
+  {
+    invalidateMergedParams();
+    defaultParams.put(key, val);
+  }
+
+  public void addDefaultParams(Map<String, Object> defaultParams)
+  {
+    invalidateMergedParams();
+    this.defaultParams.putAll(defaultParams);
+  }
+
+  public void addSystemParam(String key, Object val)
+  {
+    invalidateMergedParams();
+    this.systemParams.put(key, val);
+  }
+
+  public Object removeUserParam(String key)
+  {
+    invalidateMergedParams();
+    return userParams.remove(key);
+  }
+
+  /**
+   * Returns only the context parameters the user sets.
+   * The returned map does not include the parameters that have been removed via {@link #removeUserParam}.
+   *
+   * Callers should use {@code getX} methods or {@link #getMergedParams()} instead to use the whole context params.
+   */
+  public Map<String, Object> getUserParams()
+  {
+    return userParams;
+  }
+
+  public boolean isDebug()
+  {
+    return getAsBoolean(QueryContexts.ENABLE_DEBUG, QueryContexts.DEFAULT_ENABLE_DEBUG);
+  }
+
+  public boolean isEnableJoinLeftScanDirect()
+  {
+    return getAsBoolean(
+        QueryContexts.SQL_JOIN_LEFT_SCAN_DIRECT,
+        QueryContexts.DEFAULT_ENABLE_SQL_JOIN_LEFT_SCAN_DIRECT
+    );
+  }
+
+  @Nullable
+  public Object get(String key)
+  {
+    Object val = systemParams.get(key);
+    if (val != null) {
+      return val;
+    }
+    val = userParams.get(key);
+    return val == null ? defaultParams.get(key) : val;
+  }
+
+  @Nullable
+  public String getAsString(String key)
+  {
+    return (String) get(key);
+  }
+
+  public boolean getAsBoolean(
+      final String parameter,
+      final boolean defaultValue
+  )
+  {
+    final Object value = get(parameter);
+    if (value == null) {
+      return defaultValue;
+    } else if (value instanceof String) {
+      return Boolean.parseBoolean((String) value);
+    } else if (value instanceof Boolean) {
+      return (Boolean) value;
+    } else {
+      throw new IAE("Expected parameter[%s] to be boolean", parameter);
+    }
+  }
+
+  public int getAsInt(
+      final String parameter,
+      final int defaultValue
+  )
+  {
+    final Object value = get(parameter);
+    if (value == null) {
+      return defaultValue;
+    } else if (value instanceof String) {
+      return Numbers.parseInt(value);
+    } else if (value instanceof Number) {
+      return ((Number) value).intValue();
+    } else {
+      throw new IAE("Expected parameter[%s] to be integer", parameter);
+    }
+  }
+
+  public long getAsLong(final String parameter, final long defaultValue)
+  {
+    final Object value = get(parameter);
+    if (value == null) {
+      return defaultValue;
+    } else if (value instanceof String) {
+      return Numbers.parseLong(value);
+    } else if (value instanceof Number) {
+      return ((Number) value).longValue();
+    } else {
+      throw new IAE("Expected parameter[%s] to be long", parameter);
+    }
+  }
+
+  public Map<String, Object> getMergedParams()
+  {
+    if (mergedParams == null) {
+      final Map<String, Object> merged = new TreeMap<>(defaultParams);
+      merged.putAll(userParams);
+      merged.putAll(systemParams);
+      mergedParams = Collections.unmodifiableMap(merged);
+    }
+    return mergedParams;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    QueryContext context = (QueryContext) o;
+    return getMergedParams().equals(context.getMergedParams());

Review Comment:
   Nit:
   As we are comparing only the final merged params, two QueryContext objects that are currently equal might not be so after performing the same operation (say `removeUserParam`) on the two of them.
   
   For example:
   Context1: userParam={p1=10}, systemParam={}
   Context2: userParam={}, systemParam={p1=10}
   These two are currently equal.
   But after performing the same operation, say `removeUserParam(p1)`, the two contexts will not remain equal anymore.
   
   I guess this should be okay?



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java:
##########
@@ -8269,10 +8280,14 @@ public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename() throws E
     cannotVectorize();
     requireMergeBuffers(3);
     testQuery(
-        PLANNER_CONFIG_NO_HLL.withOverrides(ImmutableMap.of(
-            PlannerConfig.CTX_KEY_USE_GROUPING_SET_FOR_EXACT_DISTINCT,
-            "true"
-        )),
+        PLANNER_CONFIG_NO_HLL.withOverrides(
+            new QueryContext(
+                ImmutableMap.of(
+                    PlannerConfig.CTX_KEY_USE_GROUPING_SET_FOR_EXACT_DISTINCT,
+                    "true"
+                )
+            )

Review Comment:
   Nit: Maybe add a private util method for this? This code is present in 4 places.



##########
integration-tests/src/test/java/org/apache/druid/tests/security/AbstractAuthConfigurationTest.java:
##########
@@ -188,22 +199,23 @@
 
   protected HttpClient adminClient;
   protected HttpClient datasourceOnlyUserClient;
+  protected HttpClient datasourceAndContextParamsClient;
   protected HttpClient datasourceAndSysUserClient;
   protected HttpClient datasourceWithStateUserClient;
   protected HttpClient stateOnlyUserClient;
   protected HttpClient internalSystemClient;
 
 
   protected abstract void setupDatasourceOnlyUser() throws Exception;
+  protected abstract void setupDatasourceAndContextParamsUser() throws Exception;
   protected abstract void setupDatasourceAndSysTableUser() throws Exception;
   protected abstract void setupDatasourceAndSysAndStateUser() throws Exception;
   protected abstract void setupSysTableAndStateOnlyUser() throws Exception;
   protected abstract void setupTestSpecificHttpClients() throws Exception;
   protected abstract String getAuthenticatorName();
   protected abstract String getAuthorizerName();
   protected abstract String getExpectedAvaticaAuthError();
-  protected abstract Properties getAvaticaConnectionProperties();

Review Comment:
   Nit:
   Maybe we should just add a new `testAvaticaQuery(properties, url)`and leave the existing`testAvaticaQuery` as is?
   Otherwise, we are forced to pass the properties to `testAvaticaQuery` in every downstream 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: 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