You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by cw...@apache.org on 2019/07/24 02:25:53 UTC

[incubator-druid] branch 0.15.1-incubating created (now 4404e07)

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

cwylie pushed a change to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git.


      at 4404e07  fix backport

This branch includes the following new commits:

     new bdc1ca7  sketches-core-0.13.4 (#7666)
     new 2abb652  add bloom filter fallback aggregator when types are unknown (#7719)
     new cf6e464  fix timestamp ceil lower bound bug (#7823)
     new 70ef2da  discard filter when processing subtotalsSpec (#7827)
     new b88dbf1  WorkerTaskManager to create disk files atomically and ignore task file corruption (#7917)
     new 21d3600  Fix ExpressionVirtualColumn capabilities; fix groupBy's improper uses of StorageAdapter#getColumnCapabilities. (#8013)
     new 79ae3d4  set DRUID_AUTHORIZATION_CHECKED attribute for router endpoints (#8026)
     new bff1c6c  SupervisorManager: Add authorization checks to bulk endpoints. (#8044)
     new 0ada251  force native order when wrapping ByteBuffer since Druid can have it set (#8055)
     new e67dc89  Improve pull-deps reference in extensions page. (#8002)
     new c804426  Add missing reference to Materialized-View extension. (#8003)
     new 36bc4e0  Fix documentation formatting (#8079)
     new 9f03e45  fix references to bin/supervise in tutorial docs (#8087)
     new 4404e07  fix backport

The 14 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



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


[incubator-druid] 09/14: force native order when wrapping ByteBuffer since Druid can have it set (#8055)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit 0ada25132cbbf1e04b4a0ae7877681e70dc5f2d2
Author: Alexander Saydakov <13...@users.noreply.github.com>
AuthorDate: Thu Jul 11 17:17:53 2019 -0700

    force native order when wrapping ByteBuffer since Druid can have it set (#8055)
    
    incorrectly
---
 .../datasketches/quantiles/DoublesSketchMergeBufferAggregator.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchMergeBufferAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchMergeBufferAggregator.java
index f5a1e9d..a4766ac 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchMergeBufferAggregator.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchMergeBufferAggregator.java
@@ -28,6 +28,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 import org.apache.druid.segment.ColumnValueSelector;
 
 import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.IdentityHashMap;
 
 public class DoublesSketchMergeBufferAggregator implements BufferAggregator
@@ -113,7 +114,7 @@ public class DoublesSketchMergeBufferAggregator implements BufferAggregator
 
   private WritableMemory getMemory(final ByteBuffer buffer)
   {
-    return memCache.computeIfAbsent(buffer, buf -> WritableMemory.wrap(buf));
+    return memCache.computeIfAbsent(buffer, buf -> WritableMemory.wrap(buf, ByteOrder.LITTLE_ENDIAN));
   }
 
   private void putUnion(final ByteBuffer buffer, final int position, final DoublesUnion union)


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


[incubator-druid] 03/14: fix timestamp ceil lower bound bug (#7823)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit cf6e464f507d12d46b6c4ccc5d7982afc5a6cf45
Author: Xue Yu <27...@qq.com>
AuthorDate: Tue Jun 4 16:16:31 2019 +0800

    fix timestamp ceil lower bound bug (#7823)
---
 .../query/expression/TimestampCeilExprMacro.java   | 15 +++-
 .../druid/query/expression/ExprMacroTest.java      |  2 +
 .../apache/druid/sql/calcite/CalciteQueryTest.java | 83 ++++++++++++++++++++++
 3 files changed, 98 insertions(+), 2 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java
index a775c69..37b35d8 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java
@@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.granularity.PeriodGranularity;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExprMacroTable;
+import org.joda.time.DateTime;
 
 import javax.annotation.Nonnull;
 import java.util.List;
@@ -72,7 +73,12 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
         // Return null if the argument if null.
         return ExprEval.of(null);
       }
-      return ExprEval.of(granularity.bucketEnd(DateTimes.utc(arg.eval(bindings).asLong())).getMillis());
+      DateTime argTime = DateTimes.utc(arg.eval(bindings).asLong());
+      DateTime bucketStartTime = granularity.bucketStart(argTime);
+      if (argTime.equals(bucketStartTime)) {
+        return ExprEval.of(bucketStartTime.getMillis());
+      }
+      return ExprEval.of(granularity.increment(bucketStartTime).getMillis());
     }
 
     @Override
@@ -107,7 +113,12 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
     public ExprEval eval(final ObjectBinding bindings)
     {
       final PeriodGranularity granularity = getGranularity(args, bindings);
-      return ExprEval.of(granularity.bucketEnd(DateTimes.utc(args.get(0).eval(bindings).asLong())).getMillis());
+      DateTime argTime = DateTimes.utc(args.get(0).eval(bindings).asLong());
+      DateTime bucketStartTime = granularity.bucketStart(argTime);
+      if (argTime.equals(bucketStartTime)) {
+        return ExprEval.of(bucketStartTime.getMillis());
+      }
+      return ExprEval.of(granularity.increment(bucketStartTime).getMillis());
     }
 
     @Override
diff --git a/server/src/test/java/org/apache/druid/query/expression/ExprMacroTest.java b/server/src/test/java/org/apache/druid/query/expression/ExprMacroTest.java
index bbab646..d79cff1 100644
--- a/server/src/test/java/org/apache/druid/query/expression/ExprMacroTest.java
+++ b/server/src/test/java/org/apache/druid/query/expression/ExprMacroTest.java
@@ -34,6 +34,7 @@ public class ExprMacroTest
   private static final Expr.ObjectBinding BINDINGS = Parser.withMap(
       ImmutableMap.<String, Object>builder()
           .put("t", DateTimes.of("2000-02-03T04:05:06").getMillis())
+          .put("t1", DateTimes.of("2000-02-03T00:00:00").getMillis())
           .put("tstr", "2000-02-03T04:05:06")
           .put("tstr_sql", "2000-02-03 04:05:06")
           .put("x", "foo")
@@ -88,6 +89,7 @@ public class ExprMacroTest
     assertExpr("timestamp_ceil(t, 'P1D',null,'America/Los_Angeles')", DateTimes.of("2000-02-03T08").getMillis());
     assertExpr("timestamp_ceil(t, 'P1D',null,CityOfAngels)", DateTimes.of("2000-02-03T08").getMillis());
     assertExpr("timestamp_ceil(t, 'P1D','1970-01-01T01','Etc/UTC')", DateTimes.of("2000-02-04T01").getMillis());
+    assertExpr("timestamp_ceil(t1, 'P1D')", DateTimes.of("2000-02-03").getMillis());
   }
 
   @Test
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index fb4976f..f4bc6ae 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -7684,4 +7684,87 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
         )
     );
   }
+
+  @Test
+  public void testTimestampDiff() throws Exception
+  {
+    testQuery(
+        "SELECT TIMESTAMPDIFF(DAY, TIMESTAMP '1999-01-01 00:00:00', __time), \n"
+        + "TIMESTAMPDIFF(DAY, __time, DATE '2001-01-01'), \n"
+        + "TIMESTAMPDIFF(HOUR, TIMESTAMP '1999-12-31 01:00:00', __time), \n"
+        + "TIMESTAMPDIFF(MINUTE, TIMESTAMP '1999-12-31 23:58:03', __time), \n"
+        + "TIMESTAMPDIFF(SECOND, TIMESTAMP '1999-12-31 23:59:03', __time), \n"
+        + "TIMESTAMPDIFF(MONTH, TIMESTAMP '1999-11-01 00:00:00', __time), \n"
+        + "TIMESTAMPDIFF(YEAR, TIMESTAMP '1996-11-01 00:00:00', __time), \n"
+        + "TIMESTAMPDIFF(QUARTER, TIMESTAMP '1996-10-01 00:00:00', __time), \n"
+        + "TIMESTAMPDIFF(WEEK, TIMESTAMP '1998-10-01 00:00:00', __time) \n"
+        + "FROM druid.foo\n"
+        + "LIMIT 2",
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(CalciteTests.DATASOURCE1)
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .virtualColumns(
+                    expressionVirtualColumn("v0", "div((\"__time\" - 915148800000),86400000)", ValueType.LONG),
+                    expressionVirtualColumn("v1", "div((978307200000 - \"__time\"),86400000)", ValueType.LONG),
+                    expressionVirtualColumn("v2", "div((\"__time\" - 946602000000),3600000)", ValueType.LONG),
+                    expressionVirtualColumn("v3", "div((\"__time\" - 946684683000),60000)", ValueType.LONG),
+                    expressionVirtualColumn("v4", "div((\"__time\" - 946684743000),1000)", ValueType.LONG),
+                    expressionVirtualColumn("v5", "subtract_months(\"__time\",941414400000,'UTC')", ValueType.LONG),
+                    expressionVirtualColumn("v6", "div(subtract_months(\"__time\",846806400000,'UTC'),12)", ValueType.LONG),
+                    expressionVirtualColumn("v7", "div(subtract_months(\"__time\",844128000000,'UTC'),3)", ValueType.LONG),
+                    expressionVirtualColumn("v8", "div(div((\"__time\" - 907200000000),1000),604800)", ValueType.LONG)
+                )
+                .columns("v0", "v1", "v2", "v3", "v4", "v5", "v6", "v7", "v8")
+                .limit(2)
+                .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+
+        ),
+        ImmutableList.of(
+            new Object[]{365, 366, 23, 1, 57, 2, 3, 13, 65},
+            new Object[]{366, 365, 47, 1441, 86457, 2, 3, 13, 65}
+        )
+    );
+  }
+
+  @Test
+  public void testTimestampCeil() throws Exception
+  {
+    testQuery(
+        "SELECT CEIL(TIMESTAMP '2000-01-01 00:00:00' TO DAY), \n"
+        + "CEIL(TIMESTAMP '2000-01-01 01:00:00' TO DAY) \n"
+        + "FROM druid.foo\n"
+        + "LIMIT 1",
+        ImmutableList.of(
+            newScanQueryBuilder()
+                .dataSource(CalciteTests.DATASOURCE1)
+                .intervals(querySegmentSpec(Filtration.eternity()))
+                .virtualColumns(
+                    expressionVirtualColumn("v0", "946684800000", ValueType.LONG),
+                    expressionVirtualColumn("v1", "946771200000", ValueType.LONG)
+                )
+                .columns("v0", "v1")
+                .limit(1)
+                .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                .context(QUERY_CONTEXT_DEFAULT)
+                .build()
+
+        ),
+        ImmutableList.of(
+            new Object[]{
+                Calcites.jodaToCalciteTimestamp(
+                    DateTimes.of("2000-01-01"),
+                    DateTimeZone.UTC
+                ),
+                Calcites.jodaToCalciteTimestamp(
+                    DateTimes.of("2000-01-02"),
+                    DateTimeZone.UTC
+                )
+            }
+        )
+    );
+  }
+
 }


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


[incubator-druid] 07/14: set DRUID_AUTHORIZATION_CHECKED attribute for router endpoints (#8026)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit 79ae3d4761397d2b825d0cc72f24f700cdc2855e
Author: Parag Jain <pj...@users.noreply.github.com>
AuthorDate: Tue Jul 9 13:21:36 2019 +0530

    set DRUID_AUTHORIZATION_CHECKED attribute for router endpoints (#8026)
    
    * add state resource filter to router endpoints
    
    * add RouterResource to ResourceFilter test framework
---
 .../main/java/org/apache/druid/server/http/RouterResource.java    | 3 +++
 .../org/apache/druid/server/security/AuthenticationUtils.java     | 2 +-
 .../org/apache/druid/server/security/UnsecuredResourceFilter.java | 8 ++++++--
 .../druid/server/http/security/SecurityResourceFilterTest.java    | 8 ++++----
 services/src/main/java/org/apache/druid/cli/CliOverlord.java      | 6 +++---
 .../org/apache/druid/cli/CoordinatorJettyServerInitializer.java   | 8 ++++----
 .../org/apache/druid/cli/MiddleManagerJettyServerInitializer.java | 6 +++---
 .../java/org/apache/druid/cli/QueryJettyServerInitializer.java    | 4 ++--
 .../java/org/apache/druid/cli/RouterJettyServerInitializer.java   | 8 ++++----
 9 files changed, 30 insertions(+), 23 deletions(-)

diff --git a/server/src/main/java/org/apache/druid/server/http/RouterResource.java b/server/src/main/java/org/apache/druid/server/http/RouterResource.java
index df30855..20da9af 100644
--- a/server/src/main/java/org/apache/druid/server/http/RouterResource.java
+++ b/server/src/main/java/org/apache/druid/server/http/RouterResource.java
@@ -20,7 +20,9 @@
 package org.apache.druid.server.http;
 
 import com.google.inject.Inject;
+import com.sun.jersey.spi.container.ResourceFilters;
 import org.apache.druid.client.selector.Server;
+import org.apache.druid.server.http.security.StateResourceFilter;
 import org.apache.druid.server.router.TieredBrokerHostSelector;
 
 import javax.ws.rs.GET;
@@ -47,6 +49,7 @@ public class RouterResource
 
   @GET
   @Path("/brokers")
+  @ResourceFilters(StateResourceFilter.class)
   @Produces(MediaType.APPLICATION_JSON)
   public Map<String, List<String>> getBrokers()
   {
diff --git a/server/src/main/java/org/apache/druid/server/security/AuthenticationUtils.java b/server/src/main/java/org/apache/druid/server/security/AuthenticationUtils.java
index a9438cd..924f23e 100644
--- a/server/src/main/java/org/apache/druid/server/security/AuthenticationUtils.java
+++ b/server/src/main/java/org/apache/druid/server/security/AuthenticationUtils.java
@@ -57,7 +57,7 @@ public class AuthenticationUtils
     }
   }
 
-  public static void addNoopAuthorizationFilters(ServletContextHandler root, List<String> unsecuredPaths)
+  public static void addNoopAuthenticationAndAuthorizationFilters(ServletContextHandler root, List<String> unsecuredPaths)
   {
     for (String unsecuredPath : unsecuredPaths) {
       root.addFilter(new FilterHolder(new UnsecuredResourceFilter()), unsecuredPath, null);
diff --git a/server/src/main/java/org/apache/druid/server/security/UnsecuredResourceFilter.java b/server/src/main/java/org/apache/druid/server/security/UnsecuredResourceFilter.java
index 6f79771..0d73ba2 100644
--- a/server/src/main/java/org/apache/druid/server/security/UnsecuredResourceFilter.java
+++ b/server/src/main/java/org/apache/druid/server/security/UnsecuredResourceFilter.java
@@ -47,9 +47,13 @@ public class UnsecuredResourceFilter implements Filter
     // but the value doesn't matter since we skip authorization checks for requests that go through this filter
     servletRequest.setAttribute(
         AuthConfig.DRUID_AUTHENTICATION_RESULT,
-        new AuthenticationResult(AuthConfig.ALLOW_ALL_NAME, AuthConfig.ALLOW_ALL_NAME, AuthConfig.ALLOW_ALL_NAME, null)
+        new AuthenticationResult(
+            AuthConfig.ALLOW_ALL_NAME,
+            AuthConfig.ALLOW_ALL_NAME,
+            AuthConfig.ALLOW_ALL_NAME,
+            null
+        )
     );
-
     // This request will not go to an Authorizer, so we need to set this for PreResponseAuthorizationCheckFilter
     servletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
     servletRequest.setAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH, true);
diff --git a/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java b/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java
index c3de4c8..30a1c76 100644
--- a/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java
+++ b/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java
@@ -34,6 +34,7 @@ import org.apache.druid.server.http.DataSourcesResource;
 import org.apache.druid.server.http.HistoricalResource;
 import org.apache.druid.server.http.IntervalsResource;
 import org.apache.druid.server.http.MetadataResource;
+import org.apache.druid.server.http.RouterResource;
 import org.apache.druid.server.http.RulesResource;
 import org.apache.druid.server.http.ServersResource;
 import org.apache.druid.server.http.TiersResource;
@@ -46,14 +47,12 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import java.util.Collection;
-import java.util.regex.Pattern;
 
 @RunWith(Parameterized.class)
 public class SecurityResourceFilterTest extends ResourceFilterTestHelper
 {
-  private static final Pattern WORD = Pattern.compile("\\w+");
 
-  @Parameterized.Parameters
+  @Parameterized.Parameters(name = "{index}: requestPath={0}, requestMethod={1}, resourceFilter={2}")
   public static Collection<Object[]> data()
   {
     return ImmutableList.copyOf(
@@ -71,7 +70,8 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper
             getRequestPathsWithAuthorizer(CoordinatorDynamicConfigsResource.class),
             getRequestPathsWithAuthorizer(QueryResource.class),
             getRequestPathsWithAuthorizer(StatusResource.class),
-            getRequestPathsWithAuthorizer(BrokerQueryResource.class)
+            getRequestPathsWithAuthorizer(BrokerQueryResource.class),
+            getRequestPathsWithAuthorizer(RouterResource.class)
         )
     );
   }
diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java
index c11031f..e70ff4e2 100644
--- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java
+++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java
@@ -375,9 +375,9 @@ public class CliOverlord extends ServerRunnable
 
       AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper);
 
-      // perform no-op authorization for these resources
-      AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS);
-      AuthenticationUtils.addNoopAuthorizationFilters(root, authConfig.getUnsecuredPaths());
+      // perform no-op authorization/authentication for these resources
+      AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, UNSECURED_PATHS);
+      AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, authConfig.getUnsecuredPaths());
 
       final List<Authenticator> authenticators = authenticatorMapper.getAuthenticatorChain();
       AuthenticationUtils.addAuthenticationFilterChain(root, authenticators);
diff --git a/services/src/main/java/org/apache/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/org/apache/druid/cli/CoordinatorJettyServerInitializer.java
index 91064f5..9cad393 100644
--- a/services/src/main/java/org/apache/druid/cli/CoordinatorJettyServerInitializer.java
+++ b/services/src/main/java/org/apache/druid/cli/CoordinatorJettyServerInitializer.java
@@ -101,12 +101,12 @@ class CoordinatorJettyServerInitializer implements JettyServerInitializer
 
     AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper);
 
-    // perform no-op authorization for these resources
-    AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS);
-    AuthenticationUtils.addNoopAuthorizationFilters(root, authConfig.getUnsecuredPaths());
+    // perform no-op authorization/authentication for these resources
+    AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, UNSECURED_PATHS);
+    AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, authConfig.getUnsecuredPaths());
 
     if (beOverlord) {
-      AuthenticationUtils.addNoopAuthorizationFilters(root, CliOverlord.UNSECURED_PATHS);
+      AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, CliOverlord.UNSECURED_PATHS);
     }
 
     List<Authenticator> authenticators = authenticatorMapper.getAuthenticatorChain();
diff --git a/services/src/main/java/org/apache/druid/cli/MiddleManagerJettyServerInitializer.java b/services/src/main/java/org/apache/druid/cli/MiddleManagerJettyServerInitializer.java
index b544f3b..1cb3782 100644
--- a/services/src/main/java/org/apache/druid/cli/MiddleManagerJettyServerInitializer.java
+++ b/services/src/main/java/org/apache/druid/cli/MiddleManagerJettyServerInitializer.java
@@ -74,9 +74,9 @@ class MiddleManagerJettyServerInitializer implements JettyServerInitializer
 
     AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper);
 
-    // perform no-op authorization for these resources
-    AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS);
-    AuthenticationUtils.addNoopAuthorizationFilters(root, authConfig.getUnsecuredPaths());
+    // perform no-op authorization/authentication for these resources
+    AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, UNSECURED_PATHS);
+    AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, authConfig.getUnsecuredPaths());
 
     final List<Authenticator> authenticators = authenticatorMapper.getAuthenticatorChain();
     AuthenticationUtils.addAuthenticationFilterChain(root, authenticators);
