You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2018/01/18 01:35:45 UTC

[3/4] calcite git commit: [CALCITE-2107] Pass timezone as part of granularity in query sent to Druid (Nishant Bangarwa)

[CALCITE-2107] Pass timezone as part of granularity in query sent to Druid (Nishant Bangarwa)

Close apache/calcite#605


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/7f86f4e6
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/7f86f4e6
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/7f86f4e6

Branch: refs/heads/master
Commit: 7f86f4e6e416d134a436bdc9cacb478bd288be6b
Parents: 3e958cf
Author: Nishant <ni...@gmail.com>
Authored: Sat Jan 13 02:44:45 2018 +0530
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 17 17:34:54 2018 -0800

----------------------------------------------------------------------
 .../adapter/druid/DruidDateTimeUtils.java       |  33 +--
 .../calcite/adapter/druid/DruidQuery.java       |  68 +++---
 .../calcite/adapter/druid/DruidRules.java       |  24 ++-
 .../calcite/adapter/druid/Granularities.java    | 112 ++++++++++
 .../calcite/adapter/druid/Granularity.java      |  45 ++--
 .../adapter/druid/TimeExtractionFunction.java   |  31 ++-
 .../org/apache/calcite/test/DruidAdapterIT.java | 214 ++++++++++---------
 7 files changed, 338 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/7f86f4e6/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
index 566e781..d3588f9 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
@@ -293,14 +293,15 @@ public class DruidDateTimeUtils {
   }
 
   /**
-   * Infers granularity from a timeunit.
-   * It support {@code FLOOR(<time> TO <timeunit>)} and {@code EXTRACT(<timeunit> FROM <time>)}.
-   * It returns null if it cannot be inferred.
+   * Infers granularity from a time unit.
+   * It supports {@code FLOOR(<time> TO <timeunit>)}
+   * and {@code EXTRACT(<timeunit> FROM <time>)}.
+   * Returns null if it cannot be inferred.
    *
    * @param node the Rex node
    * @return the granularity, or null if it cannot be inferred
    */