diff --git a/services/src/main/java/org/apache/druid/cli/QueryJettyServerInitializer.java b/services/src/main/java/org/apache/druid/cli/QueryJettyServerInitializer.java
index 2c92602..9282ca3 100644
--- a/services/src/main/java/org/apache/druid/cli/QueryJettyServerInitializer.java
+++ b/services/src/main/java/org/apache/druid/cli/QueryJettyServerInitializer.java
@@ -96,8 +96,8 @@ public class QueryJettyServerInitializer implements JettyServerInitializer
     AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper);
 
     // perform no-op authorization for these resources
-    AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS);
-    AuthenticationUtils.addNoopAuthorizationFilters(root, authConfig.getUnsecuredPaths());
+    AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, UNSECURED_PATHS);
+    AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, authConfig.getUnsecuredPaths());
 
     List<Authenticator> authenticators = authenticatorMapper.getAuthenticatorChain();
     AuthenticationUtils.addAuthenticationFilterChain(root, authenticators);
diff --git a/services/src/main/java/org/apache/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/org/apache/druid/cli/RouterJettyServerInitializer.java
index 596dba9..9fb2a61 100644
--- a/services/src/main/java/org/apache/druid/cli/RouterJettyServerInitializer.java
+++ b/services/src/main/java/org/apache/druid/cli/RouterJettyServerInitializer.java
@@ -137,12 +137,12 @@ public class RouterJettyServerInitializer implements JettyServerInitializer
 
     AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper);
 
-    // perform no-op authorization for these resources
-    AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS);
+    // perform no-op authorization/authentication for these resources
+    AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, UNSECURED_PATHS);
     if (managementProxyConfig.isEnabled()) {
-      AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS_FOR_UI);
+      AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, UNSECURED_PATHS_FOR_UI);
     }
-    AuthenticationUtils.addNoopAuthorizationFilters(root, authConfig.getUnsecuredPaths());
+    AuthenticationUtils.addNoopAuthenticationAndAuthorizationFilters(root, authConfig.getUnsecuredPaths());
 
     final List<Authenticator> authenticators = authenticatorMapper.getAuthenticatorChain();
     AuthenticationUtils.addAuthenticationFilterChain(root, authenticators);


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


[incubator-druid] 13/14: fix references to bin/supervise in tutorial docs (#8087)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit 9f03e4595cbe90c444150d6fb9de552a88d971bc
Author: Clint Wylie <cw...@apache.org>
AuthorDate: Tue Jul 23 15:05:01 2019 -0700

    fix references to bin/supervise in tutorial docs (#8087)
---
 docs/content/tutorials/tutorial-batch-hadoop.md | 2 +-
 docs/content/tutorials/tutorial-tranquility.md  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/content/tutorials/tutorial-batch-hadoop.md b/docs/content/tutorials/tutorial-batch-hadoop.md
index 3ed7239..01dffeb 100644
--- a/docs/content/tutorials/tutorial-batch-hadoop.md
+++ b/docs/content/tutorials/tutorial-batch-hadoop.md
@@ -196,7 +196,7 @@ druid.indexer.logs.directory=/druid/indexing-logs
 
 Once the Hadoop .xml files have been copied to the Druid cluster and the segment/log storage configuration has been updated to use HDFS, the Druid cluster needs to be restarted for the new configurations to take effect.
 
-If the cluster is still running, CTRL-C to terminate the `bin/supervise` script, and re-reun it to bring the Druid services back up.
+If the cluster is still running, CTRL-C to terminate the `bin/start-micro-quickstart` script, and re-reun it to bring the Druid services back up.
 
 ## Load batch data
 
diff --git a/docs/content/tutorials/tutorial-tranquility.md b/docs/content/tutorials/tutorial-tranquility.md
index 0c2fbb7..fbf3e13 100644
--- a/docs/content/tutorials/tutorial-tranquility.md
+++ b/docs/content/tutorials/tutorial-tranquility.md
@@ -49,7 +49,7 @@ The startup scripts for the tutorial will expect the contents of the Tranquility
 ## Enable Tranquility Server
 
 - In your `conf/supervise/single-server/micro-quickstart.conf`, uncomment the `tranquility-server` line.
-- Stop your *bin/supervise* command (CTRL-C) and then restart it by again running `bin/supervise -c conf/supervise/single-server/micro-quickstart.conf`.
+- Stop *micro-quickstart* cluster command (CTRL-C) then restart it again by running `bin/start-micro-quickstart`
 
 As part of the output of *supervise* you should see something like:
 


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


[incubator-druid] 10/14: Improve pull-deps reference in extensions page. (#8002)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit e67dc893ee37d77c41ffc34c49d61707972c67c2
Author: Eyal Yurman <ey...@oath.com>
AuthorDate: Mon Jul 1 11:18:27 2019 -0700

    Improve pull-deps reference in extensions page. (#8002)
---
 docs/content/development/extensions.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md
index 2190793..6e8bb3a 100644
--- a/docs/content/development/extensions.md
+++ b/docs/content/development/extensions.md
@@ -74,7 +74,7 @@ Community extensions are not maintained by Druid committers, although we accept
 A number of community members have contributed their own extensions to Druid that are not packaged with the default Druid tarball.
 If you'd like to take on maintenance for a community extension, please post on [dev@druid.apache.org](https://lists.apache.org/list.html?dev@druid.apache.org) to let us know!
 
-All of these community extensions can be downloaded using *pull-deps* with the coordinate org.apache.druid.extensions.contrib:EXTENSION_NAME:LATEST_DRUID_STABLE_VERSION.
+All of these community extensions can be downloaded using [pull-deps](../operations/pull-deps.html) while specifying a `-c` coordinate option to pull `org.apache.druid.extensions.contrib:{EXTENSION_NAME}:{DRUID_VERSION}`.
 
 |Name|Description|Docs|
 |----|-----------|----|


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


[incubator-druid] 02/14: add bloom filter fallback aggregator when types are unknown (#7719)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit 2abb65266131c63670019ba38f55a12471b0dbbc
Author: Clint Wylie <cw...@apache.org>
AuthorDate: Thu Jun 6 14:39:32 2019 -0700

    add bloom filter fallback aggregator when types are unknown (#7719)
---
 .../bloom/BloomFilterAggregatorFactory.java        | 149 +++++++++------------
 .../bloom/BloomFilterMergeAggregator.java          |   2 +-
 ...gator.java => ObjectBloomFilterAggregator.java} |  42 +++---
 .../bloom/StringBloomFilterAggregator.java         |   6 +-
 .../bloom/BloomFilterGroupByQueryTest.java         |  79 ++++++++++-
 5 files changed, 175 insertions(+), 103 deletions(-)

diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java
index 2c633f2..53839e8 100644
--- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java
+++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java
@@ -82,95 +82,13 @@ public class BloomFilterAggregatorFactory extends AggregatorFactory
   @Override
   public Aggregator factorize(ColumnSelectorFactory columnFactory)
   {
-    ColumnCapabilities capabilities = columnFactory.getColumnCapabilities(field.getDimension());
-
-    if (capabilities == null) {
-      BaseNullableColumnValueSelector selector = columnFactory.makeColumnValueSelector(field.getDimension());
-      if (selector instanceof NilColumnValueSelector) {
-        // BloomKFilter must be the same size so we cannot use a constant for the empty agg
-        return new NoopBloomFilterAggregator(maxNumEntries, true);
-      }
-      throw new IAE(
-          "Cannot create bloom filter buffer aggregator for column selector type [%s]",
-          selector.getClass().getName()
-      );
-    }
-    ValueType type = capabilities.getType();
-    switch (type) {
-      case STRING:
-        return new StringBloomFilterAggregator(
-            columnFactory.makeDimensionSelector(field),
-            maxNumEntries,
-            true
-        );
-      case LONG:
-        return new LongBloomFilterAggregator(
-            columnFactory.makeColumnValueSelector(field.getDimension()),
-            maxNumEntries,
-            true
-        );
-      case FLOAT:
-        return new FloatBloomFilterAggregator(
-            columnFactory.makeColumnValueSelector(field.getDimension()),
-            maxNumEntries,
-            true
-        );
-      case DOUBLE:
-        return new DoubleBloomFilterAggregator(
-            columnFactory.makeColumnValueSelector(field.getDimension()),
-            maxNumEntries,
-            true
-        );
-      default:
-        throw new IAE("Cannot create bloom filter aggregator for invalid column type [%s]", type);
-    }
+    return factorizeInternal(columnFactory, true);
   }
 
   @Override
   public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnFactory)
   {
-    ColumnCapabilities capabilities = columnFactory.getColumnCapabilities(field.getDimension());
-
-    if (capabilities == null) {
-      BaseNullableColumnValueSelector selector = columnFactory.makeColumnValueSelector(field.getDimension());
-      if (selector instanceof NilColumnValueSelector) {
-        return new NoopBloomFilterAggregator(maxNumEntries, false);
-      }
-      throw new IAE(
-          "Cannot create bloom filter buffer aggregator for column selector type [%s]",
-          selector.getClass().getName()
-      );
-    }
-
-    ValueType type = capabilities.getType();
-    switch (type) {
-      case STRING:
-        return new StringBloomFilterAggregator(
-            columnFactory.makeDimensionSelector(field),
-            maxNumEntries,
-            false
-        );
-      case LONG:
-        return new LongBloomFilterAggregator(
-            columnFactory.makeColumnValueSelector(field.getDimension()),
-            maxNumEntries,
-            false
-        );
-      case FLOAT:
-        return new FloatBloomFilterAggregator(
-            columnFactory.makeColumnValueSelector(field.getDimension()),
-            maxNumEntries,
-            false
-        );
-      case DOUBLE:
-        return new DoubleBloomFilterAggregator(
-            columnFactory.makeColumnValueSelector(field.getDimension()),
-            maxNumEntries,
-            false
-        );
-      default:
-        throw new IAE("Cannot create bloom filter buffer aggregator for invalid column type [%s]", type);
-    }
+    return factorizeInternal(columnFactory, false);
   }
 
   @Override
@@ -310,4 +228,67 @@ public class BloomFilterAggregatorFactory extends AggregatorFactory
            ", maxNumEntries=" + maxNumEntries +
            '}';
   }
+
+  private BaseBloomFilterAggregator factorizeInternal(ColumnSelectorFactory columnFactory, boolean onHeap)
+  {
+    if (field == null || field.getDimension() == null) {
+      return new NoopBloomFilterAggregator(maxNumEntries, onHeap);
+    }
+
+    ColumnCapabilities capabilities = columnFactory.getColumnCapabilities(field.getDimension());
+
+    if (capabilities != null) {
+      ValueType type = capabilities.getType();
+      switch (type) {
+        case STRING:
+          return new StringBloomFilterAggregator(
+              columnFactory.makeDimensionSelector(field),
+              maxNumEntries,
+              onHeap
+          );
+        case LONG:
+          return new LongBloomFilterAggregator(
+              columnFactory.makeColumnValueSelector(field.getDimension()),
+              maxNumEntries,
+              onHeap
+          );
+        case FLOAT:
+          return new FloatBloomFilterAggregator(
+              columnFactory.makeColumnValueSelector(field.getDimension()),
+              maxNumEntries,
+              onHeap
+          );
+        case DOUBLE:
+          return new DoubleBloomFilterAggregator(
+              columnFactory.makeColumnValueSelector(field.getDimension()),
+              maxNumEntries,
+              onHeap
+          );
+        case COMPLEX:
+          // in an ideal world, we would check complex type, but until then assume it's a bloom filter
+          return new BloomFilterMergeAggregator(
+              columnFactory.makeColumnValueSelector(field.getDimension()),
+              maxNumEntries,
+              onHeap
+          );
+        default:
+          throw new IAE(
+              "Cannot create bloom filter %s for invalid column type [%s]",
+              onHeap ? "aggregator" : "buffer aggregator",
+              type
+          );
+      }
+    } else {
+      BaseNullableColumnValueSelector selector = columnFactory.makeColumnValueSelector(field.getDimension());
+      if (selector instanceof NilColumnValueSelector) {
+        return new NoopBloomFilterAggregator(maxNumEntries, onHeap);
+      }
+      // no column capabilities, use fallback 'object' aggregator
+      return new ObjectBloomFilterAggregator(
+          columnFactory.makeColumnValueSelector(field.getDimension()),
+          maxNumEntries,
+          onHeap
+      );
+    }
+  }
 }
diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregator.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregator.java
index 011f2f6..6855d83 100644
--- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregator.java
+++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregator.java
@@ -27,7 +27,7 @@ import java.nio.ByteBuffer;
 
 public final class BloomFilterMergeAggregator extends BaseBloomFilterAggregator<ColumnValueSelector<ByteBuffer>>
 {
-  public BloomFilterMergeAggregator(ColumnValueSelector<ByteBuffer> selector, int maxNumEntries, boolean onHeap)
+  BloomFilterMergeAggregator(ColumnValueSelector<ByteBuffer> selector, int maxNumEntries, boolean onHeap)
   {
     super(selector, maxNumEntries, onHeap);
   }
diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/ObjectBloomFilterAggregator.java
similarity index 51%
copy from extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java
copy to extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/ObjectBloomFilterAggregator.java
index f3f6dae..bc97fab 100644
--- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java
+++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/ObjectBloomFilterAggregator.java
@@ -19,37 +19,47 @@
 
 package org.apache.druid.query.aggregation.bloom;
 
+import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.query.filter.BloomKFilter;
+import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.DimensionSelector;
 
 import java.nio.ByteBuffer;
 
-public final class StringBloomFilterAggregator extends BaseBloomFilterAggregator<DimensionSelector>
+/**
+ * Handles "unknown" columns by examining what comes out of the selector
+ */
+class ObjectBloomFilterAggregator extends BaseBloomFilterAggregator<ColumnValueSelector>
 {
-
-  StringBloomFilterAggregator(DimensionSelector selector, int maxNumEntries, boolean onHeap)
+  ObjectBloomFilterAggregator(
+      ColumnValueSelector selector,
+      int maxNumEntries,
+      boolean onHeap
+  )
   {
     super(selector, maxNumEntries, onHeap);
   }
 
   @Override
-  public void bufferAdd(ByteBuffer buf)
+  void bufferAdd(ByteBuffer buf)
   {
-    if (selector.getRow().size() > 1) {
-      selector.getRow().forEach(v -> {
-        String value = selector.lookupName(v);
-        if (value == null) {
-          BloomKFilter.addBytes(buf, null, 0, 0);
+    final Object object = selector.getObject();
+    if (object instanceof ByteBuffer) {
+      final ByteBuffer other = (ByteBuffer) object;
+      BloomKFilter.mergeBloomFilterByteBuffers(buf, buf.position(), other, other.position());
+    } else {
+      if (NullHandling.replaceWithDefault() || !selector.isNull()) {
+        if (object instanceof Long) {
+          BloomKFilter.addLong(buf, selector.getLong());
+        } else if (object instanceof Double) {
+          BloomKFilter.addDouble(buf, selector.getDouble());
+        } else if (object instanceof Float) {
+          BloomKFilter.addFloat(buf, selector.getFloat());
         } else {
-          BloomKFilter.addString(buf, value);
+          StringBloomFilterAggregator.stringBufferAdd(buf, (DimensionSelector) selector);
         }
-      });
-    } else {
-      String value = (String) selector.getObject();
-      if (value == null) {
-        BloomKFilter.addBytes(buf, null, 0, 0);
       } else {
-        BloomKFilter.addString(buf, value);
+        BloomKFilter.addBytes(buf, null, 0, 0);
       }
     }
   }
diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java
index f3f6dae..db65ca5 100644
--- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java
+++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java
@@ -26,7 +26,6 @@ import java.nio.ByteBuffer;
 
 public final class StringBloomFilterAggregator extends BaseBloomFilterAggregator<DimensionSelector>
 {
-
   StringBloomFilterAggregator(DimensionSelector selector, int maxNumEntries, boolean onHeap)
   {
     super(selector, maxNumEntries, onHeap);
@@ -35,6 +34,11 @@ public final class StringBloomFilterAggregator extends BaseBloomFilterAggregator
   @Override
   public void bufferAdd(ByteBuffer buf)
   {
+    stringBufferAdd(buf, selector);
+  }
+
+  static void stringBufferAdd(ByteBuffer buf, DimensionSelector selector)
+  {
     if (selector.getRow().size() > 1) {
       selector.getRow().forEach(v -> {
         String value = selector.lookupName(v);
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java
index ce3b932..5661e7b 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java
@@ -32,6 +32,7 @@ import org.apache.druid.query.aggregation.AggregationTestHelper;
 import org.apache.druid.query.filter.BloomKFilter;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
 import org.apache.druid.segment.TestHelper;
 import org.junit.After;
 import org.junit.Assert;
@@ -61,6 +62,7 @@ public class BloomFilterGroupByQueryTest
   }
 
   private AggregationTestHelper helper;
+  private boolean isV2;
 
   @Rule
   public final TemporaryFolder tempFolder = new TemporaryFolder();
@@ -72,6 +74,7 @@ public class BloomFilterGroupByQueryTest
         config,
         tempFolder
     );
+    isV2 = config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2);
   }
 
   @Parameterized.Parameters(name = "{0}")
@@ -93,7 +96,6 @@ public class BloomFilterGroupByQueryTest
   @Test
   public void testQuery() throws Exception
   {
-
     String query = "{"
                    + "\"queryType\": \"groupBy\","
                    + "\"dataSource\": \"test_datasource\","
@@ -116,6 +118,81 @@ public class BloomFilterGroupByQueryTest
   }
 
   @Test
+  public void testNestedQuery() throws Exception
+  {
+    if (!isV2) {
+      return;
+    }
+
+    String query = "{"
+                   + "\"queryType\": \"groupBy\","
+                   + "\"dataSource\": {"
+                   + "\"type\": \"query\","
+                   + "\"query\": {"
+                   + "\"queryType\":\"groupBy\","
+                   + "\"dataSource\": \"test_datasource\","
+                   + "\"intervals\": [ \"1970/2050\" ],"
+                   + "\"granularity\":\"ALL\","
+                   + "\"dimensions\":[],"
+                   + "\"aggregations\": [{ \"type\":\"longSum\", \"name\":\"innerSum\", \"fieldName\":\"count\"}]"
+                   + "}"
+                   + "},"
+                   + "\"granularity\": \"ALL\","
+                   + "\"dimensions\": [],"
+                   + "\"aggregations\": ["
+                   + "  { \"type\": \"bloom\", \"name\": \"bloom\", \"field\": \"innerSum\" }"
+                   + "],"
+                   + "\"intervals\": [ \"1970/2050\" ]"
+                   + "}";
+
+    MapBasedRow row = ingestAndQuery(query);
+
+
+    BloomKFilter filter = BloomKFilter.deserialize((ByteBuffer) row.getRaw("bloom"));
+    Assert.assertTrue(filter.testLong(13L));
+    Assert.assertFalse(filter.testLong(5L));
+  }
+
+
+  @Test
+  public void testNestedQueryComplex() throws Exception
+  {
+    if (!isV2) {
+      return;
+    }
+
+    String query = "{"
+                   + "\"queryType\": \"groupBy\","
+                   + "\"dataSource\": {"
+                   + "\"type\": \"query\","
+                   + "\"query\": {"
+                   + "\"queryType\":\"groupBy\","
+                   + "\"dataSource\": \"test_datasource\","
+                   + "\"intervals\": [ \"1970/2050\" ],"
+                   + "\"granularity\":\"ALL\","
+                   + "\"dimensions\":[],"
+                   + "\"filter\":{ \"type\":\"selector\", \"dimension\":\"market\", \"value\":\"upfront\"},"
+                   + "\"aggregations\": [{ \"type\":\"bloom\", \"name\":\"innerBloom\", \"field\":\"quality\"}]"
+                   + "}"
+                   + "},"
+                   + "\"granularity\": \"ALL\","
+                   + "\"dimensions\": [],"
+                   + "\"aggregations\": ["
+                   + "  { \"type\": \"bloom\", \"name\": \"innerBloom\", \"field\": \"innerBloom\" }"
+                   + "],"
+                   + "\"intervals\": [ \"1970/2050\" ]"
+                   + "}";
+
+    MapBasedRow row = ingestAndQuery(query);
+
+
+    BloomKFilter filter = BloomKFilter.deserialize((ByteBuffer) row.getRaw("innerBloom"));
+    Assert.assertTrue(filter.testString("mezzanine"));
+    Assert.assertTrue(filter.testString("premium"));
+    Assert.assertFalse(filter.testString("entertainment"));
+  }
+
+  @Test
   public void testQueryFakeDimension() throws Exception
   {
     String query = "{"


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


[incubator-druid] 01/14: sketches-core-0.13.4 (#7666)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit bdc1ca7ab65bd98c9acf686d5ac767cbea8f6a19
Author: Alexander Saydakov <13...@users.noreply.github.com>
AuthorDate: Thu Jun 6 14:36:52 2019 -0700

    sketches-core-0.13.4 (#7666)
---
 extensions-core/datasketches/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml
index 11708a7..ffa8332 100644
--- a/extensions-core/datasketches/pom.xml
+++ b/extensions-core/datasketches/pom.xml
@@ -38,7 +38,7 @@
     <dependency>
       <groupId>com.yahoo.datasketches</groupId>
       <artifactId>sketches-core</artifactId>
-      <version>0.13.3</version>
+      <version>0.13.4</version>
       <exclusions>
         <exclusion>
           <groupId>com.google.code.findbugs</groupId>


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


[incubator-druid] 05/14: WorkerTaskManager to create disk files atomically and ignore task file corruption (#7917)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit b88dbf15c64c662563f15b0359eddaf68ccec2e9
Author: Himanshu <g....@gmail.com>
AuthorDate: Tue Jun 18 09:18:43 2019 -0700

    WorkerTaskManager to create disk files atomically and ignore task file corruption (#7917)
    
    * WorkerTaskManager to create disk files atomically and ignore task file
    corruptions
    
    * fixing weird checkstyle lambda indentation issues
---
 .../apache/druid/java/util/common/FileUtils.java   | 16 +++++---
 .../druid/indexing/worker/WorkerTaskManager.java   | 46 +++++++++++++++++++---
 2 files changed, 50 insertions(+), 12 deletions(-)

diff --git a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java
index 7be41a8..17f0ed0 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java
@@ -176,6 +176,15 @@ public class FileUtils
 
   /**
    * Write to a file atomically, by first writing to a temporary file in the same directory and then moving it to
+   * the target location. More docs at {@link FileUtils#writeAtomically(File, File, OutputStreamConsumer)} .
+   */
+  public static <T> T writeAtomically(final File file, OutputStreamConsumer<T> f) throws IOException
+  {
+    return writeAtomically(file, file.getParentFile(), f);
+  }
+
+  /**
+   * Write to a file atomically, by first writing to a temporary file in given tmpDir directory and then moving it to
    * the target location. This function attempts to clean up its temporary files when possible, but they may stick
    * around (for example, if the JVM crashes partway through executing the function). In any case, the target file
    * should be unharmed.
@@ -186,12 +195,7 @@ public class FileUtils
    *
    * This method is not just thread-safe, but is also safe to use from multiple processes on the same machine.
    */
-  public static <T> T writeAtomically(final File file, OutputStreamConsumer<T> f) throws IOException
-  {
-    return writeAtomically(file, file.getParentFile(), f);
-  }
-
-  private static <T> T writeAtomically(final File file, final File tmpDir, OutputStreamConsumer<T> f) throws IOException
+  public static <T> T writeAtomically(final File file, final File tmpDir, OutputStreamConsumer<T> f) throws IOException
   {
     final File tmpFile = new File(tmpDir, StringUtils.format(".%s.%s", file.getName(), UUID.randomUUID()));
 
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java
index 613c69b..ac5c15c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java
@@ -39,6 +39,7 @@ import org.apache.druid.indexing.common.config.TaskConfig;
 import org.apache.druid.indexing.common.task.Task;
 import org.apache.druid.indexing.overlord.TaskRunner;
 import org.apache.druid.indexing.overlord.TaskRunnerListener;
+import org.apache.druid.java.util.common.FileUtils;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.concurrent.Execs;
@@ -129,6 +130,7 @@ public abstract class WorkerTaskManager
     synchronized (lock) {
       try {
         log.info("Starting...");
+        cleanupAndMakeTmpTaskDir();
         registerLocationListener();
         restoreRestorableTasks();
         initAssignedTasks();
@@ -264,7 +266,12 @@ public abstract class WorkerTaskManager
       }
 
       try {
-        jsonMapper.writeValue(new File(getAssignedTaskDir(), task.getId()), task);
+        FileUtils.writeAtomically(new File(getAssignedTaskDir(), task.getId()), getTmpTaskDir(),
+            os -> {
+            jsonMapper.writeValue(os, task);
+            return null;
+          }
+        );
         assignedTasks.put(task.getId(), task);
       }
       catch (IOException ex) {
@@ -286,6 +293,28 @@ public abstract class WorkerTaskManager
     submitNoticeToExec(new RunNotice(task));
   }
 
+  private File getTmpTaskDir()
+  {
+    return new File(taskConfig.getBaseTaskDir(), "workerTaskManagerTmp");
+  }
+
+  private void cleanupAndMakeTmpTaskDir()
+  {
+    File tmpDir = getTmpTaskDir();
+    tmpDir.mkdirs();
+    if (!tmpDir.isDirectory()) {
+      throw new ISE("Tmp Tasks Dir [%s] does not exist/not-a-directory.", tmpDir);
+    }
+
+    // Delete any tmp files left out from before due to jvm crash.
+    try {
+      org.apache.commons.io.FileUtils.cleanDirectory(tmpDir);
+    }
+    catch (IOException ex) {
+      log.warn("Failed to cleanup tmp dir [%s].", tmpDir.getAbsolutePath());
+    }
+  }
+
   public File getAssignedTaskDir()
   {
     return new File(taskConfig.getBaseTaskDir(), "assignedTasks");
@@ -311,11 +340,11 @@ public abstract class WorkerTaskManager
           assignedTasks.put(taskId, task);
           log.info("Found assigned task[%s].", taskId);
         } else {
-          throw new ISE("Corrupted assigned task on disk[%s].", taskFile.getAbsoluteFile());
+          throw new ISE("WTF! Corrupted assigned task on disk[%s].", taskFile.getAbsoluteFile());
         }
       }
       catch (IOException ex) {
-        throw new ISE(ex, "Failed to read assigned task from disk at [%s]. Ignored.", taskFile.getAbsoluteFile());
+        log.error(ex, "Failed to read assigned task from disk at [%s]. Ignored.", taskFile.getAbsoluteFile());
       }
     }
 
@@ -395,7 +424,12 @@ public abstract class WorkerTaskManager
       completedTasks.put(taskId, taskAnnouncement);
 
       try {
-        jsonMapper.writeValue(new File(getCompletedTaskDir(), taskId), taskAnnouncement);
+        FileUtils.writeAtomically(new File(getCompletedTaskDir(), taskId), getTmpTaskDir(),
+            os -> {
+            jsonMapper.writeValue(os, taskAnnouncement);
+            return null;
+          }
+        );
       }
       catch (IOException ex) {
         log.error(ex, "Error while trying to persist completed task[%s] announcement.", taskId);
@@ -423,11 +457,11 @@ public abstract class WorkerTaskManager
           completedTasks.put(taskId, taskAnnouncement);
           log.info("Found completed task[%s] with status[%s].", taskId, taskAnnouncement.getStatus());
         } else {
-          throw new ISE("Corrupted completed task on disk[%s].", taskFile.getAbsoluteFile());
+          throw new ISE("WTF! Corrupted completed task on disk[%s].", taskFile.getAbsoluteFile());
         }
       }
       catch (IOException ex) {
-        throw new ISE(ex, "Failed to read completed task from disk at [%s]. Ignored.", taskFile.getAbsoluteFile());
+        log.error(ex, "Failed to read completed task from disk at [%s]. Ignored.", taskFile.getAbsoluteFile());
       }
     }
   }


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


[incubator-druid] 04/14: discard filter when processing subtotalsSpec (#7827)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit 70ef2daf29b9338ad4a77af7fc131e75c03c151e
Author: Himanshu <g....@gmail.com>
AuthorDate: Tue Jun 4 10:59:22 2019 -0700

    discard filter when processing subtotalsSpec (#7827)
---
 .../query/groupby/strategy/GroupByStrategyV2.java  |  2 +-
 .../query/groupby/GroupByQueryRunnerTest.java      | 90 ++++++++++++++++++++++
 2 files changed, 91 insertions(+), 1 deletion(-)

diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
index 434a828..1f30bb5 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java
@@ -389,7 +389,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
     final List<Closeable> closeOnExit = new ArrayList<>();
 
     try {
-      GroupByQuery queryWithoutSubtotalsSpec = query.withSubtotalsSpec(null);
+      GroupByQuery queryWithoutSubtotalsSpec = query.withSubtotalsSpec(null).withDimFilter(null);
       List<List<String>> subtotals = query.getSubtotalsSpec();
 
       Supplier<Grouper> grouperSupplier = Suppliers.memoize(
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
index cc25be4..b2815cf 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
@@ -6785,6 +6785,96 @@ public class GroupByQueryRunnerTest
     TestHelper.assertExpectedObjects(expectedResults, results, "subtotal");
   }
 
+  // https://github.com/apache/incubator-druid/issues/7820
+  @Test
+  public void testGroupByWithSubtotalsSpecWithRenamedDimensionAndFilter()
+  {
+    if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) {
+      return;
+    }
+
+    GroupByQuery query = GroupByQuery
+        .builder()
+        .setDataSource(QueryRunnerTestHelper.dataSource)
+        .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
+        .setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ValueType.STRING, TestExprMacroTable.INSTANCE))
+        .setDimensions(Lists.newArrayList(
+            new DefaultDimensionSpec("quality", "quality"),
+            new DefaultDimensionSpec("market", "market"),
+            new DefaultDimensionSpec("alias", "alias_renamed")
+        ))
+        .setAggregatorSpecs(
+            Arrays.asList(
+                QueryRunnerTestHelper.rowsCount,
+                new LongSumAggregatorFactory("idx", "index"),
+                new FloatSumAggregatorFactory("idxFloat", "indexFloat"),
+                new DoubleSumAggregatorFactory("idxDouble", "index")
+            )
+        )
+        .setDimFilter(new SelectorDimFilter("alias", "automotive", null))
+        .setGranularity(QueryRunnerTestHelper.dayGran)
+        .setSubtotalsSpec(ImmutableList.of(
+            ImmutableList.of("alias_renamed"),
+            ImmutableList.of()
+        ))
+        .build();
+
+    List<Row> expectedResults = Arrays.asList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01",
+            "alias_renamed",
+            "automotive",
+            "rows",
+            1L,
+            "idx",
+            135L,
+            "idxFloat",
+            135.88510131835938f,
+            "idxDouble",
+            135.88510131835938d
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02",
+            "alias_renamed",
+            "automotive",
+            "rows",
+            1L,
+            "idx",
+            147L,
+            "idxFloat",
+            147.42593f,
+            "idxDouble",
+            147.42593d
+        ),
+
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-01T00:00:00.000Z",
+            "rows",
+            1L,
+            "idx",
+            135L,
+            "idxFloat",
+            135.88510131835938f,
+            "idxDouble",
+            135.88510131835938d
+        ),
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            "2011-04-02T00:00:00.000Z",
+            "rows",
+            1L,
+            "idx",
+            147L,
+            "idxFloat",
+            147.42593f,
+            "idxDouble",
+            147.42593d
+        )
+    );
+
+    Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
+    TestHelper.assertExpectedObjects(expectedResults, results, "subtotal");
+  }
+
   @Test
   public void testGroupByWithSubtotalsSpecWithLongDimensionColumn()
   {


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


[incubator-druid] 08/14: SupervisorManager: Add authorization checks to bulk endpoints. (#8044)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit bff1c6ca369bedfebcbea33a9775b2a33ac02ba5
Author: Gian Merlino <gi...@gmail.com>
AuthorDate: Tue Jul 9 13:16:54 2019 -0700

    SupervisorManager: Add authorization checks to bulk endpoints. (#8044)
    
    The endpoints added in #6272 were missing authorization checks. This patch removes the bulk
    methods from SupervisorManager, and instead has SupervisorResource run the full list through
    filterAuthorizedSupervisorIds before calling resume/suspend/terminate one by one.
---
 .../overlord/supervisor/SupervisorManager.java     |  20 --
 .../overlord/supervisor/SupervisorResource.java    |  34 ++-
 .../supervisor/SupervisorResourceTest.java         | 237 ++++++++++++++-------
 .../server/security/AuthenticationResult.java      |   2 +
 4 files changed, 189 insertions(+), 104 deletions(-)

diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java
index 3168b89..30183ae 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java
@@ -101,26 +101,6 @@ public class SupervisorManager
     }
   }
 
-  public void stopAndRemoveAllSupervisors()
-  {
-    Preconditions.checkState(started, "SupervisorManager not started");
-
-    synchronized (lock) {
-      Preconditions.checkState(started, "SupervisorManager not started");
-      supervisors.keySet().forEach(id -> possiblyStopAndRemoveSupervisorInternal(id, true));
-    }
-  }
-
-  public void suspendOrResumeAllSupervisors(boolean suspend)
-  {
-    Preconditions.checkState(started, "SupervisorManager not started");
-
-    synchronized (lock) {
-      Preconditions.checkState(started, "SupervisorManager not started");
-      supervisors.keySet().forEach(id -> possiblySuspendOrResumeSupervisorInternal(id, suspend));
-    }
-  }
-
   @LifecycleStart
   public void start()
   {
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java
index 97e0580..122cc3b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java
@@ -263,27 +263,36 @@ public class SupervisorResource
   @POST
   @Path("/suspendAll")
   @Produces(MediaType.APPLICATION_JSON)
-  public Response suspendAll()
+  public Response suspendAll(@Context final HttpServletRequest req)
   {
-    return suspendOrResumeAll(true);
+    return suspendOrResumeAll(req, true);
   }
 
   @POST
   @Path("/resumeAll")
   @Produces(MediaType.APPLICATION_JSON)
-  public Response resumeAll()
+  public Response resumeAll(@Context final HttpServletRequest req)
   {
-    return suspendOrResumeAll(false);
+    return suspendOrResumeAll(req, false);
   }
 
   @POST
   @Path("/terminateAll")
   @Produces(MediaType.APPLICATION_JSON)
-  public Response terminateAll()
+  public Response terminateAll(@Context final HttpServletRequest req)
   {
     return asLeaderWithSupervisorManager(
         manager -> {
-          manager.stopAndRemoveAllSupervisors();
+          Set<String> authorizedSupervisorIds = filterAuthorizedSupervisorIds(
+              req,
+              manager,
+              manager.getSupervisorIds()
+          );
+
+          for (final String supervisorId : authorizedSupervisorIds) {
+            manager.stopAndRemoveSupervisor(supervisorId);
+          }
+
           return Response.ok(ImmutableMap.of("status", "success")).build();
         }
     );
@@ -429,11 +438,20 @@ public class SupervisorResource
     );
   }
 
-  private Response suspendOrResumeAll(boolean suspend)
+  private Response suspendOrResumeAll(final HttpServletRequest req, final boolean suspend)
   {
     return asLeaderWithSupervisorManager(
         manager -> {
-          manager.suspendOrResumeAllSupervisors(suspend);
+          Set<String> authorizedSupervisorIds = filterAuthorizedSupervisorIds(
+              req,
+              manager,
+              manager.getSupervisorIds()
+          );
+
+          for (final String supervisorId : authorizedSupervisorIds) {
+            manager.suspendOrResumeSupervisor(supervisorId, suspend);
+          }
+
           return Response.ok(ImmutableMap.of("status", "success")).build();
         }
     );
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java
index 3b31dec..d73b163 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java
@@ -53,6 +53,20 @@ import java.util.Set;
 @RunWith(EasyMockRunner.class)
 public class SupervisorResourceTest extends EasyMockSupport
 {
+  private static final TestSupervisorSpec SPEC1 = new TestSupervisorSpec(
+      "id1",
+      null,
+      Collections.singletonList("datasource1")
+  );
+
+  private static final TestSupervisorSpec SPEC2 = new TestSupervisorSpec(
+      "id2",
+      null,
+      Collections.singletonList("datasource2")
+  );
+
+  private static final Set<String> SUPERVISOR_IDS = ImmutableSet.of(SPEC1.getId(), SPEC2.getId());
+
   @Mock
   private TaskMaster taskMaster;
 
@@ -69,7 +83,8 @@ public class SupervisorResourceTest extends EasyMockSupport
   {
     supervisorResource = new SupervisorResource(
         taskMaster,
-        new AuthorizerMapper(null) {
+        new AuthorizerMapper(null)
+        {
           @Override
           public Authorizer getAuthorizer(String name)
           {
@@ -92,7 +107,8 @@ public class SupervisorResourceTest extends EasyMockSupport
   @Test
   public void testSpecPost()
   {
-    SupervisorSpec spec = new TestSupervisorSpec("my-id", null, null) {
+    SupervisorSpec spec = new TestSupervisorSpec("my-id", null, null)
+    {
 
       @Override
       public List<String> getDataSources()
@@ -131,28 +147,10 @@ public class SupervisorResourceTest extends EasyMockSupport
   @Test
   public void testSpecGetAll()
   {
-    Set<String> supervisorIds = ImmutableSet.of("id1", "id2");
-    SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, null) {
-
-      @Override
-      public List<String> getDataSources()
-      {
-        return Collections.singletonList("datasource1");
-      }
-    };
-    SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, null) {
-
-      @Override
-      public List<String> getDataSources()
-      {
-        return Collections.singletonList("datasource2");
-      }
-    };
-
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager));
-    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(supervisorIds).atLeastOnce();
-    EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1));
-    EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2));
+    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(SUPERVISOR_IDS).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC1.getId())).andReturn(Optional.of(SPEC1));
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC2.getId())).andReturn(Optional.of(SPEC2));
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
@@ -166,7 +164,7 @@ public class SupervisorResourceTest extends EasyMockSupport
     verifyAll();
 
     Assert.assertEquals(200, response.getStatus());
-    Assert.assertEquals(supervisorIds, response.getEntity());
+    Assert.assertEquals(SUPERVISOR_IDS, response.getEntity());
     resetAll();
 
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent());
@@ -181,29 +179,10 @@ public class SupervisorResourceTest extends EasyMockSupport
   @Test
   public void testSpecGetAllFull()
   {
-    Set<String> supervisorIds = ImmutableSet.of("id1", "id2");
-
-    SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, null) {
-
-      @Override
-      public List<String> getDataSources()
-      {
-        return Collections.singletonList("datasource1");
-      }
-    };
-    SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, null) {
-
-      @Override
-      public List<String> getDataSources()
-      {
-        return Collections.singletonList("datasource2");
-      }
-    };
-
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager));
-    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(supervisorIds).atLeastOnce();
-    EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)).times(2);
-    EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)).times(2);
+    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(SUPERVISOR_IDS).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(SPEC1)).times(2);
+    EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(SPEC2)).times(2);
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
@@ -221,8 +200,8 @@ public class SupervisorResourceTest extends EasyMockSupport
     Assert.assertTrue(
         specs.stream()
              .allMatch(spec ->
-                           ("id1".equals(spec.get("id")) && spec1.equals(spec.get("spec"))) ||
-                           ("id2".equals(spec.get("id")) && spec2.equals(spec.get("spec")))
+                           ("id1".equals(spec.get("id")) && SPEC1.equals(spec.get("spec"))) ||
+                           ("id2".equals(spec.get("id")) && SPEC2.equals(spec.get("spec")))
              )
     );
   }