-  public static Granularity extractGranularity(RexNode node) {
+  public static Granularity extractGranularity(RexNode node, String timeZone) {
     final int flagIndex;
     switch (node.getKind()) {
     case EXTRACT:
@@ -318,29 +319,7 @@ public class DruidDateTimeUtils {
     }
     final RexLiteral flag = (RexLiteral) call.operands.get(flagIndex);
     final TimeUnitRange timeUnit = (TimeUnitRange) flag.getValue();
-    if (timeUnit == null) {
-      return null;
-    }
-    switch (timeUnit) {
-    case YEAR:
-      return Granularity.YEAR;
-    case QUARTER:
-      return Granularity.QUARTER;
-    case MONTH:
-      return Granularity.MONTH;
-    case WEEK:
-      return Granularity.WEEK;
-    case DAY:
-      return Granularity.DAY;
-    case HOUR:
-      return Granularity.HOUR;
-    case MINUTE:
-      return Granularity.MINUTE;
-    case SECOND:
-      return Granularity.SECOND;
-    default:
-      return null;
-    }
+    return Granularities.createGranularity(timeUnit, timeZone);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/7f86f4e6/druid/src/main/java/org/apache/calcite/adapter/druid/DruidQuery.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidQuery.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidQuery.java
index 492392e..4dd56e5 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidQuery.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidQuery.java
@@ -542,7 +542,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
     final List<DimensionSpec> dimensions = new ArrayList<>();
     final List<JsonAggregation> aggregations = new ArrayList<>();
     final List<JsonPostAggregation> postAggs = new ArrayList<>();
-    Granularity finalGranularity = Granularity.ALL;
+    Granularity finalGranularity = Granularities.all();
     Direction timeSeriesDirection = null;
     JsonLimit limit = null;
     TimeExtractionDimensionSpec timeExtractionDimensionSpec = null;
@@ -563,7 +563,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
             final String originalFieldName = druidTable.getRowType(getCluster().getTypeFactory())
                 .getFieldList().get(ref.getIndex()).getName();
             if (originalFieldName.equals(druidTable.timestampFieldName)) {
-              finalGranularity = Granularity.ALL;
+              finalGranularity = Granularities.all();
               String extractColumnName = SqlValidatorUtil.uniquify(EXTRACT_COLUMN_NAME_PREFIX,
                   usedFieldNames, SqlValidatorUtil.EXPR_SUGGESTER);
               timeExtractionDimensionSpec = TimeExtractionDimensionSpec.makeFullTimeExtract(
@@ -579,16 +579,18 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
           } else if (project instanceof RexCall) {
             // Call, check if we should infer granularity
             final RexCall call = (RexCall) project;
-            final Granularity funcGranularity = DruidDateTimeUtils.extractGranularity(call);
+            final Granularity funcGranularity =
+                DruidDateTimeUtils.extractGranularity(call, config.timeZone());
             if (funcGranularity != null) {
               final String extractColumnName;
               switch (call.getKind()) {
               case EXTRACT:
                 // case extract field from time column
-                finalGranularity = Granularity.ALL;
-                extractColumnName = SqlValidatorUtil.uniquify(EXTRACT_COLUMN_NAME_PREFIX
-                        + "_" + funcGranularity.value, usedFieldNames,
-                    SqlValidatorUtil.EXPR_SUGGESTER);
+                finalGranularity = Granularities.all();
+                extractColumnName =
+                    SqlValidatorUtil.uniquify(EXTRACT_COLUMN_NAME_PREFIX + "_"
+                        + funcGranularity.getType().lowerName, usedFieldNames,
+                        SqlValidatorUtil.EXPR_SUGGESTER);
                 timeExtractionDimensionSpec = TimeExtractionDimensionSpec.makeTimeExtract(
                     funcGranularity, extractColumnName, config.timeZone());
                 dimensions.add(timeExtractionDimensionSpec);
@@ -598,13 +600,14 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
                 // case floor time column
                 if (groupSet.cardinality() > 1) {
                   // case we have more than 1 group by key -> then will have druid group by
-                  extractColumnName = SqlValidatorUtil.uniquify(FLOOR_COLUMN_NAME_PREFIX
-                          + "_" + funcGranularity.value, usedFieldNames,
-                      SqlValidatorUtil.EXPR_SUGGESTER);
+                  extractColumnName =
+                      SqlValidatorUtil.uniquify(FLOOR_COLUMN_NAME_PREFIX
+                          + "_" + funcGranularity.getType().lowerName,
+                          usedFieldNames, SqlValidatorUtil.EXPR_SUGGESTER);
                   dimensions.add(
                       TimeExtractionDimensionSpec.makeTimeFloor(funcGranularity,
                           extractColumnName, config.timeZone()));
-                  finalGranularity = Granularity.ALL;
+                  finalGranularity = Granularities.all();
                   builder.add(extractColumnName);
                 } else {
                   // case timeseries we can not use extraction function
@@ -630,7 +633,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
         for (int groupKey : groupSet) {
           final String s = fieldNames.get(groupKey);
           if (s.equals(druidTable.timestampFieldName)) {
-            finalGranularity = Granularity.ALL;
+            finalGranularity = Granularities.all();
             // Generate unique name as timestampFieldName is taken
             String extractColumnName = SqlValidatorUtil.uniquify(EXTRACT_COLUMN_NAME_PREFIX,
                 usedFieldNames, SqlValidatorUtil.EXPR_SUGGESTER);
@@ -702,7 +705,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
         queryType = QueryType.TIMESERIES;
         assert fetch == null;
       } else if (dimensions.size() == 1
-          && finalGranularity == Granularity.ALL
+          && finalGranularity.equals(Granularities.all())
           && sortsMetric
           && collations.size() == 1
           && fetch != null
@@ -731,7 +734,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
         generator.writeStringField("dataSource", druidTable.dataSource);
         generator.writeBooleanField("descending", timeSeriesDirection != null
             && timeSeriesDirection == Direction.DESCENDING);
-        generator.writeStringField("granularity", finalGranularity.value);
+        writeField(generator, "granularity", finalGranularity);
         writeFieldIf(generator, "filter", jsonFilter);
         writeField(generator, "aggregations", aggregations);
         writeFieldIf(generator, "postAggregations", postAggs.size() > 0 ? postAggs : null);
@@ -740,7 +743,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
         generator.writeFieldName("context");
         // The following field is necessary to conform with SQL semantics (CALCITE-1589)
         generator.writeStartObject();
-        final boolean isCountStar = Granularity.ALL == finalGranularity
+        final boolean isCountStar = finalGranularity.equals(Granularities.all())
             && aggregations.size() == 1
             && aggregations.get(0).type.equals("count");
         //Count(*) returns 0 if result set is empty thus need to set skipEmptyBuckets to false
@@ -755,7 +758,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
 
         generator.writeStringField("queryType", "topN");
         generator.writeStringField("dataSource", druidTable.dataSource);
-        generator.writeStringField("granularity", finalGranularity.value);
+        writeField(generator, "granularity", finalGranularity);
         writeField(generator, "dimension", dimensions.get(0));
         generator.writeStringField("metric", fieldNames.get(collationIndexes.get(0)));
         writeFieldIf(generator, "filter", jsonFilter);
@@ -771,7 +774,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
         generator.writeStartObject();
         generator.writeStringField("queryType", "groupBy");
         generator.writeStringField("dataSource", druidTable.dataSource);
-        generator.writeStringField("granularity", finalGranularity.value);
+        writeField(generator, "granularity", finalGranularity);
         writeField(generator, "dimensions", dimensions);
         writeFieldIf(generator, "limitSpec", limit);
         writeFieldIf(generator, "filter", jsonFilter);
@@ -793,7 +796,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
         writeFieldIf(generator, "filter", jsonFilter);
         writeField(generator, "dimensions", translator.dimensions);
         writeField(generator, "metrics", translator.metrics);
-        generator.writeStringField("granularity", finalGranularity.value);
+        writeField(generator, "granularity", finalGranularity);
 
         generator.writeFieldName("pagingSpec");
         generator.writeStartObject();
@@ -824,7 +827,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
                     ? DruidTable.DEFAULT_TIMESTAMP_COLUMN : s;
               }
             }));
-        generator.writeStringField("granularity", finalGranularity.value);
+        writeField(generator, "granularity", finalGranularity);
         generator.writeStringField("resultFormat", "compactedList");
         if (fetch != null) {
           generator.writeNumberField("limit", fetch);
@@ -1144,12 +1147,11 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
       metrics.clear();
     }
 
+    /** Formats timestamp values to druid format using
+     * {@link DruidQuery.Translator#dateFormatter}. This is needed when pushing
+     * timestamp comparisons to druid using a TimeFormatExtractionFunction that
+     * returns a string value. */
     @SuppressWarnings("incomplete-switch")
-    /**
-     * formatDateString is used to format timestamp values to druid format using
-     * {@link DruidQuery.Translator#dateFormatter}. This is needed when pushing timestamp
-     * comparisons to druid using TimeFormatExtractionFunction that returns a string value.
-     */
     String translate(RexNode e, boolean set, boolean formatDateString) {
       int index = -1;
       switch (e.getKind()) {
@@ -1178,7 +1180,7 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
       case FLOOR:
       case EXTRACT:
         final RexCall call = (RexCall) e;
-        assert DruidDateTimeUtils.extractGranularity(call) != null;
+        assert DruidDateTimeUtils.extractGranularity(call, timeZone) != null;
         index = RelOptUtil.InputFinder.bits(e).asList().get(0);
         break;
       case IS_TRUE:
@@ -1238,22 +1240,28 @@ public class DruidQuery extends AbstractRelNode implements BindableRel {
             call.getOperands().get(posRef).getType().getFamily()
                 == SqlTypeFamily.NUMERIC;
         boolean formatDateString = false;
-        final Granularity granularity = DruidDateTimeUtils.extractGranularity(posRefNode);
+        final Granularity granularity =
+            DruidDateTimeUtils.extractGranularity(posRefNode, timeZone);
         // in case no extraction the field will be omitted from the serialization
-        ExtractionFunction extractionFunction = null;
+        final ExtractionFunction extractionFunction;
         if (granularity != null) {
           switch (posRefNode.getKind()) {
           case EXTRACT:
             extractionFunction =
-                TimeExtractionFunction.createExtractFromGranularity(granularity, timeZone);
+                TimeExtractionFunction.createExtractFromGranularity(granularity,
+                    timeZone);
             break;
           case FLOOR:
             extractionFunction =
-                TimeExtractionFunction.createFloorFromGranularity(granularity, timeZone);
+                TimeExtractionFunction.createFloorFromGranularity(granularity,
+                    timeZone);
             formatDateString = true;
             break;
-
+          default:
+            extractionFunction = null;
           }
+        } else {
+          extractionFunction = null;
         }
         String dimName = tr(e, posRef, formatDateString);
         if (dimName.equals(DruidConnectionImpl.DEFAULT_RESPONSE_TIMESTAMP_COLUMN)) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/7f86f4e6/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
index 2743820..290f548 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
@@ -257,9 +257,12 @@ public class DruidRules {
       final List<RexNode> residualPreds = new ArrayList<>(triple.getRight());
       List<Interval> intervals = null;
       if (!triple.getLeft().isEmpty()) {
+        final String timeZone = cluster.getPlanner().getContext()
+            .unwrap(CalciteConnectionConfig.class).timeZone();
+        assert timeZone != null;
         intervals = DruidDateTimeUtils.createInterval(
             RexUtil.composeConjunction(rexBuilder, triple.getLeft(), false),
-            cluster.getPlanner().getContext().unwrap(CalciteConnectionConfig.class).timeZone());
+            timeZone);
         if (intervals == null || intervals.isEmpty()) {
           // Case we have an filter with extract that can not be written as interval push down
           triple.getMiddle().addAll(triple.getLeft());
@@ -640,6 +643,7 @@ public class DruidRules {
       if (!DruidQuery.isValidSignature(query.signature() + 'a')) {
         return;
       }
+
       if (aggregate.indicator
               || aggregate.getGroupSets().size() != 1
               || BAD_AGG.apply(ImmutableTriple.of(aggregate, (RelNode) aggregate, query))
@@ -920,7 +924,10 @@ public class DruidRules {
         if (e instanceof RexCall) {
           // It is a call, check that it is EXTRACT and follow-up conditions
           final RexCall call = (RexCall) e;
-          if (DruidDateTimeUtils.extractGranularity(call) == null) {
+          final String timeZone = query.getCluster().getPlanner().getContext()
+              .unwrap(CalciteConnectionConfig.class).timeZone();
+          assert timeZone != null;
+          if (DruidDateTimeUtils.extractGranularity(call, timeZone) == null) {
             return -1;
           }
           if (idxTimestamp != -1 && hasFloor) {
@@ -1108,7 +1115,10 @@ public class DruidRules {
         RexNode node = project.getProjects().get(index);
         if (node instanceof RexCall) {
           RexCall call = (RexCall) node;
-          assert DruidDateTimeUtils.extractGranularity(call) != null;
+          final String timeZone = query.getCluster().getPlanner().getContext()
+              .unwrap(CalciteConnectionConfig.class).timeZone();
+          assert timeZone != null;
+          assert DruidDateTimeUtils.extractGranularity(call, timeZone) != null;
           if (call.getKind() == SqlKind.FLOOR) {
             newSet.addAll(RelOptUtil.InputFinder.bits(call));
           }
@@ -1140,8 +1150,12 @@ public class DruidRules {
           newSet.set(((RexInputRef) node).getIndex());
         } else if (node instanceof RexCall) {
           RexCall call = (RexCall) node;
-          assert DruidDateTimeUtils.extractGranularity(call) != null;
-          // when we have extract from time columnthe rexCall is in the form of /Reinterpret$0
+          final String timeZone = query.getCluster().getPlanner().getContext()
+              .unwrap(CalciteConnectionConfig.class).timeZone();
+          assert timeZone != null;
+          assert DruidDateTimeUtils.extractGranularity(call, timeZone) != null;
+          // when we have extract from time column the rexCall is of the form
+          // "/Reinterpret$0"
           newSet.addAll(RelOptUtil.InputFinder.bits(call));
         }
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/7f86f4e6/druid/src/main/java/org/apache/calcite/adapter/druid/Granularities.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/Granularities.java b/druid/src/main/java/org/apache/calcite/adapter/druid/Granularities.java
new file mode 100644
index 0000000..df1b291
--- /dev/null
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/Granularities.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.druid;
+
+import org.apache.calcite.avatica.util.TimeUnitRange;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import javax.annotation.Nonnull;
+
+import static org.apache.calcite.adapter.druid.DruidQuery.writeFieldIf;
+
+/**
+ * Factory methods and helpers for {@link Granularity}.
+ */
+public class Granularities {
+  // Private constructor for utility class
+  private Granularities() {}
+
+  /** Returns a Granularity that causes all rows to be rolled up into one. */
+  public static Granularity all() {
+    return AllGranularity.INSTANCE;
+  }
+
+  /** Creates a Granularity based on a time unit.
+   *
+   * <p>When used in a query, Druid will rollup and round time values based on
+   * specified period and timezone. */
+  @Nonnull public static Granularity createGranularity(TimeUnitRange timeUnit,
+      String timeZone) {
+    switch (timeUnit) {
+    case YEAR:
+      return new PeriodGranularity(Granularity.Type.YEAR, "P1Y", timeZone);
+    case QUARTER:
+      return new PeriodGranularity(Granularity.Type.QUARTER, "P3M", timeZone);
+    case MONTH:
+      return new PeriodGranularity(Granularity.Type.MONTH, "P1M", timeZone);
+    case WEEK:
+      return new PeriodGranularity(Granularity.Type.WEEK, "P1W", timeZone);
+    case DAY:
+      return new PeriodGranularity(Granularity.Type.DAY, "P1D", timeZone);
+    case HOUR:
+      return new PeriodGranularity(Granularity.Type.HOUR, "PT1H", timeZone);
+    case MINUTE:
+      return new PeriodGranularity(Granularity.Type.MINUTE, "PT1M", timeZone);
+    case SECOND:
+      return new PeriodGranularity(Granularity.Type.SECOND, "PT1S", timeZone);
+    default:
+      throw new AssertionError(timeUnit);
+    }
+  }
+
+  /** Implementation of {@link Granularity} for {@link Granularity.Type#ALL}.
+   * A singleton. */
+  private enum AllGranularity implements Granularity {
+    INSTANCE;
+
+    @Override public void write(JsonGenerator generator) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("type", "all");
+      generator.writeEndObject();
+    }
+
+    @Nonnull public Type getType() {
+      return Type.ALL;
+    }
+  }
+
+  /** Implementation of {@link Granularity} based on a time unit.
+   * Corresponds to PeriodGranularity in Druid. */
+  private static class PeriodGranularity implements Granularity {
+    private final Type type;
+    private final String period;
+    private final String timeZone;
+
+    private PeriodGranularity(Type type, String period, String timeZone) {
+      this.type = Preconditions.checkNotNull(type);
+      this.period = Preconditions.checkNotNull(period);
+      this.timeZone = Preconditions.checkNotNull(timeZone);
+    }
+
+    @Override public void write(JsonGenerator generator) throws IOException {
+      generator.writeStartObject();
+      generator.writeStringField("type", "period");
+      writeFieldIf(generator, "period", period);
+      writeFieldIf(generator, "timeZone", timeZone);
+      generator.writeEndObject();
+    }
+
+    @Nonnull public Type getType() {
+      return type;
+    }
+  }
+}
+
+// End Granularities.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/7f86f4e6/druid/src/main/java/org/apache/calcite/adapter/druid/Granularity.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/Granularity.java b/druid/src/main/java/org/apache/calcite/adapter/druid/Granularity.java
index 0ee2e20..ffedec2 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/Granularity.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/Granularity.java
@@ -17,22 +17,39 @@
 package org.apache.calcite.adapter.druid;
 
 import java.util.Locale;
+import javax.annotation.Nonnull;
 
-/** Granularity of a Druid query. */
-public enum Granularity {
-  ALL,
-  YEAR,
-  QUARTER,
-  MONTH,
-  WEEK,
-  DAY,
-  HOUR,
-  MINUTE,
-  SECOND,
-  NONE;
+/**
+ * A strategy by which Druid rolls up rows into sub-totals based on their
+ * timestamp values.
+ *
+ * <p>Typical granularities are based upon time units (e.g. 1 day or
+ * 15 minutes). A special granularity, all, combines all rows into a single
+ * total.
+ *
+ * <p>A Granularity instance is immutable, and generates a JSON string as
+ * part of a Druid query.
+ *
+ * @see Granularities
+ */
+public interface Granularity extends DruidQuery.Json {
+  /** Type of supported periods for granularity. */
+  enum Type {
+    ALL,
+    YEAR,
+    QUARTER,
+    MONTH,
+    WEEK,
+    DAY,
+    HOUR,
+    MINUTE,
+    SECOND;
+
+    /** Lower-case name, e.g. "all", "minute". */
+    public final String lowerName = name().toLowerCase(Locale.ROOT);
+  }
 
-  /** JSON attribute value in a Druid query. */
-  public final String value = name().toLowerCase(Locale.ROOT);
+  @Nonnull Type getType();
 }
 
 // End Granularity.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/7f86f4e6/druid/src/main/java/org/apache/calcite/adapter/druid/TimeExtractionFunction.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/TimeExtractionFunction.java b/druid/src/main/java/org/apache/calcite/adapter/druid/TimeExtractionFunction.java
index ba4d9d7..d05949b 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/TimeExtractionFunction.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/TimeExtractionFunction.java
@@ -45,16 +45,20 @@ public class TimeExtractionFunction implements ExtractionFunction {
       TimeUnitRange.YEAR,
       TimeUnitRange.MONTH,
       TimeUnitRange.DAY,
-      TimeUnitRange.WEEK);
+      TimeUnitRange.WEEK,
+      TimeUnitRange.HOUR,
+      TimeUnitRange.MINUTE,
+      TimeUnitRange.SECOND);
 
   public static final String ISO_TIME_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
 
   private final String format;
-  private final String granularity;
+  private final Granularity granularity;
   private final String timeZone;
   private final String local;
 
-  public TimeExtractionFunction(String format, String granularity, String timeZone, String local) {
+  public TimeExtractionFunction(String format, Granularity granularity, String timeZone,
+      String local) {
     this.format = format;
     this.granularity = granularity;
     this.timeZone = timeZone;
@@ -82,7 +86,6 @@ public class TimeExtractionFunction implements ExtractionFunction {
 
   /**
    * Creates the time format extraction function for the given granularity.
-   * Only YEAR, MONTH, and DAY granularity are supported.
    *
    * @param granularity granularity to apply to the column
    * @return the time extraction function corresponding to the granularity input unit
@@ -90,16 +93,22 @@ public class TimeExtractionFunction implements ExtractionFunction {
    */
   public static TimeExtractionFunction createExtractFromGranularity(
       Granularity granularity, String timeZone) {
-    switch (granularity) {
+    final String local = Locale.ROOT.toLanguageTag();
+    switch (granularity.getType()) {
     case DAY:
-      return new TimeExtractionFunction("d", null, timeZone, Locale.getDefault().toLanguageTag());
+      return new TimeExtractionFunction("d", null, timeZone, local);
     case MONTH:
-      return new TimeExtractionFunction("M", null, timeZone, Locale.getDefault().toLanguageTag());
+      return new TimeExtractionFunction("M", null, timeZone, local);
     case YEAR:
-      return new TimeExtractionFunction("yyyy", null, timeZone,
-          Locale.getDefault().toLanguageTag());
+      return new TimeExtractionFunction("yyyy", null, timeZone, local);
     case WEEK:
-      return new TimeExtractionFunction("w", null, timeZone, Locale.getDefault().toLanguageTag());
+      return new TimeExtractionFunction("w", null, timeZone, local);
+    case HOUR:
+      return new TimeExtractionFunction("H", null, timeZone, local);
+    case MINUTE:
+      return new TimeExtractionFunction("m", null, timeZone, local);
+    case SECOND:
+      return new TimeExtractionFunction("s", null, timeZone, local);
     default:
       throw new IllegalArgumentException("Granularity [" + granularity + "] is not supported");
     }
@@ -113,7 +122,7 @@ public class TimeExtractionFunction implements ExtractionFunction {
    */
   public static TimeExtractionFunction createFloorFromGranularity(
       Granularity granularity, String timeZone) {
-    return new TimeExtractionFunction(ISO_TIME_FORMAT, granularity.value, timeZone, Locale
+    return new TimeExtractionFunction(ISO_TIME_FORMAT, granularity, timeZone, Locale
         .getDefault().toLanguageTag());
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/7f86f4e6/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
----------------------------------------------------------------------
diff --git a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
index 3e7f7bd..e4b535e 100644
--- a/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
+++ b/druid/src/test/java/org/apache/calcite/test/DruidAdapterIT.java
@@ -197,7 +197,7 @@ public class DruidAdapterIT {
         + "intervals=[[1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z]], "
         + "filter=[=($17, 'Jeremy Corbyn')], groups=[{7}], aggs=[[]])\n";
     final String druidQuery = "{'queryType':'groupBy',"
-        + "'dataSource':'wikiticker','granularity':'all',"
+        + "'dataSource':'wikiticker','granularity':{'type':'all'},"
         + "'dimensions':[{'type':'default','dimension':'countryName'}],'limitSpec':{'type':'default'},"
         + "'filter':{'type':'selector','dimension':'page','value':'Jeremy Corbyn'},"
         + "'aggregations':[],"
@@ -223,7 +223,7 @@ public class DruidAdapterIT {
         + "  BindableProject(EXPR$0=[$1])\n"
         + "    DruidQuery(table=[[wiki, wikiticker]], intervals=[[1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z]], projects=[[FLOOR($0, FLAG(DAY)), $1]], groups=[{0}], aggs=[[SUM($1)]])\n";
     final String druidQuery = "{'queryType':'timeseries',"
-        + "'dataSource':'wikiticker','descending':false,'granularity':'day',"
+        + "'dataSource':'wikiticker','descending':false,'granularity':{'type':'period','period':'P1D','timeZone':'UTC'},"
         + "'aggregations':[{'type':'longSum','name':'EXPR$0','fieldName':'added'}],"
         + "'intervals':['1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z'],"
         + "'context':{'skipEmptyBuckets':true}}";
@@ -243,7 +243,7 @@ public class DruidAdapterIT {
     final String druidQuery = "{'queryType':'scan',"
         + "'dataSource':'wikiticker',"
         + "'intervals':['1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z'],"
-        + "'columns':['__time'],'granularity':'all',"
+        + "'columns':['__time'],'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList','limit':1}";
 
     sql(sql, WIKI_AUTO2)
@@ -262,7 +262,7 @@ public class DruidAdapterIT {
     final String explain =
         "DruidQuery(table=[[wiki, wikiticker]], intervals=[[1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z]], projects=[[FLOOR($0, FLAG(DAY)), $1]], groups=[{0}], aggs=[[SUM($1)]])\n";
     final String druidQuery = "{'queryType':'timeseries',"
-        + "'dataSource':'wikiticker','descending':false,'granularity':'day',"
+        + "'dataSource':'wikiticker','descending':false,'granularity':{'type':'period','period':'P1D','timeZone':'UTC'},"
         + "'aggregations':[{'type':'longSum','name':'EXPR$1','fieldName':'added'}],"
         + "'intervals':['1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z'],"
         + "'context':{'skipEmptyBuckets':true}}";
@@ -302,7 +302,7 @@ public class DruidAdapterIT {
         + "where \"page\" = 'Jeremy Corbyn'\n"
         + "group by floor(\"__time\" to SECOND)";
     final String druidQuery = "{'queryType':'timeseries',"
-        + "'dataSource':'wikiticker','descending':false,'granularity':'second',"
+        + "'dataSource':'wikiticker','descending':false,'granularity':{'type':'period','period':'PT1S','timeZone':'UTC'},"
         + "'filter':{'type':'selector','dimension':'page','value':'Jeremy Corbyn'},"
         + "'aggregations':[{'type':'longSum','name':'EXPR$1','fieldName':'added'}],"
         + "'intervals':['1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z'],"
@@ -319,7 +319,7 @@ public class DruidAdapterIT {
         + "from \"" + tableName + "\"\n"
         + "where \"page\" = 'Jeremy Corbyn'";
     final String druidQuery = "{'queryType':'groupBy',"
-        + "'dataSource':'wikiticker','granularity':'all',"
+        + "'dataSource':'wikiticker','granularity':{'type':'all'},"
         + "'dimensions':[{'type':'default','dimension':'countryName'}],'limitSpec':{'type':'default'},"
         + "'filter':{'type':'selector','dimension':'page','value':'Jeremy Corbyn'},"
         + "'aggregations':[],"
@@ -344,7 +344,7 @@ public class DruidAdapterIT {
     final String druidQuery = "{'queryType':'scan',"
         + "'dataSource':'wikiticker',"
         + "'intervals':['1900-01-01T00:00:00.000Z/2015-10-12T00:00:00.000Z'],"
-        + "'columns':['__time'],'granularity':'all',"
+        + "'columns':['__time'],'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList'";
     sql(sql, WIKI_AUTO2)
         .limit(2)
@@ -366,7 +366,7 @@ public class DruidAdapterIT {
         + "intervals=[[1900-01-01T00:00:00.000Z/2015-10-12T00:00:00.000Z]], "
         + "groups=[{0}], aggs=[[]])\n";
     final String subDruidQuery = "{'queryType':'groupBy','dataSource':'wikiticker',"
-        + "'granularity':'all','dimensions':[{'type':'extraction',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'extraction',"
         + "'dimension':'__time','outputName':'extract',"
         + "'extractionFn':{'type':'timeFormat'";
     sql(sql, WIKI_AUTO2)
@@ -411,7 +411,8 @@ public class DruidAdapterIT {
         + "EnumerableInterpreter\n"
         + "  DruidQuery(table=[[foodmart, foodmart]], intervals=[[1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z]], groups=[{30}], aggs=[[]])";
     final String sql = "select distinct \"state_province\" from \"foodmart\"";
-    final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart','granularity':'all',"
+    final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
+        + "'granularity':{'type':'all'},"
         + "'dimensions':[{'type':'default','dimension':'state_province'}],'limitSpec':{'type':'default'},"
         + "'aggregations':[],"
         + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z']}";
@@ -442,17 +443,18 @@ public class DruidAdapterIT {
 
   @Test public void testGroupbyMetric() {
     final String sql = "select  \"store_sales\" ,\"product_id\" from \"foodmart\" "
-            + "where \"product_id\" = 1020" + "group by \"store_sales\" ,\"product_id\" ";
+        + "where \"product_id\" = 1020" + "group by \"store_sales\" ,\"product_id\" ";
     final String plan = "PLAN=EnumerableInterpreter\n"
         + "  DruidQuery(table=[[foodmart, foodmart]], "
         + "intervals=[[1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z]], filter=[=($1, 1020)],"
         + " projects=[[$90, $1]], groups=[{0, 1}], aggs=[[]])";
-    final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart','granularity':'all',"
-            + "'dimensions':[{'type':'default','dimension':'store_sales'},"
-            + "{'type':'default','dimension':'product_id'}],'limitSpec':{'type':'default'},'"
-            + "filter':{'type':'selector','dimension':'product_id','value':'1020'},"
-            + "'aggregations':[],"
-            + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z']}";
+    final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
+        + "'granularity':{'type':'all'},"
+        + "'dimensions':[{'type':'default','dimension':'store_sales'},"
+        + "{'type':'default','dimension':'product_id'}],'limitSpec':{'type':'default'},'"
+        + "filter':{'type':'selector','dimension':'product_id','value':'1020'},"
+        + "'aggregations':[],"
+        + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z']}";
     sql(sql)
         .explainContains(plan)
         .queryContains(druidChecker(druidQuery))
@@ -467,7 +469,7 @@ public class DruidAdapterIT {
     final String sql = "select \"product_id\" from \"foodmart\" where "
             + "\"product_id\" = 1020 group by \"product_id\"";
     final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-            + "'granularity':'all','dimensions':[{'type':'default',"
+            + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
             + "'dimension':'product_id'}],"
             + "'limitSpec':{'type':'default'},'filter':{'type':'selector',"
             + "'dimension':'product_id','value':'1020'},"
@@ -481,7 +483,7 @@ public class DruidAdapterIT {
             + "\"product_id\" = 1020";
     final String sql = "select \"id\" from (" + innerQuery + ") group by \"id\"";
     final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-            + "'granularity':'all',"
+            + "'granularity':{'type':'all'},"
             + "'dimensions':[{'type':'default','dimension':'product_id'}],"
             + "'limitSpec':{'type':'default'},"
             + "'filter':{'type':'selector','dimension':'product_id','value':'1020'},"
@@ -531,7 +533,7 @@ public class DruidAdapterIT {
             "gender=F; state_province=WA")
         .queryContains(
             druidChecker("{'queryType':'groupBy','dataSource':'foodmart',"
-                + "'granularity':'all','dimensions':[{'type':'default',"
+                + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
                 + "'dimension':'gender'},{'type':'default',"
                 + "'dimension':'state_province'}],'limitSpec':{'type':'default',"
                 + "'columns':[{'dimension':'state_province','direction':'ascending',"
@@ -566,7 +568,7 @@ public class DruidAdapterIT {
         + "offset 2 fetch next 3 rows only";
     final String druidQuery = "{'queryType':'scan','dataSource':'foodmart',"
         + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
-        + "'columns':['state_province','product_name'],'granularity':'all',"
+        + "'columns':['state_province','product_name'],'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList'}";
     sql(sql)
         .runs()
@@ -578,7 +580,7 @@ public class DruidAdapterIT {
         + "from \"foodmart\" fetch next 3 rows only";
     final String druidQuery = "{'queryType':'scan','dataSource':'foodmart',"
         + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
-        + "'columns':['gender','state_province'],'granularity':'all',"
+        + "'columns':['gender','state_province'],'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList','limit':3";
     sql(sql)
         .runs()
@@ -589,7 +591,7 @@ public class DruidAdapterIT {
     final String sql = "select distinct \"gender\", \"state_province\"\n"
         + "from \"foodmart\" fetch next 3 rows only";
     final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'default','dimension':'gender'},"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'default','dimension':'gender'},"
         + "{'type':'default','dimension':'state_province'}],'limitSpec':{'type':'default',"
         + "'limit':3,'columns':[]},"
         + "'aggregations':[],"
@@ -615,7 +617,7 @@ public class DruidAdapterIT {
         + "group by \"brand_name\", \"gender\"\n"
         + "order by s desc limit 3";
     final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'default',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
         + "'dimension':'brand_name'},{'type':'default','dimension':'gender'}],"
         + "'limitSpec':{'type':'default','limit':3,'columns':[{'dimension':'S',"
         + "'direction':'descending','dimensionOrder':'numeric'}]},"
@@ -653,13 +655,14 @@ public class DruidAdapterIT {
         + "from \"foodmart\"\n"
         + "group by \"brand_name\"\n"
         + "order by s desc limit 3";
-    final String approxDruid = "{'queryType':'topN','dataSource':'foodmart','granularity':'all',"
+    final String approxDruid = "{'queryType':'topN','dataSource':'foodmart',"
+        + "'granularity':{'type':'all'},"
         + "'dimension':{'type':'default','dimension':'brand_name'},'metric':'S',"
         + "'aggregations':[{'type':'longSum','name':'S','fieldName':'unit_sales'}],"
         + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
         + "'threshold':3}";
     final String exactDruid = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'default',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
         + "'dimension':'brand_name'}],'limitSpec':{'type':'default','limit':3,"
         + "'columns':[{'dimension':'S','direction':'descending',"
         + "'dimensionOrder':'numeric'}]},'aggregations':[{'type':'longSum',"
@@ -708,7 +711,7 @@ public class DruidAdapterIT {
             "brand_name=Hermanos; D=1997-05-09 00:00:00; S=115")
         .explainContains(explain)
         .queryContains(
-            druidChecker("'queryType':'groupBy'", "'granularity':'all'", "'limitSpec"
+            druidChecker("'queryType':'groupBy'", "'granularity':{'type':'all'}", "'limitSpec"
                 + "':{'type':'default','limit':30,'columns':[{'dimension':'S',"
                 + "'direction':'descending','dimensionOrder':'numeric'}]}"));
   }
@@ -729,7 +732,7 @@ public class DruidAdapterIT {
         + "group by \"brand_name\", floor(\"timestamp\" to DAY)\n"
         + "order by s desc limit 30";
     final String druidQueryPart1 = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'default',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
         + "'dimension':'brand_name'},{'type':'extraction','dimension':'__time',"
         + "'outputName':'floor_day','extractionFn':{'type':'timeFormat'";
     final String druidQueryPart2 = "'limitSpec':{'type':'default','limit':30,"
@@ -761,7 +764,7 @@ public class DruidAdapterIT {
         + "group by \"brand_name\", floor(\"timestamp\" to DAY)\n"
         + "order by \"brand_name\"";
     final String subDruidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'default',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
         + "'dimension':'brand_name'},{'type':'extraction','dimension':'__time',"
         + "'outputName':'floor_day','extractionFn':{'type':'timeFormat'";
     final String explain = "    DruidQuery(table=[[foodmart, foodmart]], "
@@ -787,7 +790,7 @@ public class DruidAdapterIT {
         + "'filter':{'type':'and','fields':["
         + "{'type':'bound','dimension':'product_id','lower':'1500','lowerStrict':false,'ordering':'lexicographic'},"
         + "{'type':'bound','dimension':'product_id','upper':'1502','upperStrict':false,'ordering':'lexicographic'}]},"
-        + "'columns':['product_name','state_province','product_id'],'granularity':'all',"
+        + "'columns':['product_name','state_province','product_id'],'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList'";
     sql(sql)
         .limit(4)
@@ -820,7 +823,7 @@ public class DruidAdapterIT {
         + "'filter':{'type':'and','fields':["
         + "{'type':'bound','dimension':'product_id','lower':'1500','lowerStrict':false,'ordering':'numeric'},"
         + "{'type':'bound','dimension':'product_id','upper':'1502','upperStrict':false,'ordering':'numeric'}]},"
-        + "'columns':['product_name','state_province','product_id'],'granularity':'all',"
+        + "'columns':['product_name','state_province','product_id'],'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList'";
     sql(sql)
         .limit(4)
@@ -850,7 +853,7 @@ public class DruidAdapterIT {
     final String druidQuery = "{'queryType':'scan','dataSource':'foodmart',"
         + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
         + "'filter':{'type':'selector','dimension':'product_id','value':'-1'},"
-        + "'columns':['product_name'],'granularity':'all',"
+        + "'columns':['product_name'],'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList'}";
     sql(sql)
         .limit(4)
@@ -866,7 +869,7 @@ public class DruidAdapterIT {
         + "order by \"state_province\" desc, \"product_id\"";
     final String druidQuery = "{'queryType':'scan','dataSource':'foodmart',"
         + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
-        + "'columns':['product_id','product_name','state_province'],'granularity':'all',"
+        + "'columns':['product_id','product_name','state_province'],'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList'}";
     sql(sql)
         .limit(4)
@@ -927,7 +930,7 @@ public class DruidAdapterIT {
 
   @Test public void testCountGroupByEmpty() {
     final String druidQuery = "{'queryType':'timeseries','dataSource':'foodmart',"
-        + "'descending':false,'granularity':'all',"
+        + "'descending':false,'granularity':{'type':'all'},"
         + "'aggregations':[{'type':'count','name':'EXPR$0'}],"
         + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
         + "'context':{'skipEmptyBuckets':false}}";
@@ -1159,7 +1162,7 @@ public class DruidAdapterIT {
         + "($0, FLAG(MONTH)), $89]], groups=[{0, 1}], aggs=[[SUM($2), MAX($2)]], sort0=[2], "
         + "dir0=[DESC], fetch=[3])";
     final String druidQueryPart1 = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'default',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
         + "'dimension':'state_province'},{'type':'extraction','dimension':'__time',"
         + "'outputName':'floor_month','extractionFn':{'type':'timeFormat','format'";
     final String druidQueryPart2 = "'limitSpec':{'type':'default','limit':3,"
@@ -1192,7 +1195,7 @@ public class DruidAdapterIT {
         + "($0, FLAG(DAY)), $89]], groups=[{0, 1}], aggs=[[SUM($2), MAX($2)]], sort0=[2], "
         + "dir0=[DESC], fetch=[6])";
     final String druidQueryType = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions'";
+        + "'granularity':{'type':'all'},'dimensions'";
     final String limitSpec = "'limitSpec':{'type':'default','limit':6,"
         + "'columns':[{'dimension':'S','direction':'descending','dimensionOrder':'numeric'}]}";
     sql(sql)
@@ -1255,7 +1258,7 @@ public class DruidAdapterIT {
         + "intervals=[[1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z]], groups=[{29, 30}], "
         + "aggs=[[]])";
     final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'default','dimension':'city'},"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'default','dimension':'city'},"
         + "{'type':'default','dimension':'state_province'}],"
         + "'limitSpec':{'type':'default'},'aggregations':[],"
         + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z']}";
@@ -1292,7 +1295,8 @@ public class DruidAdapterIT {
         + "where \"product_name\" = 'High Top Dried Mushrooms'\n"
         + "and \"quarter\" in ('Q2', 'Q3')\n"
         + "and \"state_province\" = 'WA'";
-    final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart','granularity':'all',"
+    final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
+        + "'granularity':{'type':'all'},"
         + "'dimensions':[{'type':'default','dimension':'state_province'},"
         + "{'type':'default','dimension':'city'},"
         + "{'type':'default','dimension':'product_name'}],'limitSpec':{'type':'default'},"
@@ -1342,7 +1346,7 @@ public class DruidAdapterIT {
         + "{'type':'selector','dimension':'quarter','value':'Q3'}]},"
         + "{'type':'selector','dimension':'state_province','value':'WA'}]},"
         + "'columns':['state_province','city','product_name'],"
-        + "'granularity':'all',"
+        + "'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList'}";
     final String explain = "PLAN=EnumerableInterpreter\n"
         + "  DruidQuery(table=[[foodmart, foodmart]], "
@@ -1449,7 +1453,7 @@ public class DruidAdapterIT {
         + "'dataSource':'wikiticker',"
         + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
         + "'columns':['__time','countryName'],"
-        + "'granularity':'all',"
+        + "'granularity':{'type':'all'},"
         + "'resultFormat':'compactedList'";
     sql(sql, WIKI).explainContains(plan);
     sql(sql, WIKI).queryContains(druidChecker(druidQuery));
@@ -1480,7 +1484,7 @@ public class DruidAdapterIT {
         + "and \"timestamp\" > '1990-01-01 00:00:00 UTC' "
         + "group by \"timestamp\", \"product_id\" ";
     String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'extraction',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'extraction',"
         + "'dimension':'__time','outputName':'extract',"
         + "'extractionFn':{'type':'timeFormat','format':'yyyy-MM-dd";
     sql(sql)
@@ -1497,7 +1501,7 @@ public class DruidAdapterIT {
     sql(sql)
         .queryContains(
             druidChecker(
-                ",'granularity':'all'",
+                ",'granularity':{'type':'all'}",
                 "{'type':'extraction',"
                     + "'dimension':'__time','outputName':'extract_year',"
                     + "'extractionFn':{'type':'timeFormat','format':'yyyy',"
@@ -1514,7 +1518,7 @@ public class DruidAdapterIT {
     sql(sql)
         .queryContains(
             druidChecker(
-                ",'granularity':'all'",
+                ",'granularity':{'type':'all'}",
                 "{'type':'extraction',"
                     + "'dimension':'__time','outputName':'extract_month',"
                     + "'extractionFn':{'type':'timeFormat','format':'M',"
@@ -1533,7 +1537,7 @@ public class DruidAdapterIT {
     sql(sql)
         .queryContains(
             druidChecker(
-                ",'granularity':'all'",
+                ",'granularity':{'type':'all'}",
                 "{'type':'extraction',"
                     + "'dimension':'__time','outputName':'extract_day',"
                     + "'extractionFn':{'type':'timeFormat','format':'d',"
@@ -1556,7 +1560,7 @@ public class DruidAdapterIT {
     sql(sql)
         .queryContains(
             druidChecker(
-                ",'granularity':'all'",
+                ",'granularity':{'type':'all'}",
                 "{'type':'extraction',"
                     + "'dimension':'__time','outputName':'extract_0',"
                     + "'extractionFn':{'type':'timeFormat','format':'H',"
@@ -1577,7 +1581,7 @@ public class DruidAdapterIT {
     sql(sql)
         .queryContains(
             druidChecker(
-                ",'granularity':'all'",
+                ",'granularity':{'type':'all'}",
                 "{'type':'extraction',"
                     + "'dimension':'__time','outputName':'extract_day',"
                     + "'extractionFn':{'type':'timeFormat','format':'d',"
@@ -1611,7 +1615,7 @@ public class DruidAdapterIT {
     sql(sql)
         .queryContains(
             druidChecker(
-                ",'granularity':'all'", "{'type':'extraction',"
+                ",'granularity':{'type':'all'}", "{'type':'extraction',"
                     + "'dimension':'__time','outputName':'extract_day',"
                     + "'extractionFn':{'type':'timeFormat','format':'d',"
                     + "'timeZone':'UTC','locale':'en-US'}}", "{'type':'extraction',"
@@ -1643,7 +1647,7 @@ public class DruidAdapterIT {
     sql(sql)
         .queryContains(
             druidChecker(
-                ",'granularity':'all'", "{'type':'extraction',"
+                ",'granularity':{'type':'all'}", "{'type':'extraction',"
                     + "'dimension':'__time','outputName':'extract_day',"
                     + "'extractionFn':{'type':'timeFormat','format':'d',"
                     + "'timeZone':'UTC','locale':'en-US'}}"))
@@ -1662,7 +1666,7 @@ public class DruidAdapterIT {
         + "\"cases_per_pallet\" >= 8 and \"cases_per_pallet\" <= 10 and "
         + "\"units_per_case\" < 15 ";
     String druidQuery = "{'queryType':'timeseries','dataSource':'foodmart',"
-        + "'descending':false,'granularity':'all','filter':{'type':'and',"
+        + "'descending':false,'granularity':{'type':'all'},'filter':{'type':'and',"
         + "'fields':[{'type':'bound','dimension':'cases_per_pallet','lower':'8',"
         + "'lowerStrict':false,'ordering':'numeric'},{'type':'bound',"
         + "'dimension':'cases_per_pallet','upper':'10','upperStrict':false,"
@@ -1694,7 +1698,7 @@ public class DruidAdapterIT {
     sql(sql)
         .queryContains(
             druidChecker("{'queryType':'groupBy','dataSource':'foodmart',"
-                + "'granularity':'all','dimensions':[{'type':'default',"
+                + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
                 + "'dimension':'product_id'},{'type':'extraction','dimension':'__time',"
                 + "'outputName':'extract_day','extractionFn':{'type':'timeFormat',"
                 + "'format':'d','timeZone':'UTC','locale':'en-US'}},{'type':'extraction',"
@@ -1724,7 +1728,7 @@ public class DruidAdapterIT {
     sql(sql)
         .queryContains(
             druidChecker("{'queryType':'groupBy','dataSource':'foodmart',"
-                + "'granularity':'all','dimensions':[{'type':'default',"
+                + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
                 + "'dimension':'product_id'},{'type':'extraction','dimension':'__time',"
                 + "'outputName':'extract_day','extractionFn':{'type':'timeFormat',"
                 + "'format':'d','timeZone':'UTC','locale':'en-US'}},{'type':'extraction',"
@@ -1754,7 +1758,7 @@ public class DruidAdapterIT {
         + " WHERE EXTRACT(month from \"timestamp\") BETWEEN 10 AND 11 AND  \"product_id\" >= 1558"
         + " GROUP BY \"product_id\", EXTRACT(month from \"timestamp\")";
     String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'default',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'default',"
         + "'dimension':'product_id'},{'type':'extraction','dimension':'__time',"
         + "'outputName':'extract_month','extractionFn':{'type':'timeFormat',"
         + "'format':'M','timeZone':'UTC','locale':'en-US'}}],"
@@ -1781,7 +1785,7 @@ public class DruidAdapterIT {
     sql(sqlQuery)
         .queryContains(
             druidChecker("{'queryType':'groupBy',"
-                + "'dataSource':'foodmart','granularity':'all',"
+                + "'dataSource':'foodmart','granularity':{'type':'all'},"
                 + "'dimensions':[{'type':'default','dimension':'product_id'},"
                 + "{'type':'extraction','dimension':'__time','outputName':'extract_month',"
                 + "'extractionFn':{'type':'timeFormat','format':'M','timeZone':'UTC',"
@@ -1807,7 +1811,7 @@ public class DruidAdapterIT {
         + "\"product_id\"";
     sql(sqlQuery).queryContains(
         druidChecker("{'queryType':'groupBy','dataSource':'foodmart',"
-            + "'granularity':'all','dimensions':[{'type':'extraction',"
+            + "'granularity':{'type':'all'},'dimensions':[{'type':'extraction',"
             + "'dimension':'__time','outputName':'extract_month',"
             + "'extractionFn':{'type':'timeFormat','format':'M','timeZone':'UTC',"
             + "'locale':'en-US'}},{'type':'default','dimension':'product_id'}],"
@@ -1874,7 +1878,7 @@ public class DruidAdapterIT {
         + "sort1=[1], sort2=[3], sort3=[2], dir0=[DESC], "
         + "dir1=[ASC], dir2=[DESC], dir3=[ASC], fetch=[3])";
     final String expectedDruidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'extraction',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'extraction',"
         + "'dimension':'__time','outputName':'extract_year',"
         + "'extractionFn':{'type':'timeFormat','format':'yyyy','timeZone':'UTC',"
         + "'locale':'en-US'}},{'type':'extraction','dimension':'__time',"
@@ -1909,7 +1913,7 @@ public class DruidAdapterIT {
         + "EXTRACT(FLAG(MONTH), $0), $1, $89]], groups=[{0, 1, 2}], aggs=[[SUM($3)]], "
         + "sort0=[3], sort1=[1], sort2=[2], dir0=[DESC], dir1=[DESC], dir2=[ASC], fetch=[3])";
     final String expectedDruidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'extraction',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'extraction',"
         + "'dimension':'__time','outputName':'extract_year',"
         + "'extractionFn':{'type':'timeFormat','format':'yyyy','timeZone':'UTC',"
         + "'locale':'en-US'}},{'type':'extraction','dimension':'__time',"
@@ -2011,7 +2015,7 @@ public class DruidAdapterIT {
         + "(week from \"timestamp\")";
 
     final String druidQuery = "{'queryType':'groupBy','dataSource':'foodmart',"
-        + "'granularity':'all','dimensions':[{'type':'extraction',"
+        + "'granularity':{'type':'all'},'dimensions':[{'type':'extraction',"
         + "'dimension':'__time','outputName':'extract_week',"
         + "'extractionFn':{'type':'timeFormat','format':'w','timeZone':'UTC',"
         + "'locale':'en-US'}}],'limitSpec':{'type':'default'},"
@@ -2546,7 +2550,7 @@ public class DruidAdapterIT {
     String sql = "select sum(\"store_sales\") "
             + "filter (where \"the_year\" >= 1997) from \"foodmart\"";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','filter':{'type':'bound','dimension':'the_year','lower':'1997',"
+            + "'granularity':{'type':'all'},'filter':{'type':'bound','dimension':'the_year','lower':'1997',"
             + "'lowerStrict':false,'ordering':'numeric'},'aggregations':[{'type':'doubleSum','name'"
             + ":'EXPR$0','fieldName':'store_sales'}],'intervals':['1900-01-09T00:00:00.000Z/2992-01"
             + "-10T00:00:00.000Z'],'context':{'skipEmptyBuckets':true}}";
@@ -2562,7 +2566,7 @@ public class DruidAdapterIT {
     // select sum("store_sales") from "foodmart"
     String sql = "select sum(\"store_sales\") filter (where 1 = 1) from \"foodmart\"";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','aggregations':[{'type':'doubleSum','name':'EXPR$0','fieldName':"
+            + "'granularity':{'type':'all'},'aggregations':[{'type':'doubleSum','name':'EXPR$0','fieldName':"
             + "'store_sales'}],'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
             + "'context':{'skipEmptyBuckets':true}}";
 
@@ -2579,7 +2583,7 @@ public class DruidAdapterIT {
     String sql = "select sum(\"store_sales\") filter (where 1 = 1), "
             + "sum(\"store_cost\") from \"foodmart\"";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','aggregations':[{'type':'doubleSum','name':'EXPR$0','fieldName':"
+            + "'granularity':{'type':'all'},'aggregations':[{'type':'doubleSum','name':'EXPR$0','fieldName':"
             + "'store_sales'},{'type':'doubleSum','name':'EXPR$1','fieldName':'store_cost'}],"
             + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
             + "'context':{'skipEmptyBuckets':true}}";
@@ -2599,7 +2603,7 @@ public class DruidAdapterIT {
             + "sum(\"store_cost\") filter (where \"store_state\" = 'CA') "
             + "from \"foodmart\"";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','aggregations':[{'type':'doubleSum','name':'EXPR$0','fieldName'"
+            + "'granularity':{'type':'all'},'aggregations':[{'type':'doubleSum','name':'EXPR$0','fieldName'"
             + ":'store_sales'},{'type':'filtered','filter':{'type':'selector','dimension':"
             + "'store_state','value':'CA'},'aggregator':{'type':'doubleSum','name':'EXPR$1',"
             + "'fieldName':'store_cost'}}],'intervals':"
@@ -2619,7 +2623,7 @@ public class DruidAdapterIT {
     String sql = "select sum(\"store_sales\") filter (where 1 = 1), sum(\"store_cost\") "
             + "from \"foodmart\" where \"store_city\" = 'Seattle'";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','filter':{'type':'selector','dimension':'store_city',"
+            + "'granularity':{'type':'all'},'filter':{'type':'selector','dimension':'store_city',"
             + "'value':'Seattle'},'aggregations':[{'type':'doubleSum','name':'EXPR$0',"
             + "'fieldName':'store_sales'},{'type':'doubleSum','name':'EXPR$1',"
             + "'fieldName':'store_cost'}],'intervals':"
@@ -2670,7 +2674,7 @@ public class DruidAdapterIT {
     String sql = "select sum(\"store_sales\") filter (where \"store_city\" = 'Seattle') "
             + "from \"foodmart\" where \"store_city\" = 'Seattle'";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','filter':{'type':'selector','dimension':'store_city','value':"
+            + "'granularity':{'type':'all'},'filter':{'type':'selector','dimension':'store_city','value':"
             + "'Seattle'},'aggregations':[{'type':'doubleSum','name':'EXPR$0','fieldName':"
             + "'store_sales'}],'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
             + "'context':{'skipEmptyBuckets':true}}";
@@ -2688,7 +2692,7 @@ public class DruidAdapterIT {
     String sql = "select sum(\"store_sales\") filter (where \"store_state\" = 'CA'), "
             + "sum(\"store_cost\") from \"foodmart\"";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','aggregations':[{'type':'filtered','filter':{'type':'selector',"
+            + "'granularity':{'type':'all'},'aggregations':[{'type':'filtered','filter':{'type':'selector',"
             + "'dimension':'store_state','value':'CA'},'aggregator':{'type':'doubleSum','name':"
             + "'EXPR$0','fieldName':'store_sales'}},{'type':'doubleSum','name':'EXPR$1','fieldName'"
             + ":'store_cost'}],'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
@@ -2706,7 +2710,7 @@ public class DruidAdapterIT {
     String sql = "select sum(\"store_sales\") filter (where \"store_state\" = 'CA'), "
             + "sum(\"store_cost\") from \"foodmart\" where \"the_year\" >= 1997";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','filter':{'type':'bound','dimension':'the_year','lower':'1997',"
+            + "'granularity':{'type':'all'},'filter':{'type':'bound','dimension':'the_year','lower':'1997',"
             + "'lowerStrict':false,'ordering':'numeric'},'aggregations':[{'type':'filtered',"
             + "'filter':{'type':'selector','dimension':'store_state','value':'CA'},'aggregator':{"
             + "'type':'doubleSum','name':'EXPR$0','fieldName':'store_sales'}},{'type':'doubleSum',"
@@ -2727,7 +2731,7 @@ public class DruidAdapterIT {
             + "sum(\"store_sales\") filter (where \"store_state\" = 'WA') "
             + "from \"foodmart\"";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','filter':{'type':'or','fields':[{'type':'selector','dimension':"
+            + "'granularity':{'type':'all'},'filter':{'type':'or','fields':[{'type':'selector','dimension':"
             + "'store_state','value':'CA'},{'type':'selector','dimension':'store_state',"
             + "'value':'WA'}]},'aggregations':[{'type':'filtered','filter':{'type':'selector',"
             + "'dimension':'store_state','value':'CA'},'aggregator':{'type':'doubleSum','name':"
@@ -2752,7 +2756,7 @@ public class DruidAdapterIT {
             + "sum(\"store_sales\") filter (where \"store_state\" = 'WA') "
             + "from \"foodmart\" where \"brand_name\" = 'Super'";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','filter':{'type':'and','fields':[{'type':'or','fields':[{'type':"
+            + "'granularity':{'type':'all'},'filter':{'type':'and','fields':[{'type':'or','fields':[{'type':"
             + "'selector','dimension':'store_state','value':'CA'},{'type':'selector','dimension':"
             + "'store_state','value':'WA'}]},{'type':'selector','dimension':'brand_name','value':"
             + "'Super'}]},'aggregations':[{'type':'filtered','filter':{'type':'selector',"
@@ -2784,7 +2788,7 @@ public class DruidAdapterIT {
     // Aggregates should lose reference to any filter clause
     String expectedAggregateExplain = "aggs=[[SUM($0), SUM($2)]]";
     String expectedQuery = "{'queryType':'timeseries','dataSource':'foodmart','descending':false,"
-            + "'granularity':'all','filter':{'type':'and','fields':[{'type':'selector','dimension':"
+            + "'granularity':{'type':'all'},'filter':{'type':'and','fields':[{'type':'selector','dimension':"
             + "'store_state','value':'CA'},{'type':'selector','dimension':'brand_name','value':"
             + "'Super'}]},'aggregations':[{'type':'doubleSum','name':'EXPR$0','fieldName':"
             + "'store_sales'},{'type':'doubleSum','name':'EXPR$1','fieldName':'store_cost'}],"
@@ -3139,7 +3143,7 @@ public class DruidAdapterIT {
             // customer_id_ts. The thetaSketch aggregation is used to compute the count distinct.
             .queryContains(
                     druidChecker("{'queryType':'timeseries','dataSource':"
-                            + "'foodmart','descending':false,'granularity':'all','aggregations':[{'type':"
+                            + "'foodmart','descending':false,'granularity':{'type':'all'},'aggregations':[{'type':"
                             + "'thetaSketch','name':'EXPR$0','fieldName':'customer_id_ts'}],"
                             + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
                             + "'context':{'skipEmptyBuckets':true}}"))
@@ -3264,7 +3268,7 @@ public class DruidAdapterIT {
             + "from \"foodmart\" group by \"customer_id\"")
             .queryContains(
                     druidChecker("{'queryType':'groupBy','dataSource':'foodmart',"
-                            + "'granularity':'all','dimensions':[{'type':'default','dimension':"
+                            + "'granularity':{'type':'all'},'dimensions':[{'type':'default','dimension':"
                             + "'customer_id'}],'limitSpec':{'type':'default'},'aggregations':[{"
                             + "'type':'cardinality','name':'EXPR$0','fieldNames':['the_month']}],"
                             + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z']}"));
@@ -3284,7 +3288,7 @@ public class DruidAdapterIT {
                 + "'columns':['__time','channel','cityName','comment','countryIsoCode','countryName',"
                 + "'isAnonymous','isMinor','isNew','isRobot','isUnpatrolled','metroCode',"
                 + "'namespace','page','regionIsoCode','regionName','count','added',"
-                + "'deleted','delta'],'granularity':'all',"
+                + "'deleted','delta'],'granularity':{'type':'all'},"
                 + "'resultFormat':'compactedList','limit':5"));
   }
 
@@ -3305,7 +3309,7 @@ public class DruidAdapterIT {
         + "  DruidQuery(table=[[foodmart, foodmart]], intervals=[[1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z]], "
         + "filter=[=(CAST($1):DOUBLE, 1016.0)], groups=[{}], aggs=[[SUM($91)]])";
     final String druidQuery =
-        "{'queryType':'timeseries','dataSource':'foodmart','descending':false,'granularity':'all',"
+        "{'queryType':'timeseries','dataSource':'foodmart','descending':false,'granularity':{'type':'all'},"
             + "'filter':{'type':'bound','dimension':'product_id','lower':'1016.0',"
             + "'lowerStrict':false,'upper':'1016.0','upperStrict':false,'ordering':'numeric'},"
             + "'aggregations':[{'type':'doubleSum','name':'A','fieldName':'store_cost'}],"
@@ -3332,7 +3336,7 @@ public class DruidAdapterIT {
         + "  DruidQuery(table=[[foodmart, foodmart]], intervals=[[1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z]], "
         + "filter=[<>(CAST($1):DOUBLE, 1016.0)], groups=[{}], aggs=[[SUM($91)]])";
     final String druidQuery =
-        "{'queryType':'timeseries','dataSource':'foodmart','descending':false,'granularity':'all',"
+        "{'queryType':'timeseries','dataSource':'foodmart','descending':false,'granularity':{'type':'all'},"
             + "'filter':{'type':'or','fields':[{'type':'bound','dimension':'product_id','lower':'1016.0',"
             + "'lowerStrict':true,'ordering':'numeric'},{'type':'bound','dimension':'product_id',"
             + "'upper':'1016.0','upperStrict':true,'ordering':'numeric'}]},"
@@ -3357,7 +3361,7 @@ public class DruidAdapterIT {
         + "from \"foodmart\" "
         + "where \"product_id\" is null";
     final String druidQuery =
-        "{'queryType':'timeseries','dataSource':'foodmart','descending':false,'granularity':'all',"
+        "{'queryType':'timeseries','dataSource':'foodmart','descending':false,'granularity':{'type':'all'},"
             + "'filter':{'type':'selector','dimension':'product_id','value':null},"
             + "'aggregations':[{'type':'count','name':'C'}],"
             + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
@@ -3373,7 +3377,7 @@ public class DruidAdapterIT {
         + "from \"foodmart\" "
         + "where \"product_id\" is not null";
     final String druidQuery =
-        "{'queryType':'timeseries','dataSource':'foodmart','descending':false,'granularity':'all',"
+        "{'queryType':'timeseries','dataSource':'foodmart','descending':false,'granularity':{'type':'all'},"
             + "'filter':{'type':'not','field':{'type':'selector','dimension':'product_id','value':null}},"
             + "'aggregations':[{'type':'count','name':'C'}],"
             + "'intervals':['1900-01-09T00:00:00.000Z/2992-01-10T00:00:00.000Z'],"
@@ -3395,11 +3399,12 @@ public class DruidAdapterIT {
         + "\"lowerStrict\":false,\"ordering\":\"lexicographic\","
         + "\"extractionFn\":{\"type\":\"timeFormat\",\"format\":\"yyyy-MM-dd";
     final String druidQueryPart2 = "HH:mm:ss.SSS";
-    final String druidQueryPart3 = ",\"granularity\":\"month\",\"timeZone\":\"UTC\","
+    final String druidQueryPart3 = ",\"granularity\":{\"type\":\"period\",\"period\":\"P1M\","
+        + "\"timeZone\":\"UTC\"},\"timeZone\":\"UTC\","
         + "\"locale\":\"en-US\"}},{\"type\":\"bound\",\"dimension\":\"__time\""
         + ",\"upper\":\"1997-03-01T00:00:00.000Z\",\"upperStrict\":false,"
         + "\"ordering\":\"lexicographic\",\"extractionFn\":{\"type\":\"timeFormat\"";
-    final String druidQueryPart4 = "\"columns\":[\"__time\"],\"granularity\":\"all\"";
+    final String druidQueryPart4 = "\"columns\":[\"__time\"],\"granularity\":{\"type\":\"all\"}";
 
     sql(sql, FOODMART)
         .queryContains(
@@ -3415,14 +3420,15 @@ public class DruidAdapterIT {
         + "\"lower\":\"1997-05-01T00:00:00.000Z\",\"lowerStrict\":false,"
         + "\"ordering\":\"lexicographic\",\"extractionFn\":{\"type\":\"timeFormat\","
         + "\"format\":\"yyyy-MM-dd";
-    final String druidQueryPart2 = "\"granularity\":\"month\",\"timeZone\":\"UTC\","
-        + "\"locale\":\"en-US\"}},\"columns\":[\"__time\"],\"granularity\":\"all\"";
+    final String druidQueryPart2 = "\"granularity\":{\"type\":\"period\",\"period\":\"P1M\","
+        + "\"timeZone\":\"UTC\"},\"timeZone\":\"UTC\","
+        + "\"locale\":\"en-US\"}},\"columns\":[\"__time\"],\"granularity\":{\"type\":\"all\"}";
 
     sql(sql, FOODMART).queryContains(druidChecker(druidQueryPart1, druidQueryPart2))
         .returnsOrdered("T=1997-05-01 00:00:00");
   }
 
-  @Test public void testTmeWithFilterOnFloorOnTimeAndCastToTimestamp() {
+  @Test public void testTimeWithFilterOnFloorOnTimeAndCastToTimestamp() {
     final String sql = "Select cast(floor(\"timestamp\" to MONTH) as timestamp) as t from "
         + "\"foodmart\" where floor(\"timestamp\" to MONTH) >= cast('1997-05-01 00:00:00' as TIMESTAMP) order by t"
         + " limit 1";
@@ -3430,57 +3436,61 @@ public class DruidAdapterIT {
         + "\"lower\":\"1997-05-01T00:00:00.000Z\",\"lowerStrict\":false,"
         + "\"ordering\":\"lexicographic\",\"extractionFn\":{\"type\":\"timeFormat\","
         + "\"format\":\"yyyy-MM-dd";
-    final String druidQueryPart2 = "\"granularity\":\"month\",\"timeZone\":\"UTC\","
-        + "\"locale\":\"en-US\"}},\"columns\":[\"__time\"],\"granularity\":\"all\"";
+    final String druidQueryPart2 = "\"granularity\":{\"type\":\"period\",\"period\":\"P1M\","
+        + "\"timeZone\":\"UTC\"},\"timeZone\":\"UTC\","
+        + "\"locale\":\"en-US\"}},\"columns\":[\"__time\"],\"granularity\":{\"type\":\"all\"}";
 
     sql(sql, FOODMART).queryContains(druidChecker(druidQueryPart1, druidQueryPart2))
         .returnsOrdered("T=1997-05-01 00:00:00");
   }
 
-  @Test public void testTmeWithFilterOnFloorOnTimeWithTimezone() {
-    final String sql = "Select cast(floor(\"timestamp\" to MONTH) as timestamp) as t from "
-        + "\"foodmart\" where floor(\"timestamp\" to MONTH) >= cast('1997-05-01 00:00:00'"
+  @Test public void testTimeWithFilterOnFloorOnTimeWithTimezone() {
+    final String sql = "Select cast(\"__time\" as timestamp) as t from "
+        + "\"wikiticker\" where floor(\"__time\" to HOUR) >= cast('2015-09-12 08:00:00'"
         + " as TIMESTAMP) order by t limit 1";
     final String druidQueryPart1 = "filter\":{\"type\":\"bound\",\"dimension\":\"__time\","
-        + "\"lower\":\"1997-05-01T00:00:00.000Z\",\"lowerStrict\":false,"
+        + "\"lower\":\"2015-09-12T08:00:00.000Z\",\"lowerStrict\":false,"
         + "\"ordering\":\"lexicographic\",\"extractionFn\":{\"type\":\"timeFormat\","
         + "\"format\":\"yyyy-MM-dd";
-    final String druidQueryPart2 = "\"granularity\":\"month\",\"timeZone\":\"IST\","
-        + "\"locale\":\"en-US\"}},\"columns\":[\"__time\"],\"granularity\":\"all\"";
+    final String druidQueryPart2 = "\"granularity\":{\"type\":\"period\",\"period\":\"PT1H\","
+        + "\"timeZone\":\"IST\"},\"timeZone\":\"IST\","
+        + "\"locale\":\"en-US\"}},\"columns\":[\"__time\"],\"granularity\":{\"type\":\"all\"}";
 
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .with(ImmutableMap.of("model", WIKI_AUTO2.getPath()))
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "IST")
         .query(sql)
         .runs()
         .queryContains(druidChecker(druidQueryPart1, druidQueryPart2))
-        // NOTE: this return value is not as expected
-        // see https://issues.apache.org/jira/browse/CALCITE-2107
-        .returnsOrdered("T=1997-05-01 05:30:00");
+        .returnsOrdered("T=2015-09-12 08:00:02");
   }
 
-  @Test public void testTmeWithFilterOnFloorOnTimeWithTimezoneConversion() {
-    final String sql = "Select cast(floor(\"timestamp\" to MONTH) as timestamp) as t from "
-        + "\"foodmart\" where floor(\"timestamp\" to MONTH) >= '1997-04-30 18:30:00 UTC' order by t"
-        + " limit 1";
+  @Test public void testTimeWithFilterOnFloorOnTimeWithTimezoneConversion() {
+    final String sql = "Select cast(\"__time\" as timestamp) as t, \"countryName\" as s, "
+        + "count(*) as c from \"wikiticker\" where floor(\"__time\" to HOUR)"
+        + " >= '2015-09-12 08:00:00 IST' group by cast(\"__time\" as timestamp), \"countryName\""
+        + " order by t limit 4";
     final String druidQueryPart1 = "filter\":{\"type\":\"bound\",\"dimension\":\"__time\","
-        + "\"lower\":\"1997-05-01T00:00:00.000Z\",\"lowerStrict\":false,"
+        + "\"lower\":\"2015-09-12T08:00:00.000Z\",\"lowerStrict\":false,"
         + "\"ordering\":\"lexicographic\",\"extractionFn\":{\"type\":\"timeFormat\","
         + "\"format\":\"yyyy-MM-dd";
-    final String druidQueryPart2 = "\"granularity\":\"month\",\"timeZone\":\"IST\","
-        + "\"locale\":\"en-US\"}},\"columns\":[\"__time\"],\"granularity\":\"all\"";
+    final String druidQueryPart2 = "\"granularity\":{\"type\":\"period\",\"period\":\"PT1H\","
+        + "\"timeZone\":\"IST\"},\"timeZone\":\"IST\","
+        + "\"locale\":\"en-US\"}},\"columns\":[\"__time\",\"countryName\"],"
+        + "\"granularity\":{\"type\":\"all\"}";
 
     CalciteAssert.that()
         .enable(enabled())
-        .with(ImmutableMap.of("model", FOODMART.getPath()))
+        .with(ImmutableMap.of("model", WIKI_AUTO2.getPath()))
         .with(CalciteConnectionProperty.TIME_ZONE.camelName(), "IST")
         .query(sql)
         .runs()
         .queryContains(druidChecker(druidQueryPart1, druidQueryPart2))
-        // NOTE: this return value is not as expected
-        // see https://issues.apache.org/jira/browse/CALCITE-2107
-        .returnsOrdered("T=1997-05-01 05:30:00");
+        .returnsOrdered("T=2015-09-12 08:00:02; S=null; C=1",
+            "T=2015-09-12 08:00:04; S=null; C=1",
+            "T=2015-09-12 08:00:05; S=null; C=1",
+            "T=2015-09-12 08:00:07; S=null; C=1");
   }
 
   /** Test case for