@@ -292,7 +271,8 @@ public class SupervisorResourceTest extends EasyMockSupport
   @Test
   public void testSpecSuspend()
   {
-    TestSupervisorSpec suspended = new TestSupervisorSpec("my-id", null, null, true) {
+    TestSupervisorSpec suspended = new TestSupervisorSpec("my-id", null, null, true)
+    {
       @Override
       public List<String> getDataSources()
       {
@@ -329,7 +309,8 @@ public class SupervisorResourceTest extends EasyMockSupport
   @Test
   public void testSpecResume()
   {
-    TestSupervisorSpec running = new TestSupervisorSpec("my-id", null, null, false) {
+    TestSupervisorSpec running = new TestSupervisorSpec("my-id", null, null, false)
+    {
       @Override
       public List<String> getDataSources()
       {
@@ -396,11 +377,46 @@ public class SupervisorResourceTest extends EasyMockSupport
   public void testSuspendAll()
   {
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager));
-    supervisorManager.suspendOrResumeAllSupervisors(true);
-    EasyMock.expectLastCall();
+    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(SUPERVISOR_IDS).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC1.getId())).andReturn(Optional.of(SPEC1));
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC2.getId())).andReturn(Optional.of(SPEC2));
+    EasyMock.expect(supervisorManager.suspendOrResumeSupervisor(SPEC1.getId(), true)).andReturn(true);
+    EasyMock.expect(supervisorManager.suspendOrResumeSupervisor(SPEC2.getId(), true)).andReturn(true);
+
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
+        new AuthenticationResult("druid", "druid", null, null)
+    ).atLeastOnce();
+    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    EasyMock.expectLastCall().anyTimes();
     replayAll();
 
-    Response response = supervisorResource.suspendAll();
+    Response response = supervisorResource.suspendAll(request);
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(ImmutableMap.of("status", "success"), response.getEntity());
+    verifyAll();
+  }
+
+  @Test
+  public void testSuspendAllWithPartialAuthorization()
+  {
+    EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager));
+    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(SUPERVISOR_IDS).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC1.getId())).andReturn(Optional.of(SPEC1));
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC2.getId())).andReturn(Optional.of(SPEC2));
+    EasyMock.expect(supervisorManager.suspendOrResumeSupervisor(SPEC1.getId(), true)).andReturn(true);
+
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
+        new AuthenticationResult("notDruid", "druid", null, null)
+    ).atLeastOnce();
+    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    EasyMock.expectLastCall().anyTimes();
+    replayAll();
+
+    Response response = supervisorResource.suspendAll(request);
     Assert.assertEquals(200, response.getStatus());
     Assert.assertEquals(ImmutableMap.of("status", "success"), response.getEntity());
     verifyAll();
@@ -410,11 +426,46 @@ public class SupervisorResourceTest extends EasyMockSupport
   public void testResumeAll()
   {
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager));
-    supervisorManager.suspendOrResumeAllSupervisors(false);
-    EasyMock.expectLastCall();
+    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(SUPERVISOR_IDS).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC1.getId())).andReturn(Optional.of(SPEC1));
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC2.getId())).andReturn(Optional.of(SPEC2));
+    EasyMock.expect(supervisorManager.suspendOrResumeSupervisor(SPEC1.getId(), false)).andReturn(true);
+    EasyMock.expect(supervisorManager.suspendOrResumeSupervisor(SPEC2.getId(), false)).andReturn(true);
+
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
+        new AuthenticationResult("druid", "druid", null, null)
+    ).atLeastOnce();
+    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    EasyMock.expectLastCall().anyTimes();
     replayAll();
 
-    Response response = supervisorResource.resumeAll();
+    Response response = supervisorResource.resumeAll(request);
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(ImmutableMap.of("status", "success"), response.getEntity());
+    verifyAll();
+  }
+
+  @Test
+  public void testResumeAllWithPartialAuthorization()
+  {
+    EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager));
+    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(SUPERVISOR_IDS).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC1.getId())).andReturn(Optional.of(SPEC1));
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC2.getId())).andReturn(Optional.of(SPEC2));
+    EasyMock.expect(supervisorManager.suspendOrResumeSupervisor(SPEC1.getId(), false)).andReturn(true);
+
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
+        new AuthenticationResult("notDruid", "druid", null, null)
+    ).atLeastOnce();
+    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    EasyMock.expectLastCall().anyTimes();
+    replayAll();
+
+    Response response = supervisorResource.resumeAll(request);
     Assert.assertEquals(200, response.getStatus());
     Assert.assertEquals(ImmutableMap.of("status", "success"), response.getEntity());
     verifyAll();
@@ -424,11 +475,46 @@ public class SupervisorResourceTest extends EasyMockSupport
   public void testTerminateAll()
   {
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager));
-    supervisorManager.stopAndRemoveAllSupervisors();
-    EasyMock.expectLastCall();
+    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(SUPERVISOR_IDS).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC1.getId())).andReturn(Optional.of(SPEC1));
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC2.getId())).andReturn(Optional.of(SPEC2));
+    EasyMock.expect(supervisorManager.stopAndRemoveSupervisor(SPEC1.getId())).andReturn(true);
+    EasyMock.expect(supervisorManager.stopAndRemoveSupervisor(SPEC2.getId())).andReturn(true);
+
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
+        new AuthenticationResult("druid", "druid", null, null)
+    ).atLeastOnce();
+    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    EasyMock.expectLastCall().anyTimes();
     replayAll();
 
-    Response response = supervisorResource.terminateAll();
+    Response response = supervisorResource.terminateAll(request);
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(ImmutableMap.of("status", "success"), response.getEntity());
+    verifyAll();
+  }
+
+  @Test
+  public void testTerminateAllWithPartialAuthorization()
+  {
+    EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager));
+    EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(SUPERVISOR_IDS).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC1.getId())).andReturn(Optional.of(SPEC1));
+    EasyMock.expect(supervisorManager.getSupervisorSpec(SPEC2.getId())).andReturn(Optional.of(SPEC2));
+    EasyMock.expect(supervisorManager.stopAndRemoveSupervisor(SPEC1.getId())).andReturn(true);
+
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
+    EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
+        new AuthenticationResult("notDruid", "druid", null, null)
+    ).atLeastOnce();
+    request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true);
+    EasyMock.expectLastCall().anyTimes();
+    replayAll();
+
+    Response response = supervisorResource.terminateAll(request);
     Assert.assertEquals(200, response.getStatus());
     Assert.assertEquals(ImmutableMap.of("status", "success"), response.getEntity());
     verifyAll();
@@ -498,10 +584,8 @@ public class SupervisorResourceTest extends EasyMockSupport
 
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2);
     EasyMock.expect(supervisorManager.getSupervisorHistory()).andReturn(history);
-    SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, Collections.singletonList("datasource1"));
-    SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, Collections.singletonList("datasource2"));
-    EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)).atLeastOnce();
-    EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(SPEC1)).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(SPEC2)).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
@@ -528,7 +612,7 @@ public class SupervisorResourceTest extends EasyMockSupport
   }
 
   @Test
-  public void testSpecGetAllHistoryWithAuthFailureFiltering()
+  public void testSpecGetAllHistoryWithPartialAuthorization()
   {
     List<VersionedSupervisorSpec> versions1 = ImmutableList.of(
         new VersionedSupervisorSpec(
@@ -611,10 +695,8 @@ public class SupervisorResourceTest extends EasyMockSupport
 
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2);
     EasyMock.expect(supervisorManager.getSupervisorHistory()).andReturn(history);
-    SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, Collections.singletonList("datasource1"));
-    SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, Collections.singletonList("datasource2"));
-    EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)).atLeastOnce();
-    EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(SPEC1)).atLeastOnce();
+    EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(SPEC2)).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce();
     EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(
@@ -872,8 +954,14 @@ public class SupervisorResourceTest extends EasyMockSupport
     Capture<String> id1 = Capture.newInstance();
     Capture<String> id2 = Capture.newInstance();
     EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2);
-    EasyMock.expect(supervisorManager.resetSupervisor(EasyMock.capture(id1), EasyMock.anyObject(DataSourceMetadata.class))).andReturn(true);
-    EasyMock.expect(supervisorManager.resetSupervisor(EasyMock.capture(id2), EasyMock.anyObject(DataSourceMetadata.class))).andReturn(false);
+    EasyMock.expect(supervisorManager.resetSupervisor(
+        EasyMock.capture(id1),
+        EasyMock.anyObject(DataSourceMetadata.class)
+    )).andReturn(true);
+    EasyMock.expect(supervisorManager.resetSupervisor(
+        EasyMock.capture(id2),
+        EasyMock.anyObject(DataSourceMetadata.class)
+    )).andReturn(false);
     replayAll();
 
     Response response = supervisorResource.reset("my-id");
@@ -908,12 +996,9 @@ public class SupervisorResourceTest extends EasyMockSupport
     NoopSupervisorSpec deserializedSpec = mapper.readValue(oldSpec, NoopSupervisorSpec.class);
     Assert.assertEquals(expectedSpec, deserializedSpec);
 
-    NoopSupervisorSpec spec1 = new NoopSupervisorSpec("abcd", Collections.singletonList("defg"));
-    NoopSupervisorSpec spec2 = mapper.readValue(
-        mapper.writeValueAsBytes(spec1),
-        NoopSupervisorSpec.class
-    );
-    Assert.assertEquals(spec1, spec2);
+    NoopSupervisorSpec spec = new NoopSupervisorSpec("abcd", Collections.singletonList("defg"));
+    NoopSupervisorSpec specRoundTrip = mapper.readValue(mapper.writeValueAsBytes(spec), NoopSupervisorSpec.class);
+    Assert.assertEquals(spec, specRoundTrip);
   }
 
   private static class TestSupervisorSpec implements SupervisorSpec
diff --git a/server/src/main/java/org/apache/druid/server/security/AuthenticationResult.java b/server/src/main/java/org/apache/druid/server/security/AuthenticationResult.java
index 854aa5e..4e7a3da 100644
--- a/server/src/main/java/org/apache/druid/server/security/AuthenticationResult.java
+++ b/server/src/main/java/org/apache/druid/server/security/AuthenticationResult.java
@@ -76,11 +76,13 @@ public class AuthenticationResult
     return authorizerName;
   }
 
+  @Nullable
   public Map<String, Object> getContext()
   {
     return context;
   }
 
+  @Nullable
   public String getAuthenticatedBy()
   {
     return authenticatedBy;


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


[incubator-druid] 06/14: Fix ExpressionVirtualColumn capabilities; fix groupBy's improper uses of StorageAdapter#getColumnCapabilities. (#8013)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit 21d3600e61ccbb052e1618d1ac4f6be53c6687dd
Author: Gian Merlino <gi...@gmail.com>
AuthorDate: Fri Jul 5 13:17:05 2019 -0700

    Fix ExpressionVirtualColumn capabilities; fix groupBy's improper uses of StorageAdapter#getColumnCapabilities. (#8013)
    
    * GroupBy: Fix improper uses of StorageAdapter#getColumnCapabilities.
    
    1) A usage in "isArrayAggregateApplicable" that would potentially incorrectly use
       array-based aggregation on a virtual column that shadows a real column.
    2) A usage in "process" that would potentially use the more expensive multi-value
       aggregation path on a singly-valued virtual column. (No correctness issue, but
       a performance issue.)
    
    * Add addl javadoc.
    
    * ExpressionVirtualColumn: Set multi-value flag.
---
 .../epinephelinae/GroupByQueryEngineV2.java        | 55 +++++++++++++++-------
 .../org/apache/druid/segment/StorageAdapter.java   |  5 ++
 .../segment/virtual/ExpressionVirtualColumn.java   |  5 +-
 3 files changed, 48 insertions(+), 17 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java
index e3c609e..7594b90 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java
@@ -48,11 +48,13 @@ import org.apache.druid.query.groupby.epinephelinae.column.LongGroupByColumnSele
 import org.apache.druid.query.groupby.epinephelinae.column.NullableValueGroupByColumnSelectorStrategy;
 import org.apache.druid.query.groupby.epinephelinae.column.StringGroupByColumnSelectorStrategy;
 import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
+import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.DimensionHandlerUtils;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.StorageAdapter;
+import org.apache.druid.segment.VirtualColumns;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.segment.data.IndexedInts;
@@ -115,14 +117,6 @@ public class GroupByQueryEngineV2
         null
     );
 
-    final boolean allSingleValueDims = query
-        .getDimensions()
-        .stream()
-        .allMatch(dimension -> {
-          final ColumnCapabilities columnCapabilities = storageAdapter.getColumnCapabilities(dimension.getDimension());
-          return columnCapabilities != null && !columnCapabilities.hasMultipleValues();
-        });
-
     final ResourceHolder<ByteBuffer> bufferHolder = intermediateResultsBufferPool.take();
 
     final String fudgeTimestampString = NullHandling.emptyToNullIfNeeded(
@@ -140,18 +134,38 @@ public class GroupByQueryEngineV2
               @Override
               public GroupByEngineIterator make()
               {
+                final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+                final boolean allSingleValueDims = query
+                    .getDimensions()
+                    .stream()
+                    .allMatch(dimension -> {
+                      final ColumnCapabilities columnCapabilities = columnSelectorFactory.getColumnCapabilities(
+                          dimension.getDimension()
+                      );
+                      return columnCapabilities != null && !columnCapabilities.hasMultipleValues();
+                    });
+
                 ColumnSelectorPlus<GroupByColumnSelectorStrategy>[] selectorPlus = DimensionHandlerUtils
                     .createColumnSelectorPluses(
                         STRATEGY_FACTORY,
                         query.getDimensions(),
-                        cursor.getColumnSelectorFactory()
+                        columnSelectorFactory
                     );
                 GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(selectorPlus);
 
                 final ByteBuffer buffer = bufferHolder.get();
 
-                // Check array-based aggregation is applicable
-                if (isArrayAggregateApplicable(querySpecificConfig, query, dims, storageAdapter, buffer)) {
+                // Check if array-based aggregation is applicable
+                final boolean useArrayAggregation = isArrayAggregateApplicable(
+                    querySpecificConfig,
+                    query,
+                    dims,
+                    storageAdapter,
+                    query.getVirtualColumns(),
+                    buffer
+                );
+
+                if (useArrayAggregation) {
                   return new ArrayAggregateIterator(
                       query,
                       querySpecificConfig,
@@ -191,6 +205,7 @@ public class GroupByQueryEngineV2
       GroupByQuery query,
       GroupByColumnSelectorPlus[] dims,
       StorageAdapter storageAdapter,
+      VirtualColumns virtualColumns,
       ByteBuffer buffer
   )
   {
@@ -206,11 +221,19 @@ public class GroupByQueryEngineV2
       columnCapabilities = null;
       cardinality = 1;
     } else if (dims.length == 1) {
+      // Only real columns can use array-based aggregation, since virtual columns cannot currently report their
+      // cardinality. We need to check if a virtual column exists with the same name, since virtual columns can shadow
+      // real columns, and we might miss that since we're going directly to the StorageAdapter (which only knows about
+      // real columns).
+      if (virtualColumns.exists(dims[0].getName())) {
+        return false;
+      }
+
       columnCapabilities = storageAdapter.getColumnCapabilities(dims[0].getName());
       cardinality = storageAdapter.getDimensionCardinality(dims[0].getName());
     } else {
-      columnCapabilities = null;
-      cardinality = -1; // ArrayAggregateIterator is not available
+      // Cannot use array-based aggregation with more than one dimension.
+      return false;
     }
 
     // Choose array-based aggregation if the grouping key is a single string dimension of a
@@ -225,11 +248,11 @@ public class GroupByQueryEngineV2
           aggregatorFactories
       );
 
-      // Check that all keys and aggregated values can be contained the buffer
+      // Check that all keys and aggregated values can be contained in the buffer
       return requiredBufferCapacity <= buffer.capacity();
+    } else {
+      return false;
     }
-
-    return false;
   }
 
   private static class GroupByStrategyFactory implements ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
index 3319d09..1dfc00e 100644
--- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java
@@ -56,6 +56,11 @@ public interface StorageAdapter extends CursorFactory
    * the column does exist but the capabilities are unknown. The latter is possible with dynamically discovered
    * columns.
    *
+   * Note that StorageAdapters are representations of "real" segments, so they are not aware of any virtual columns
+   * that may be involved in a query. In general, query engines should instead use the method
+   * {@link ColumnSelectorFactory#getColumnCapabilities(String)}, which returns capabilities for virtual columns as
+   * well.
+   *
    * @param column column name
    *
    * @return capabilities, or null
diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
index 45a25c4..f8988d6 100644
--- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
+++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
@@ -103,7 +103,10 @@ public class ExpressionVirtualColumn implements VirtualColumn
   @Override
   public ColumnCapabilities capabilities(String columnName)
   {
-    return new ColumnCapabilitiesImpl().setType(outputType);
+    // Note: Ideally we would only "setHasMultipleValues(true)" if the expression in question could potentially return
+    // multiple values. However, we don't currently have a good way of determining this, so to be safe we always
+    // set the flag.
+    return new ColumnCapabilitiesImpl().setType(outputType).setHasMultipleValues(true);
   }
 
   @Override


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


[incubator-druid] 11/14: Add missing reference to Materialized-View extension. (#8003)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit c8044264b9da239bc54b258c9c1472c49cf768ad
Author: Eyal Yurman <ey...@oath.com>
AuthorDate: Sat Jul 6 13:50:41 2019 -0700

    Add missing reference to Materialized-View extension. (#8003)
    
    * Reference Materialized View extension from extensions page.
    
    * Add comma
---
 docs/content/development/extensions.md | 1 +
 1 file changed, 1 insertion(+)

diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md
index 6e8bb3a..15b087c 100644
--- a/docs/content/development/extensions.md
+++ b/docs/content/development/extensions.md
@@ -95,6 +95,7 @@ All of these community extensions can be downloaded using [pull-deps](../operati
 |kafka-emitter|Kafka metrics emitter|[link](../development/extensions-contrib/kafka-emitter.html)|
 |druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)|
 |druid-opentsdb-emitter|OpenTSDB metrics emitter |[link](../development/extensions-contrib/opentsdb-emitter.html)|
+|materialized-view-selection, materialized-view-maintenance|Materialized View|[link](../development/extensions-contrib/materialized-view.html)|
 |druid-moving-average-query|Support for [Moving Average](https://en.wikipedia.org/wiki/Moving_average) and other Aggregate [Window Functions](https://en.wikibooks.org/wiki/Structured_Query_Language/Window_functions) in Druid queries.|[link](../development/extensions-contrib/moving-average-query.html)|
 
 ## Promoting Community Extension to Core Extension


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


[incubator-druid] 14/14: fix backport

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit 4404e07cde5a1f595f38b9d5edcada3b8c7aaf8b
Author: Clint Wylie <cw...@apache.org>
AuthorDate: Tue Jul 23 16:47:38 2019 -0700

    fix backport
---
 .../apache/druid/sql/calcite/CalciteQueryTest.java | 45 +---------------------
 1 file changed, 1 insertion(+), 44 deletions(-)

diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index f4bc6ae..c8335ff 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -68,6 +68,7 @@ import org.apache.druid.query.topn.TopNQueryBuilder;
 import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.sql.calcite.expression.DruidExpression;
 import org.apache.druid.sql.calcite.filtration.Filtration;
+import org.apache.druid.sql.calcite.planner.Calcites;
 import org.apache.druid.sql.calcite.rel.CannotBuildQueryException;
 import org.apache.druid.sql.calcite.util.CalciteTests;
 import org.hamcrest.CoreMatchers;
@@ -7686,50 +7687,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
   }
 
   @Test
-  public void testTimestampDiff() throws Exception
-  {
-    testQuery(
-        "SELECT TIMESTAMPDIFF(DAY, TIMESTAMP '1999-01-01 00:00:00', __time), \n"
-        + "TIMESTAMPDIFF(DAY, __time, DATE '2001-01-01'), \n"
-        + "TIMESTAMPDIFF(HOUR, TIMESTAMP '1999-12-31 01:00:00', __time), \n"
-        + "TIMESTAMPDIFF(MINUTE, TIMESTAMP '1999-12-31 23:58:03', __time), \n"
-        + "TIMESTAMPDIFF(SECOND, TIMESTAMP '1999-12-31 23:59:03', __time), \n"
-        + "TIMESTAMPDIFF(MONTH, TIMESTAMP '1999-11-01 00:00:00', __time), \n"
-        + "TIMESTAMPDIFF(YEAR, TIMESTAMP '1996-11-01 00:00:00', __time), \n"
-        + "TIMESTAMPDIFF(QUARTER, TIMESTAMP '1996-10-01 00:00:00', __time), \n"
-        + "TIMESTAMPDIFF(WEEK, TIMESTAMP '1998-10-01 00:00:00', __time) \n"
-        + "FROM druid.foo\n"
-        + "LIMIT 2",
-        ImmutableList.of(
-            newScanQueryBuilder()
-                .dataSource(CalciteTests.DATASOURCE1)
-                .intervals(querySegmentSpec(Filtration.eternity()))
-                .virtualColumns(
-                    expressionVirtualColumn("v0", "div((\"__time\" - 915148800000),86400000)", ValueType.LONG),
-                    expressionVirtualColumn("v1", "div((978307200000 - \"__time\"),86400000)", ValueType.LONG),
-                    expressionVirtualColumn("v2", "div((\"__time\" - 946602000000),3600000)", ValueType.LONG),
-                    expressionVirtualColumn("v3", "div((\"__time\" - 946684683000),60000)", ValueType.LONG),
-                    expressionVirtualColumn("v4", "div((\"__time\" - 946684743000),1000)", ValueType.LONG),
-                    expressionVirtualColumn("v5", "subtract_months(\"__time\",941414400000,'UTC')", ValueType.LONG),
-                    expressionVirtualColumn("v6", "div(subtract_months(\"__time\",846806400000,'UTC'),12)", ValueType.LONG),
-                    expressionVirtualColumn("v7", "div(subtract_months(\"__time\",844128000000,'UTC'),3)", ValueType.LONG),
-                    expressionVirtualColumn("v8", "div(div((\"__time\" - 907200000000),1000),604800)", ValueType.LONG)
-                )
-                .columns("v0", "v1", "v2", "v3", "v4", "v5", "v6", "v7", "v8")
-                .limit(2)
-                .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
-                .context(QUERY_CONTEXT_DEFAULT)
-                .build()
-
-        ),
-        ImmutableList.of(
-            new Object[]{365, 366, 23, 1, 57, 2, 3, 13, 65},
-            new Object[]{366, 365, 47, 1441, 86457, 2, 3, 13, 65}
-        )
-    );
-  }
-
-  @Test
   public void testTimestampCeil() throws Exception
   {
     testQuery(


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


[incubator-druid] 12/14: Fix documentation formatting (#8079)

Posted by cw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 0.15.1-incubating
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git

commit 36bc4e09d637871e96eae8958d851faa41b0b3b6
Author: Magnus Henoch <ma...@gameanalytics.com>
AuthorDate: Mon Jul 15 17:55:18 2019 +0100

    Fix documentation formatting (#8079)
    
    The Markdown dialect used when publishing the documentation to the web
    site is much more sensitive than Github-flavoured Markdown.  In
    particular, it requires an empty line before code blocks (unless the
    code block starts right after a heading), otherwise the code block
    gets formatted in-line with the previous paragraph.  Likewise for
    bullet-point lists.
---
 .../extensions-contrib/distinctcount.md            |   4 +-
 .../development/extensions-contrib/influx.md       |   2 +
 .../extensions-contrib/materialized-view.md        |   3 +
 .../extensions-contrib/momentsketch-quantiles.md   |   6 +
 .../extensions-contrib/moving-average-query.md     |   9 ++
 .../extensions-core/druid-basic-security.md        | 137 ++++++++++++++++++++-
 .../development/extensions-core/druid-lookups.md   |   1 +
 docs/content/development/extensions-core/orc.md    |   4 +
 docs/content/querying/filters.md                   |   6 +
 9 files changed, 169 insertions(+), 3 deletions(-)

diff --git a/docs/content/development/extensions-contrib/distinctcount.md b/docs/content/development/extensions-contrib/distinctcount.md
index a392360..7cf67b5 100644
--- a/docs/content/development/extensions-contrib/distinctcount.md
+++ b/docs/content/development/extensions-contrib/distinctcount.md
@@ -28,8 +28,8 @@ To use this Apache Druid (incubating) extension, make sure to [include](../../op
 
 Additionally, follow these steps:
 
-(1) First, use a single dimension hash-based partition spec to partition data by a single dimension. For example visitor_id. This to make sure all rows with a particular value for that dimension will go into the same segment, or this might over count.
-(2) Second, use distinctCount to calculate the distinct count, make sure queryGranularity is divided exactly by segmentGranularity or else the result will be wrong.
+1. First, use a single dimension hash-based partition spec to partition data by a single dimension. For example visitor_id. This to make sure all rows with a particular value for that dimension will go into the same segment, or this might over count.
+2. Second, use distinctCount to calculate the distinct count, make sure queryGranularity is divided exactly by segmentGranularity or else the result will be wrong.
 
 There are some limitations, when used with groupBy, the groupBy keys' numbers should not exceed maxIntermediateRows in every segment. If exceeded the result will be wrong. When used with topN, numValuesPerPass should not be too big. If too big the distinctCount will use a lot of memory and might cause the JVM to go our of memory.
 
diff --git a/docs/content/development/extensions-contrib/influx.md b/docs/content/development/extensions-contrib/influx.md
index c5c071b..62e036b 100644
--- a/docs/content/development/extensions-contrib/influx.md
+++ b/docs/content/development/extensions-contrib/influx.md
@@ -35,6 +35,7 @@ A typical line looks like this:
 ```cpu,application=dbhost=prdb123,region=us-east-1 usage_idle=99.24,usage_user=0.55 1520722030000000000```
 
 which contains four parts:
+
   - measurement: A string indicating the name of the measurement represented (e.g. cpu, network, web_requests)
   - tags: zero or more key-value pairs (i.e. dimensions)
   - measurements: one or more key-value pairs; values can be numeric, boolean, or string
@@ -43,6 +44,7 @@ which contains four parts:
 The parser extracts these fields into a map, giving the measurement the key `measurement` and the timestamp the key `_ts`. The tag and measurement keys are copied verbatim, so users should take care to avoid name collisions. It is up to the ingestion spec to decide which fields should be treated as dimensions and which should be treated as metrics (typically tags correspond to dimensions and measurements correspond to metrics).
 
 The parser is configured like so:
+
 ```json
 "parser": {
       "type": "string",
diff --git a/docs/content/development/extensions-contrib/materialized-view.md b/docs/content/development/extensions-contrib/materialized-view.md
index 95bfde9..963a944 100644
--- a/docs/content/development/extensions-contrib/materialized-view.md
+++ b/docs/content/development/extensions-contrib/materialized-view.md
@@ -33,6 +33,7 @@ In materialized-view-maintenance, dataSouces user ingested are called "base-data
 The `derivativeDataSource` supervisor is used to keep the timeline of derived-dataSource consistent with base-dataSource. Each `derivativeDataSource` supervisor  is responsible for one derived-dataSource.
 
 A sample derivativeDataSource supervisor spec is shown below:
+
 ```json
    {
        "type": "derivativeDataSource",
@@ -90,6 +91,7 @@ A sample derivativeDataSource supervisor spec is shown below:
 In materialized-view-selection, we implement a new query type `view`. When we request a view query, Druid will try its best to optimize the query based on query dataSource and intervals.
 
 A sample view query spec is shown below:
+
 ```json
    {
        "queryType": "view",
@@ -124,6 +126,7 @@ A sample view query spec is shown below:
        }
    }
 ```
+
 There are 2 parts in a view query:
 
 |Field|Description|Required|
diff --git a/docs/content/development/extensions-contrib/momentsketch-quantiles.md b/docs/content/development/extensions-contrib/momentsketch-quantiles.md
index 966caa2..3eeadaf 100644
--- a/docs/content/development/extensions-contrib/momentsketch-quantiles.md
+++ b/docs/content/development/extensions-contrib/momentsketch-quantiles.md
@@ -38,6 +38,7 @@ druid.extensions.loadList=["druid-momentsketch"]
 The result of the aggregation is a momentsketch that is the union of all sketches either built from raw data or read from the segments.
 
 The `momentSketch` aggregator operates over raw data while the `momentSketchMerge` aggregator should be used when aggregating pre-computed sketches.
+
 ```json
 {
   "type" : <aggregator_type>,
@@ -59,6 +60,7 @@ The `momentSketch` aggregator operates over raw data while the `momentSketchMerg
 ### Post Aggregators
 
 Users can query for a set of quantiles using the `momentSketchSolveQuantiles` post-aggregator on the sketches created by the `momentSketch` or `momentSketchMerge` aggregators.
+
 ```json
 {
   "type"  : "momentSketchSolveQuantiles",
@@ -69,6 +71,7 @@ Users can query for a set of quantiles using the `momentSketchSolveQuantiles` po
 ```
 
 Users can also query for the min/max of a distribution:
+
 ```json
 {
   "type" : "momentSketchMin" | "momentSketchMax",
@@ -79,6 +82,7 @@ Users can also query for the min/max of a distribution:
 
 ### Example
 As an example of a query with sketches pre-aggregated at ingestion time, one could set up the following aggregator at ingest:
+
 ```json
 {
   "type": "momentSketch", 
@@ -88,7 +92,9 @@ As an example of a query with sketches pre-aggregated at ingestion time, one cou
   "compress": true,
 }
 ```
+
 and make queries using the following aggregator + post-aggregator:
+
 ```json
 {
   "aggregations": [{
diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md
index 5fc7268..7e028cc 100644
--- a/docs/content/development/extensions-contrib/moving-average-query.md
+++ b/docs/content/development/extensions-contrib/moving-average-query.md
@@ -33,6 +33,7 @@ These Aggregate Window Functions consume standard Druid Aggregators and outputs
 Moving Average encapsulates the [groupBy query](../../querying/groupbyquery.html) (Or [timeseries](../../querying/timeseriesquery.html) in case of no dimensions) in order to rely on the maturity of these query types.
 
 It runs the query in two main phases:
+
 1. Runs an inner [groupBy](../../querying/groupbyquery.html) or [timeseries](../../querying/timeseriesquery.html) query to compute Aggregators (i.e. daily count of events).
 2. Passes over aggregated results in Broker, in order to compute Averagers (i.e. moving 7 day average of the daily count).
 
@@ -110,6 +111,7 @@ These are properties which are common to all Averagers:
 #### Standard averagers
 
 These averagers offer four functions:
+
 * Mean (Average)
 * MeanNoNulls (Ignores empty buckets).
 * Max
@@ -121,6 +123,7 @@ In that case, the first records will ignore missing buckets and average won't be
 However, this also means that empty days in a sparse dataset will also be ignored.
 
 Example of usage:
+
 ```json
 { "type" : "doubleMean", "name" : <output_name>, "fieldName": <input_name> }
 ```
@@ -130,6 +133,7 @@ This optional parameter is used to calculate over a single bucket within each cy
 A prime example would be weekly buckets, resulting in a Day of Week calculation. (Other examples: Month of year, Hour of day).
 
 I.e. when using these parameters:
+
 * *granularity*: period=P1D (daily)
 * *buckets*: 28
 * *cycleSize*: 7
@@ -146,6 +150,7 @@ All examples are based on the Wikipedia dataset provided in the Druid [tutorials
 Calculating a 7-buckets moving average for Wikipedia edit deltas.
 
 Query syntax:
+
 ```json
 {
   "queryType": "movingAverage",
@@ -176,6 +181,7 @@ Query syntax:
 ```
 
 Result:
+
 ```json
 [ {
    "version" : "v1",
@@ -217,6 +223,7 @@ Result:
 Calculating a 7-buckets moving average for Wikipedia edit deltas, plus a ratio between the current period and the moving average.
 
 Query syntax:
+
 ```json
 {
   "queryType": "movingAverage",
@@ -264,6 +271,7 @@ Query syntax:
 ```
 
 Result:
+
 ```json
 [ {
   "version" : "v1",
@@ -306,6 +314,7 @@ Result:
 Calculating an average of every first 10-minutes of the last 3 hours:
 
 Query syntax:
+
 ```json
 {
   "queryType": "movingAverage",
diff --git a/docs/content/development/extensions-core/druid-basic-security.md b/docs/content/development/extensions-core/druid-basic-security.md
index 28eff1f..e067fdf 100644
--- a/docs/content/development/extensions-core/druid-basic-security.md
+++ b/docs/content/development/extensions-core/druid-basic-security.md
@@ -172,6 +172,90 @@ Return a list of all user names.
 `GET(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
 Return the name and role information of the user with name {userName}
 
+Example output:
+
+```json
+{
+  "name": "druid2",
+  "roles": [
+    "druidRole"
+  ]
+}
+```
+
+This API supports the following flags:
+
+- `?full`: The response will also include the full information for each role currently assigned to the user.
+
+Example output:
+
+```json
+{
+  "name": "druid2",
+  "roles": [
+    {
+      "name": "druidRole",
+      "permissions": [
+        {
+          "resourceAction": {
+            "resource": {
+              "name": "A",
+              "type": "DATASOURCE"
+            },
+            "action": "READ"
+          },
+          "resourceNamePattern": "A"
+        },
+        {
+          "resourceAction": {
+            "resource": {
+              "name": "C",
+              "type": "CONFIG"
+            },
+            "action": "WRITE"
+          },
+          "resourceNamePattern": "C"
+        }
+      ]
+    }
+  ]
+}
+```
+
+The output format of this API when `?full` is specified is deprecated and in later versions will be switched to the output format used when both `?full` and `?simplifyPermissions` flag is set. 
+
+The `resourceNamePattern` is a compiled version of the resource name regex. It is redundant and complicates the use of this API for clients such as frontends that edit the authorization configuration, as the permission format in this output does not match the format used for adding permissions to a role.
+
+- `?full?simplifyPermissions`: When both `?full` and `?simplifyPermissions` are set, the permissions in the output will contain only a list of `resourceAction` objects, without the extraneous `resourceNamePattern` field.
+
+```json
+{
+  "name": "druid2",
+  "roles": [
+    {
+      "name": "druidRole",
+      "users": null,
+      "permissions": [
+        {
+          "resource": {
+            "name": "A",
+            "type": "DATASOURCE"
+          },
+          "action": "READ"
+        },
+        {
+          "resource": {
+            "name": "C",
+            "type": "CONFIG"
+          },
+          "action": "WRITE"
+        }
+      ]
+    }
+  ]
+}
+```
+
 `POST(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
 Create a new user with name {userName}
 
@@ -184,7 +268,58 @@ Delete the user with name {userName}
 Return a list of all role names.
 
 `GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
-Return name and permissions for the role named {roleName}
+Return name and permissions for the role named {roleName}.
+
+Example output:
+
+```json
+{
+  "name": "druidRole2",
+  "permissions": [
+    {
+      "resourceAction": {
+        "resource": {
+          "name": "E",
+          "type": "DATASOURCE"
+        },
+        "action": "WRITE"
+      },
+      "resourceNamePattern": "E"
+    }
+  ]
+}
+```
+
+The default output format of this API is deprecated and in later versions will be switched to the output format used when the `?simplifyPermissions` flag is set. The `resourceNamePattern` is a compiled version of the resource name regex. It is redundant and complicates the use of this API for clients such as frontends that edit the authorization configuration, as the permission format in this output does not match the format used for adding permissions to a role.
+
+This API supports the following flags:
+
+- `?full`: The output will contain an extra `users` list, containing the users that currently have this role.
+
+```json
+"users":["druid"]
+```
+
+- `?simplifyPermissions`: The permissions in the output will contain only a list of `resourceAction` objects, without the extraneous `resourceNamePattern` field. The `users` field will be null when `?full` is not specified.
+
+Example output:
+
+```json
+{
+  "name": "druidRole2",
+  "users": null,
+  "permissions": [
+    {
+      "resource": {
+        "name": "E",
+        "type": "DATASOURCE"
+      },
+      "action": "WRITE"
+    }
+  ]
+}
+```
+
 
 `POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
 Create a new role with name {roleName}.
diff --git a/docs/content/development/extensions-core/druid-lookups.md b/docs/content/development/extensions-core/druid-lookups.md
index 53476eb..9f5798e 100644
--- a/docs/content/development/extensions-core/druid-lookups.md
+++ b/docs/content/development/extensions-core/druid-lookups.md
@@ -75,6 +75,7 @@ Same for Loading cache, developer can implement a new type of loading cache by i
 
 #####   Example of Polling On-heap Lookup
 This example demonstrates a polling cache that will update its on-heap cache every 10 minutes
+
 ```json
 {
     "type":"pollingLookup",
diff --git a/docs/content/development/extensions-core/orc.md b/docs/content/development/extensions-core/orc.md
index af7a315..791531d 100644
--- a/docs/content/development/extensions-core/orc.md
+++ b/docs/content/development/extensions-core/orc.md
@@ -269,6 +269,7 @@ This extension, first available in version 0.15.0, replaces the previous 'contri
 ingestion task is *incompatible*, and will need modified to work with the newer 'core' extension. 
 
 To migrate to 0.15.0+:
+
 * In `inputSpec` of `ioConfig`, `inputFormat` must be changed from `"org.apache.hadoop.hive.ql.io.orc.OrcNewInputFormat"` to 
 `"org.apache.orc.mapreduce.OrcInputFormat"`
 * The 'contrib' extension supported a `typeString` property, which provided the schema of the
@@ -276,6 +277,7 @@ ORC file, of which was essentially required to have the types correct, but notab
 facilitated column renaming. In the 'core' extension, column renaming can be achieved with 
 [`flattenSpec` expressions](../../ingestion/flatten-json.html). For example, `"typeString":"struct<time:string,name:string>"`
 with the actual schema `struct<_col0:string,_col1:string>`, to preserve Druid schema would need replaced with:
+
 ```json
 "flattenSpec": {
   "fields": [
@@ -293,10 +295,12 @@ with the actual schema `struct<_col0:string,_col1:string>`, to preserve Druid sc
   ...
 }
 ```
+
 * The 'contrib' extension supported a `mapFieldNameFormat` property, which provided a way to specify a dimension to
  flatten `OrcMap` columns with primitive types. This functionality has also been replaced with
  [`flattenSpec` expressions](../../ingestion/flatten-json.html). For example: `"mapFieldNameFormat": "<PARENT>_<CHILD>"`
  for a dimension `nestedData_dim1`, to preserve Druid schema could be replaced with 
+
  ```json
 "flattenSpec": {
   "fields": [
diff --git a/docs/content/querying/filters.md b/docs/content/querying/filters.md
index 2f9b23a..53e0853 100644
--- a/docs/content/querying/filters.md
+++ b/docs/content/querying/filters.md
@@ -282,6 +282,7 @@ greater than, less than, greater than or equal to, less than or equal to, and "b
 Bound filters support the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
 
 The following bound filter expresses the condition `21 <= age <= 31`:
+
 ```json
 {
     "type": "bound",
@@ -293,6 +294,7 @@ The following bound filter expresses the condition `21 <= age <= 31`:
 ```
 
 This filter expresses the condition `foo <= name <= hoo`, using the default lexicographic sorting order.
+
 ```json
 {
     "type": "bound",
@@ -303,6 +305,7 @@ This filter expresses the condition `foo <= name <= hoo`, using the default lexi
 ```
 
 Using strict bounds, this filter expresses the condition `21 < age < 31`
+
 ```json
 {
     "type": "bound",
@@ -316,6 +319,7 @@ Using strict bounds, this filter expresses the condition `21 < age < 31`
 ```
 
 The user can also specify a one-sided bound by omitting "upper" or "lower". This filter expresses `age < 31`.
+
 ```json
 {
     "type": "bound",
@@ -327,6 +331,7 @@ The user can also specify a one-sided bound by omitting "upper" or "lower". This
 ```
 
 Likewise, this filter expresses `age >= 18`
+
 ```json
 {
     "type": "bound",
@@ -355,6 +360,7 @@ The interval filter supports the use of extraction functions, see [Filtering wit
 If an extraction function is used with this filter, the extraction function should output values that are parseable as long milliseconds.
 
 The following example filters on the time ranges of October 1-7, 2014 and November 15-16, 2014.
+
 ```json
 {
     "type" : "interval",


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