You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2019/09/13 06:55:47 UTC

[hive] branch master updated: HIVE-20683: Add the Ability to push Dynamic Between and Bloom filters to Druid (Nishant Bangarwa, reviewed by Slim Bouguerra)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 38190f3  HIVE-20683: Add the Ability to push Dynamic Between and Bloom filters to Druid (Nishant Bangarwa, reviewed by Slim Bouguerra)
38190f3 is described below

commit 38190f3e95752c85188682d8a78d259455e173c2
Author: Nishant Bangarwa <ni...@gmail.com>
AuthorDate: Thu Sep 12 23:54:40 2019 -0700

    HIVE-20683: Add the Ability to push Dynamic Between and Bloom filters to Druid (Nishant Bangarwa, reviewed by Slim Bouguerra)
    
    Close apache/hive#723
---
 .../datasets/druid_table_alltypesorc/load.hive.sql |   10 +-
 druid-handler/pom.xml                              |   51 +-
 .../hadoop/hive/druid/DruidStorageHandler.java     |   30 +
 .../hive/druid/DruidStorageHandlerUtils.java       |  260 +++
 .../druid/serde/DruidGroupByQueryRecordReader.java |    6 +-
 .../hive/druid/serde/DruidQueryRecordReader.java   |  161 +-
 .../druid/serde/DruidScanQueryRecordReader.java    |    6 +-
 .../druid/serde/DruidSelectQueryRecordReader.java  |    6 +-
 .../serde/DruidTimeseriesQueryRecordReader.java    |    6 +-
 .../druid/serde/DruidTopNQueryRecordReader.java    |    6 +-
 .../hadoop/hive/druid/serde/TestDruidSerDe.java    |    8 +-
 itests/qtest-druid/pom.xml                         |    5 +
 .../test/resources/testconfiguration.properties    |    3 +-
 .../hive/ql/metadata/HiveStorageHandler.java       |   13 +
 .../DynamicPartitionPruningOptimization.java       |    4 +-
 .../apache/hadoop/hive/ql/plan/TableScanDesc.java  |   29 +-
 .../queries/clientpositive/druidmini_expressions.q |    4 +-
 .../test/queries/clientpositive/druidmini_joins.q  |    4 +-
 .../druidmini_semijoin_reduction_all_types.q       |  129 ++
 .../clientpositive/druid/druid_timestamptz.q.out   |   88 +
 .../druid/druidmini_expressions.q.out              |   91 +-
 .../druid/druidmini_extractTime.q.out              |    6 +-
 .../clientpositive/druid/druidmini_joins.q.out     |   42 +-
 .../clientpositive/druid/druidmini_masking.q.out   |    4 +
 .../druidmini_semijoin_reduction_all_types.q.out   | 1973 ++++++++++++++++++++
 .../clientpositive/druid/druidmini_test1.q.out     |   12 +-
 .../clientpositive/druid/druidmini_test_ts.q.out   |   12 +-
 .../llap/dynamic_semijoin_reduction_3.q.out        |    2 +
 .../clientpositive/llap/semijoin_hint.q.out        |    2 +
 .../perf/tez/constraints/query1b.q.out             |    1 +
 30 files changed, 2775 insertions(+), 199 deletions(-)

diff --git a/data/files/datasets/druid_table_alltypesorc/load.hive.sql b/data/files/datasets/druid_table_alltypesorc/load.hive.sql
index 0641665..c322e3f 100644
--- a/data/files/datasets/druid_table_alltypesorc/load.hive.sql
+++ b/data/files/datasets/druid_table_alltypesorc/load.hive.sql
@@ -10,7 +10,10 @@ CREATE TABLE alltypesorc1(
     ctimestamp1 TIMESTAMP,
     ctimestamp2 TIMESTAMP,
     cboolean1 BOOLEAN,
-    cboolean2 BOOLEAN)
+    cboolean2 BOOLEAN,
+    cintstring STRING,
+    cfloatstring STRING,
+    cdoublestring STRING)
     STORED AS ORC;
 
 LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/alltypesorc"
@@ -30,5 +33,8 @@ SELECT cast (`ctimestamp1` as timestamp with local time zone) as `__time`,
   cint,
   cbigint,
   cboolean1,
-  cboolean2
+  cboolean2,
+  cast(cint as string) as cintstring,
+  cast(cfloat as string) as cfloatstring,
+  cast(cdouble as string) as cdoublestring
   FROM alltypesorc1 where ctimestamp1 IS NOT NULL;
diff --git a/druid-handler/pom.xml b/druid-handler/pom.xml
index 0454747..3146cbe 100644
--- a/druid-handler/pom.xml
+++ b/druid-handler/pom.xml
@@ -171,30 +171,35 @@
       <version>${druid.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
-      <version>${hadoop.version}</version>
-      <optional>true</optional>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-log4j12</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.guava</groupId>
-          <artifactId>guava</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>commons-logging</groupId>
-          <artifactId>commons-logging</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.fasterxml.jackson.core</groupId>
-          <artifactId>jackson-databind</artifactId>
-        </exclusion>
-      </exclusions>
+        <groupId>org.apache.druid.extensions</groupId>
+        <artifactId>druid-bloom-filter</artifactId>
+        <version>${druid.version}</version>
     </dependency>
+      <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+          <scope>provided</scope>
+          <version>${hadoop.version}</version>
+          <optional>true</optional>
+          <exclusions>
+              <exclusion>
+                  <groupId>org.slf4j</groupId>
+                  <artifactId>slf4j-log4j12</artifactId>
+              </exclusion>
+              <exclusion>
+                  <groupId>com.google.guava</groupId>
+                  <artifactId>guava</artifactId>
+              </exclusion>
+              <exclusion>
+                  <groupId>commons-logging</groupId>
+                  <artifactId>commons-logging</artifactId>
+              </exclusion>
+              <exclusion>
+                  <groupId>com.fasterxml.jackson.core</groupId>
+                  <artifactId>jackson-databind</artifactId>
+              </exclusion>
+          </exclusions>
+      </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <scope>provided</scope>
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
index a14837d..287cf5f 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
@@ -26,6 +26,7 @@ import com.google.common.base.Suppliers;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.druid.data.input.impl.DimensionSchema;
 import org.apache.druid.data.input.impl.DimensionsSpec;
@@ -49,6 +50,8 @@ import org.apache.druid.metadata.storage.mysql.MySQLConnector;
 import org.apache.druid.metadata.storage.mysql.MySQLConnectorConfig;
 import org.apache.druid.metadata.storage.postgresql.PostgreSQLConnector;
 import org.apache.druid.metadata.storage.postgresql.PostgreSQLConnectorConfig;
+import org.apache.druid.query.BaseQuery;
+import org.apache.druid.query.Query;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.segment.IndexSpec;
 import org.apache.druid.segment.indexing.DataSchema;
@@ -84,7 +87,10 @@ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
 import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -964,4 +970,28 @@ import static org.apache.hadoop.hive.druid.DruidStorageHandlerUtils.JSON_MAPPER;
       return null;
     }
   }
+
+  @Override public Map<String, String> getOperatorDescProperties(OperatorDesc operatorDesc,
+      Map<String, String> initialProps) {
+    if (operatorDesc instanceof TableScanDesc) {
+      TableScanDesc tableScanDesc = (TableScanDesc) operatorDesc;
+      ExprNodeGenericFuncDesc filterExpr = tableScanDesc.getFilterExpr();
+      String druidQuery = initialProps.get(Constants.DRUID_QUERY_JSON);
+
+      if (filterExpr != null && druidQuery != null) {
+        try {
+          Query query = DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, BaseQuery.class);
+          Query queryWithDynamicFilters = DruidStorageHandlerUtils.addDynamicFilters(query, filterExpr, conf, false);
+          Map<String, String> props = Maps.newHashMap(initialProps);
+          props.put(Constants.DRUID_QUERY_JSON,
+              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(queryWithDynamicFilters));
+          return props;
+        } catch (IOException e) {
+          LOG.error("Exception while deserializing druid query. Explain plan may not have final druid query", e);
+        }
+      }
+    }
+    // Case when we do not have any additional info to add.
+    return initialProps;
+  }
 }
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
index 1fab1e0..430cc34 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
@@ -25,13 +25,17 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Interner;
 import com.google.common.collect.Interners;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Ordering;
+import org.apache.calcite.adapter.druid.DruidQuery;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.druid.data.input.impl.DimensionSchema;
 import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.guice.BloomFilterSerializersModule;
 import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.JodaUtils;
 import org.apache.druid.java.util.common.MapUtils;
@@ -50,6 +54,7 @@ import org.apache.druid.metadata.MetadataStorageTablesConfig;
 import org.apache.druid.metadata.SQLMetadataConnector;
 import org.apache.druid.metadata.storage.mysql.MySQLConnector;
 import org.apache.druid.query.DruidProcessingConfig;
+import org.apache.druid.query.Druids;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
 import org.apache.druid.query.aggregation.FloatSumAggregatorFactory;
@@ -63,12 +68,29 @@ import org.apache.druid.query.expression.TimestampFormatExprMacro;
 import org.apache.druid.query.expression.TimestampParseExprMacro;
 import org.apache.druid.query.expression.TimestampShiftExprMacro;
 import org.apache.druid.query.expression.TrimExprMacro;
+import org.apache.druid.query.filter.AndDimFilter;
+import org.apache.druid.query.filter.BloomDimFilter;
+import org.apache.druid.query.filter.BloomKFilter;
+import org.apache.druid.query.filter.BloomKFilterHolder;
+import org.apache.druid.query.filter.BoundDimFilter;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.filter.OrDimFilter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.ordering.StringComparator;
+import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.scan.ScanQuery;
+import org.apache.druid.query.select.SelectQuery;
 import org.apache.druid.query.select.SelectQueryConfig;
 import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.topn.TopNQuery;
+import org.apache.druid.query.topn.TopNQueryBuilder;
 import org.apache.druid.segment.IndexIO;
 import org.apache.druid.segment.IndexMergerV9;
 import org.apache.druid.segment.IndexSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.segment.data.BitmapSerdeFactory;
 import org.apache.druid.segment.data.ConciseBitmapSerdeFactory;
 import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
@@ -76,6 +98,7 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec;
 import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
 import org.apache.druid.segment.loading.DataSegmentPusher;
 import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
 import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
 import org.apache.druid.storage.hdfs.HdfsDataSegmentPusher;
 import org.apache.druid.storage.hdfs.HdfsDataSegmentPusherConfig;
@@ -97,11 +120,30 @@ import org.apache.hadoop.hive.druid.conf.DruidConstants;
 import org.apache.hadoop.hive.druid.json.AvroParseSpec;
 import org.apache.hadoop.hive.druid.json.AvroStreamInputRowParser;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.exec.ExprNodeDynamicValueEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.udf.UDFToDouble;
+import org.apache.hadoop.hive.ql.udf.UDFToFloat;
+import org.apache.hadoop.hive.ql.udf.UDFToLong;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFInBloomFilter;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToString;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.util.StringUtils;
@@ -131,11 +173,14 @@ import java.net.InetAddress;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -209,6 +254,11 @@ public final class DruidStorageHandlerUtils {
     SMILE_MAPPER.registerSubtypes(new NamedType(AvroParseSpec.class, "avro"));
     JSON_MAPPER.registerSubtypes(new NamedType(AvroStreamInputRowParser.class, "avro_stream"));
     SMILE_MAPPER.registerSubtypes(new NamedType(AvroStreamInputRowParser.class, "avro_stream"));
+    // Register Bloom Filter Serializers
+    BloomFilterSerializersModule bloomFilterSerializersModule = new BloomFilterSerializersModule();
+    JSON_MAPPER.registerModule(bloomFilterSerializersModule);
+    SMILE_MAPPER.registerModule(bloomFilterSerializersModule);
+
     // set the timezone of the object mapper
     // THIS IS NOT WORKING workaround is to set it as part of java opts -Duser.timezone="UTC"
     JSON_MAPPER.setTimeZone(TimeZone.getTimeZone("UTC"));
@@ -894,4 +944,214 @@ public final class DruidStorageHandlerUtils {
     ImmutableList<AggregatorFactory> aggregatorFactories = aggregatorFactoryBuilder.build();
     return Pair.of(dimensions, aggregatorFactories.toArray(new AggregatorFactory[0]));
   }
+
+  // Druid only supports String,Long,Float,Double selectors
+  private static Set<TypeInfo> druidSupportedTypeInfos =
+      ImmutableSet.<TypeInfo>of(TypeInfoFactory.stringTypeInfo, TypeInfoFactory.charTypeInfo,
+          TypeInfoFactory.varcharTypeInfo, TypeInfoFactory.byteTypeInfo, TypeInfoFactory.intTypeInfo,
+          TypeInfoFactory.longTypeInfo, TypeInfoFactory.shortTypeInfo, TypeInfoFactory.doubleTypeInfo);
+
+  private static Set<TypeInfo> stringTypeInfos =
+      ImmutableSet.<TypeInfo>of(TypeInfoFactory.stringTypeInfo, TypeInfoFactory.charTypeInfo,
+          TypeInfoFactory.varcharTypeInfo);
+
+  public static org.apache.druid.query.Query addDynamicFilters(org.apache.druid.query.Query query,
+      ExprNodeGenericFuncDesc filterExpr, Configuration conf, boolean resolveDynamicValues) {
+    List<VirtualColumn> virtualColumns = Arrays.asList(getVirtualColumns(query).getVirtualColumns());
+    org.apache.druid.query.Query rv = query;
+    DimFilter joinReductionFilter = toDruidFilter(filterExpr, conf, virtualColumns, resolveDynamicValues);
+    if (joinReductionFilter != null) {
+      String type = query.getType();
+      DimFilter filter = new AndDimFilter(joinReductionFilter, query.getFilter());
+      switch (type) {
+      case org.apache.druid.query.Query.TIMESERIES:
+        rv = Druids.TimeseriesQueryBuilder.copy((TimeseriesQuery) query).filters(filter)
+            .virtualColumns(VirtualColumns.create(virtualColumns)).build();
+        break;
+      case org.apache.druid.query.Query.TOPN:
+        rv = new TopNQueryBuilder((TopNQuery) query).filters(filter)
+            .virtualColumns(VirtualColumns.create(virtualColumns)).build();
+        break;
+      case org.apache.druid.query.Query.GROUP_BY:
+        rv = new GroupByQuery.Builder((GroupByQuery) query).setDimFilter(filter)
+            .setVirtualColumns(VirtualColumns.create(virtualColumns)).build();
+        break;
+      case org.apache.druid.query.Query.SCAN:
+        rv = ScanQuery.ScanQueryBuilder.copy((ScanQuery) query).filters(filter)
+            .virtualColumns(VirtualColumns.create(virtualColumns)).build();
+        break;
+      case org.apache.druid.query.Query.SELECT:
+        rv = Druids.SelectQueryBuilder.copy((SelectQuery) query).filters(filter)
+            .virtualColumns(VirtualColumns.create(virtualColumns)).build();
+        break;
+      default:
+        throw new UnsupportedOperationException("Unsupported Query type " + type);
+      }
+    }
+    return rv;
+  }
+
+  @Nullable private static DimFilter toDruidFilter(ExprNodeDesc filterExpr, Configuration configuration,
+      List<VirtualColumn> virtualColumns, boolean resolveDynamicValues) {
+    if (filterExpr == null) {
+      return null;
+    }
+    Class<? extends GenericUDF> genericUDFClass = getGenericUDFClassFromExprDesc(filterExpr);
+    if (FunctionRegistry.isOpAnd(filterExpr)) {
+      Iterator<ExprNodeDesc> iterator = filterExpr.getChildren().iterator();
+      List<DimFilter> delegates = Lists.newArrayList();
+      while (iterator.hasNext()) {
+        DimFilter filter = toDruidFilter(iterator.next(), configuration, virtualColumns, resolveDynamicValues);
+        if (filter != null) {
+          delegates.add(filter);
+        }
+      }
+      if (!delegates.isEmpty()) {
+        return new AndDimFilter(delegates);
+      }
+    }
+    if (FunctionRegistry.isOpOr(filterExpr)) {
+      Iterator<ExprNodeDesc> iterator = filterExpr.getChildren().iterator();
+      List<DimFilter> delegates = Lists.newArrayList();
+      while (iterator.hasNext()) {
+        DimFilter filter = toDruidFilter(iterator.next(), configuration, virtualColumns, resolveDynamicValues);
+        if (filter != null) {
+          delegates.add(filter);
+        }
+      }
+      if (!delegates.isEmpty()) {
+        return new OrDimFilter(delegates);
+      }
+    } else if (GenericUDFBetween.class == genericUDFClass) {
+      List<ExprNodeDesc> child = filterExpr.getChildren();
+      String col = extractColName(child.get(1), virtualColumns);
+      if (col != null) {
+        try {
+          StringComparator comparator = stringTypeInfos
+              .contains(child.get(1).getTypeInfo()) ? StringComparators.LEXICOGRAPHIC : StringComparators.NUMERIC;
+          String lower = evaluate(child.get(2), configuration, resolveDynamicValues);
+          String upper = evaluate(child.get(3), configuration, resolveDynamicValues);
+          return new BoundDimFilter(col, lower, upper, false, false, null, null, comparator);
+
+        } catch (HiveException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    } else if (GenericUDFInBloomFilter.class == genericUDFClass) {
+      List<ExprNodeDesc> child = filterExpr.getChildren();
+      String col = extractColName(child.get(0), virtualColumns);
+      if (col != null) {
+        try {
+          BloomKFilter bloomFilter = evaluateBloomFilter(child.get(1), configuration, resolveDynamicValues);
+          return new BloomDimFilter(col, BloomKFilterHolder.fromBloomKFilter(bloomFilter), null);
+        } catch (HiveException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+    return null;
+  }
+
+  private static String evaluate(ExprNodeDesc desc, Configuration configuration, boolean resolveDynamicValue)
+      throws HiveException {
+    ExprNodeEvaluator exprNodeEvaluator = ExprNodeEvaluatorFactory.get(desc, configuration);
+    if (exprNodeEvaluator instanceof ExprNodeDynamicValueEvaluator && !resolveDynamicValue) {
+      return desc.getExprStringForExplain();
+    } else {
+      return exprNodeEvaluator.evaluate(null).toString();
+    }
+  }
+
+  private static BloomKFilter evaluateBloomFilter(ExprNodeDesc desc, Configuration configuration,
+      boolean resolveDynamicValue) throws HiveException, IOException {
+    if (!resolveDynamicValue) {
+      // return a dummy bloom filter for explain
+      return new BloomKFilter(1);
+    } else {
+      BytesWritable bw = (BytesWritable) ExprNodeEvaluatorFactory.get(desc, configuration).evaluate(null);
+      return BloomKFilter.deserialize(ByteBuffer.wrap(bw.getBytes()));
+    }
+  }
+
+  @Nullable public static String extractColName(ExprNodeDesc expr, List<VirtualColumn> virtualColumns) {
+    if (!druidSupportedTypeInfos.contains(expr.getTypeInfo())) {
+      // This column type is currently not supported in druid.(e.g boolean)
+      // We cannot pass the bloom filter to druid since bloom filter tests for exact object bytes.
+      return null;
+    }
+    if (expr instanceof ExprNodeColumnDesc) {
+      return ((ExprNodeColumnDesc) expr).getColumn();
+    }
+
+    ExprNodeGenericFuncDesc funcDesc = null;
+    if (expr instanceof ExprNodeGenericFuncDesc) {
+      funcDesc = (ExprNodeGenericFuncDesc) expr;
+    }
+    if (null == funcDesc) {
+      return null;
+    }
+    GenericUDF udf = funcDesc.getGenericUDF();
+    // bail out if its not a simple cast expression.
+    if (funcDesc.getChildren().size() == 1 && funcDesc.getChildren().get(0) instanceof ExprNodeColumnDesc) {
+      return null;
+    }
+    String columnName = ((ExprNodeColumnDesc) (funcDesc.getChildren().get(0))).getColumn();
+    ValueType targetType = null;
+    if (udf instanceof GenericUDFBridge) {
+      Class<? extends UDF> udfClass = ((GenericUDFBridge) udf).getUdfClass();
+      if (udfClass.equals(UDFToDouble.class)) {
+        targetType = ValueType.DOUBLE;
+      } else if (udfClass.equals(UDFToFloat.class)) {
+        targetType = ValueType.FLOAT;
+      } else if (udfClass.equals(UDFToLong.class)) {
+        targetType = ValueType.LONG;
+      }
+    } else if (udf instanceof GenericUDFToString) {
+      targetType = ValueType.STRING;
+    }
+
+    if (targetType == null) {
+      return null;
+    }
+    String virtualColumnExpr = DruidQuery.format("CAST(%s, '%s')", columnName, targetType.toString());
+    for (VirtualColumn column : virtualColumns) {
+      if (column instanceof ExpressionVirtualColumn && ((ExpressionVirtualColumn) column).getExpression()
+          .equals(virtualColumnExpr)) {
+        // Found an existing virtual column with same expression, no need to add another virtual column
+        return column.getOutputName();
+      }
+    }
+    Set<String> usedColumnNames = virtualColumns.stream().map(col -> col.getOutputName()).collect(Collectors.toSet());
+    final String name = SqlValidatorUtil.uniquify("vc", usedColumnNames, SqlValidatorUtil.EXPR_SUGGESTER);
+    ExpressionVirtualColumn expressionVirtualColumn =
+        new ExpressionVirtualColumn(name, virtualColumnExpr, targetType, ExprMacroTable.nil());
+    virtualColumns.add(expressionVirtualColumn);
+    return name;
+  }
+
+  public static VirtualColumns getVirtualColumns(org.apache.druid.query.Query query) {
+    String type = query.getType();
+    switch (type) {
+    case org.apache.druid.query.Query.TIMESERIES:
+      return ((TimeseriesQuery) query).getVirtualColumns();
+    case org.apache.druid.query.Query.TOPN:
+      return ((TopNQuery) query).getVirtualColumns();
+    case org.apache.druid.query.Query.GROUP_BY:
+      return ((GroupByQuery) query).getVirtualColumns();
+    case org.apache.druid.query.Query.SCAN:
+      return ((ScanQuery) query).getVirtualColumns();
+    case org.apache.druid.query.Query.SELECT:
+      return ((SelectQuery) query).getVirtualColumns();
+    default:
+      throw new UnsupportedOperationException("Unsupported Query type " + query);
+    }
+  }
+
+  @Nullable private static Class<? extends GenericUDF> getGenericUDFClassFromExprDesc(ExprNodeDesc desc) {
+    if (!(desc instanceof ExprNodeGenericFuncDesc)) {
+      return null;
+    }
+    ExprNodeGenericFuncDesc genericFuncDesc = (ExprNodeGenericFuncDesc) desc;
+    return genericFuncDesc.getGenericUDF().getClass();
+  }
 }
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
index 038b1f6..4def386 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
@@ -45,8 +45,8 @@ public class DruidGroupByQueryRecordReader extends DruidQueryRecordReader<Row> {
   @Override public boolean nextKeyValue() {
     // Results
 
-    if (queryResultsIterator.hasNext()) {
-      final Row row = queryResultsIterator.next();
+    if (getQueryResultsIterator().hasNext()) {
+      final Row row = getQueryResultsIterator().next();
       // currently druid supports only MapBasedRow as Jackson SerDe so it should safe to cast without check
       currentRow = (MapBasedRow) row;
       currentEvent = currentRow.getEvent();
@@ -87,7 +87,7 @@ public class DruidGroupByQueryRecordReader extends DruidQueryRecordReader<Row> {
   }
 
   @Override public float getProgress() throws IOException {
-    return queryResultsIterator.hasNext() ? 0 : 1;
+    return getQueryResultsIterator().hasNext() ? 0 : 1;
   }
 
 }
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
index edbea22..4142e48 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
@@ -36,6 +36,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.druid.DruidStorageHandler;
 import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
 import org.apache.hadoop.hive.druid.io.HiveDruidSplit;
+import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
@@ -56,7 +59,7 @@ import java.util.concurrent.Future;
  * Base record reader for given a Druid query. This class contains the logic to
  * send the query to the broker and retrieve the results. The transformation to
  * emit records needs to be done by the classes that extend the reader.
- *
+ * <p>
  * The key for each record will be a NullWritable, while the value will be a
  * DruidWritable containing the timestamp as well as all values resulting from
  * the query.
@@ -65,7 +68,12 @@ public abstract class DruidQueryRecordReader<R extends Comparable<R>> extends Re
     implements org.apache.hadoop.mapred.RecordReader<NullWritable, DruidWritable> {
 
   private static final Logger LOG = LoggerFactory.getLogger(DruidQueryRecordReader.class);
-
+  private final Object initLock = new Object();
+  // Smile mapper is used to read query results that are serialized as binary instead of json
+  private ObjectMapper smileMapper;
+  private Configuration conf;
+  private String[] locations;
+  private HttpClient httpClient;
   /**
    * Query that Druid executes.
    */
@@ -74,37 +82,36 @@ public abstract class DruidQueryRecordReader<R extends Comparable<R>> extends Re
   /**
    * Query results as a streaming iterator.
    */
-  JsonParserIterator<R> queryResultsIterator = null;
+  private volatile JsonParserIterator<R> queryResultsIterator = null;
 
-  @Override public void initialize(InputSplit split, TaskAttemptContext context) throws IOException {
-    initialize(split, context.getConfiguration());
+  public JsonParserIterator<R> getQueryResultsIterator() {
+    if (this.queryResultsIterator == null) {
+      synchronized (initLock) {
+        if (this.queryResultsIterator == null) {
+          this.queryResultsIterator = createQueryResultsIterator();
+        }
+      }
+    }
+    return this.queryResultsIterator;
   }
 
-  public void initialize(InputSplit split, ObjectMapper mapper, ObjectMapper smileMapper, HttpClient httpClient)
-      throws IOException {
-    HiveDruidSplit hiveDruidSplit = (HiveDruidSplit) split;
-    Preconditions.checkNotNull(hiveDruidSplit, "input split is null ???");
-    Preconditions.checkNotNull(httpClient, "need Http Client can not be null");
-    ObjectMapper objectMapper = Preconditions.checkNotNull(mapper, "object Mapper can not be null");
-    // Smile mapper is used to read query results that are serialized as binary instead of json
-    // Smile mapper is used to read query results that are serialized as binary instead of json
-    ObjectMapper smileObjectMapper = Preconditions.checkNotNull(smileMapper, "Smile Mapper can not be null");
-    // Create query
-    this.query = objectMapper.readValue(Preconditions.checkNotNull(hiveDruidSplit.getDruidQuery()), Query.class);
-    Preconditions.checkNotNull(query);
-    /*
-      Result type definition used to read the rows, this is query dependent.
-     */
-    JavaType resultsType = getResultTypeDef();
+  public JsonParserIterator<R> createQueryResultsIterator() {
+    JsonParserIterator<R> iterator = null;
+    String filterExprSerialized = conf.get(TableScanDesc.FILTER_EXPR_CONF_STR);
+    if (filterExprSerialized != null) {
+      ExprNodeGenericFuncDesc filterExpr = SerializationUtilities.deserializeExpression(filterExprSerialized);
+      query = DruidStorageHandlerUtils.addDynamicFilters(query, filterExpr, conf, true);
+    }
 
-    final String[] locations = hiveDruidSplit.getLocations();
+    // Result type definition used to read the rows, this is query dependent.
+    JavaType resultsType = getResultTypeDef();
     boolean initialized = false;
     int currentLocationIndex = 0;
     Exception ex = null;
     while (!initialized && currentLocationIndex < locations.length) {
       String address = locations[currentLocationIndex++];
       if (Strings.isNullOrEmpty(address)) {
-        throw new IOException("can not fetch results from empty or null host value");
+        throw new RE("can not fetch results from empty or null host value");
       }
       // Execute query
       LOG.debug("Retrieving data from druid location[{}] using query:[{}] ", address, query);
@@ -112,23 +119,16 @@ public abstract class DruidQueryRecordReader<R extends Comparable<R>> extends Re
         Request request = DruidStorageHandlerUtils.createSmileRequest(address, query);
         Future<InputStream> inputStreamFuture = httpClient.go(request, new InputStreamResponseHandler());
         //noinspection unchecked
-        queryResultsIterator =
-            new JsonParserIterator(smileObjectMapper,
-                resultsType,
-                inputStreamFuture,
-                request.getUrl().toString(),
-                query);
-        queryResultsIterator.init();
+        iterator =
+            new JsonParserIterator(smileMapper, resultsType, inputStreamFuture, request.getUrl().toString(), query);
+        iterator.init();
         initialized = true;
-      } catch (IOException | ExecutionException | InterruptedException e) {
-        if (queryResultsIterator != null) {
+      } catch (Exception e) {
+        if (iterator != null) {
           // We got exception while querying results from this host.
-          queryResultsIterator.close();
+          CloseQuietly.close(iterator);
         }
-        LOG.error("Failure getting results for query[{}] from host[{}] because of [{}]",
-            query,
-            address,
-            e.getMessage());
+        LOG.error("Failure getting results for query[{}] from host[{}] because of [{}]", query, address, e.getMessage());
         if (ex == null) {
           ex = e;
         } else {
@@ -138,19 +138,36 @@ public abstract class DruidQueryRecordReader<R extends Comparable<R>> extends Re
     }
 
     if (!initialized) {
-      throw new RE(ex,
-          "Failure getting results for query[%s] from locations[%s] because of [%s]",
-          query,
-          locations,
+      throw new RE(ex, "Failure getting results for query[%s] from locations[%s] because of [%s]", query, locations,
           Objects.requireNonNull(ex).getMessage());
     }
+    return iterator;
+  }
+
+  @Override public void initialize(InputSplit split, TaskAttemptContext context) throws IOException {
+    initialize(split, context.getConfiguration());
+  }
+
+  public void initialize(InputSplit split, ObjectMapper mapper, ObjectMapper smileMapper, HttpClient httpClient,
+      Configuration conf) throws IOException {
+    this.conf = conf;
+    HiveDruidSplit hiveDruidSplit = (HiveDruidSplit) split;
+    Preconditions.checkNotNull(hiveDruidSplit, "input split is null ???");
+    Preconditions.checkNotNull(httpClient, "need Http Client can not be null");
+    // Smile mapper is used to read query results that are serialized as binary instead of json
+    this.smileMapper = Preconditions.checkNotNull(smileMapper, "Smile Mapper can not be null");
+    // Create query
+    this.query = mapper.readValue(Preconditions.checkNotNull(hiveDruidSplit.getDruidQuery()), Query.class);
+    Preconditions.checkNotNull(query);
+
+    this.locations = hiveDruidSplit.getLocations();
+    this.httpClient = httpClient;
   }
 
   public void initialize(InputSplit split, Configuration conf) throws IOException {
     initialize(split,
         DruidStorageHandlerUtils.JSON_MAPPER,
-        DruidStorageHandlerUtils.SMILE_MAPPER,
-        DruidStorageHandler.getHttpClient());
+        DruidStorageHandlerUtils.SMILE_MAPPER, DruidStorageHandler.getHttpClient(), conf);
   }
 
   protected abstract JavaType getResultTypeDef();
@@ -181,7 +198,9 @@ public abstract class DruidQueryRecordReader<R extends Comparable<R>> extends Re
   @Override public abstract float getProgress() throws IOException;
 
   @Override public void close() {
-    CloseQuietly.close(queryResultsIterator);
+    if (queryResultsIterator != null) {
+      CloseQuietly.close(queryResultsIterator);
+    }
   }
 
   /**
@@ -202,14 +221,17 @@ public abstract class DruidQueryRecordReader<R extends Comparable<R>> extends Re
     private final String url;
 
     /**
-     * @param mapper mapper used to deserialize the stream of data (we use smile factory)
+     * @param mapper  mapper used to deserialize the stream of data (we use smile factory)
      * @param typeRef Type definition of the results objects
-     * @param future Future holding the input stream (the input stream is not owned but it will be closed
-     *               when org.apache.hadoop.hive.druid.serde.DruidQueryRecordReader.JsonParserIterator#close() is called
-     *               or reach the end of the steam)
-     * @param url URL used to fetch the data, used mostly as message with exception stack to identify the faulty stream,
-     *           thus this can be empty string.
-     * @param query Query used to fetch the data, used mostly as message with exception stack, thus can be empty string.
+     * @param future  Future holding the input stream (the input stream is not owned but it will be closed
+     *                when org.apache.hadoop.hive.druid.serde.DruidQueryRecordReader.JsonParserIterator#close() is
+     *                called
+     *                or reach the end of the steam)
+     * @param url     URL used to fetch the data, used mostly as message with exception stack to identify the faulty
+     *                stream,
+     *                thus this can be empty string.
+     * @param query   Query used to fetch the data, used mostly as message with exception stack, thus can be empty
+     *                string.
      */
     JsonParserIterator(ObjectMapper mapper, JavaType typeRef, Future<InputStream> future, String url, Query query) {
       this.typeRef = typeRef;
@@ -246,23 +268,28 @@ public abstract class DruidQueryRecordReader<R extends Comparable<R>> extends Re
       throw new UnsupportedOperationException();
     }
 
-    private void init() throws IOException, ExecutionException, InterruptedException {
+    private void init() {
       if (jp == null) {
-        InputStream is = future.get();
-        if (is == null) {
-          throw new IOException(String.format("query[%s] url[%s] timed out", query, url));
-        } else {
-          jp = mapper.getFactory().createParser(is).configure(JsonParser.Feature.AUTO_CLOSE_SOURCE, true);
-        }
-        final JsonToken nextToken = jp.nextToken();
-        if (nextToken == JsonToken.START_OBJECT) {
-          QueryInterruptedException cause = jp.getCodec().readValue(jp, QueryInterruptedException.class);
-          throw new QueryInterruptedException(cause);
-        } else if (nextToken != JsonToken.START_ARRAY) {
-          throw new IAE("Next token wasn't a START_ARRAY, was[%s] from url [%s]", jp.getCurrentToken(), url);
-        } else {
-          jp.nextToken();
-          objectCodec = jp.getCodec();
+        try {
+          InputStream is = future.get();
+          if (is == null) {
+            throw new IOException(String.format("query[%s] url[%s] timed out", query, url));
+          } else {
+            jp = mapper.getFactory().createParser(is).configure(JsonParser.Feature.AUTO_CLOSE_SOURCE, true);
+          }
+          final JsonToken nextToken = jp.nextToken();
+          if (nextToken == JsonToken.START_OBJECT) {
+            QueryInterruptedException cause = jp.getCodec().readValue(jp, QueryInterruptedException.class);
+            throw new QueryInterruptedException(cause);
+          } else if (nextToken != JsonToken.START_ARRAY) {
+            throw new IAE("Next token wasn't a START_ARRAY, was[%s] from url [%s]", jp.getCurrentToken(), url);
+          } else {
+            jp.nextToken();
+            objectCodec = jp.getCodec();
+          }
+
+        } catch (IOException | InterruptedException | ExecutionException e) {
+          throw new RE(e, "Failure getting results for query[%s] url[%s] because of [%s]", query, url, e.getMessage());
         }
       }
     }
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java
index 6a843aa..9afb76a 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java
@@ -48,8 +48,8 @@ public class DruidScanQueryRecordReader extends DruidQueryRecordReader<ScanResul
     if (compactedValues.hasNext()) {
       return true;
     }
-    if (queryResultsIterator.hasNext()) {
-      ScanResultValue current = queryResultsIterator.next();
+    if (getQueryResultsIterator().hasNext()) {
+      ScanResultValue current = getQueryResultsIterator().next();
       //noinspection unchecked
       compactedValues = ((List<List<Object>>) current.getEvents()).iterator();
       return nextKeyValue();
@@ -82,7 +82,7 @@ public class DruidScanQueryRecordReader extends DruidQueryRecordReader<ScanResul
   }
 
   @Override public float getProgress() {
-    return queryResultsIterator.hasNext() || compactedValues.hasNext() ? 0 : 1;
+    return getQueryResultsIterator().hasNext() || compactedValues.hasNext() ? 0 : 1;
   }
 
 }
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
index 0b88fe2..907558f 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
@@ -52,8 +52,8 @@ public class DruidSelectQueryRecordReader extends DruidQueryRecordReader<Result<
     if (values.hasNext()) {
       return true;
     }
-    if (queryResultsIterator.hasNext()) {
-      Result<SelectResultValue> current = queryResultsIterator.next();
+    if (getQueryResultsIterator().hasNext()) {
+      Result<SelectResultValue> current = getQueryResultsIterator().next();
       values = current.getValue().getEvents().iterator();
       return nextKeyValue();
     }
@@ -86,7 +86,7 @@ public class DruidSelectQueryRecordReader extends DruidQueryRecordReader<Result<
   }
 
   @Override public float getProgress() {
-    return queryResultsIterator.hasNext() || values.hasNext() ? 0 : 1;
+    return getQueryResultsIterator().hasNext() || values.hasNext() ? 0 : 1;
   }
 
 }
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
index d8d261b..95af15d 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
@@ -44,8 +44,8 @@ public class DruidTimeseriesQueryRecordReader
 
   @Override
   public boolean nextKeyValue() {
-    if (queryResultsIterator.hasNext()) {
-      current = queryResultsIterator.next();
+    if (getQueryResultsIterator().hasNext()) {
+      current = getQueryResultsIterator().next();
       return true;
     }
     return false;
@@ -83,7 +83,7 @@ public class DruidTimeseriesQueryRecordReader
 
   @Override
   public float getProgress() throws IOException {
-    return queryResultsIterator.hasNext() ? 0 : 1;
+    return getQueryResultsIterator().hasNext() ? 0 : 1;
   }
 
 }
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
index 743858b..57c8b0e 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
@@ -53,8 +53,8 @@ public class DruidTopNQueryRecordReader extends DruidQueryRecordReader<Result<To
     if (values.hasNext()) {
       return true;
     }
-    if (queryResultsIterator.hasNext()) {
-      current = queryResultsIterator.next();
+    if (getQueryResultsIterator().hasNext()) {
+      current = getQueryResultsIterator().next();
       values = current.getValue().getValue().iterator();
       return nextKeyValue();
     }
@@ -90,7 +90,7 @@ public class DruidTopNQueryRecordReader extends DruidQueryRecordReader<Result<To
   }
 
   @Override public float getProgress() {
-    return queryResultsIterator.hasNext() || values.hasNext() ? 0 : 1;
+    return getQueryResultsIterator().hasNext() || values.hasNext() ? 0 : 1;
   }
 
 }
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
index 546eac6..7457697 100644
--- a/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java
@@ -808,8 +808,9 @@ import org.junit.rules.ExpectedException;
     DruidQueryRecordReader<?> reader = DruidQueryBasedInputFormat.getDruidQueryReader(queryType);
 
     final HiveDruidSplit split = new HiveDruidSplit(jsonQuery, new Path("empty"), new String[]{"testing_host"});
-
-    reader.initialize(split, DruidStorageHandlerUtils.JSON_MAPPER, DruidStorageHandlerUtils.SMILE_MAPPER, httpClient);
+    Configuration conf = new Configuration();
+    reader.initialize(split, DruidStorageHandlerUtils.JSON_MAPPER, DruidStorageHandlerUtils.SMILE_MAPPER, httpClient,
+        conf);
     StructObjectInspector oi = (StructObjectInspector) serDe.getObjectInspector();
     List<? extends StructField> fieldRefs = oi.getAllStructFieldRefs();
 
@@ -834,7 +835,8 @@ import org.junit.rules.ExpectedException;
     futureResult.set(new ByteArrayInputStream(resultString));
     when(httpClient.go(anyObject(), any(HttpResponseHandler.class))).thenReturn(futureResult);
     reader = DruidQueryBasedInputFormat.getDruidQueryReader(queryType);
-    reader.initialize(split, DruidStorageHandlerUtils.JSON_MAPPER, DruidStorageHandlerUtils.SMILE_MAPPER, httpClient);
+    reader.initialize(split, DruidStorageHandlerUtils.JSON_MAPPER, DruidStorageHandlerUtils.SMILE_MAPPER, httpClient,
+        conf);
 
     pos = 0;
     while (reader.nextKeyValue()) {
diff --git a/itests/qtest-druid/pom.xml b/itests/qtest-druid/pom.xml
index 3b9c5c3..e693283 100644
--- a/itests/qtest-druid/pom.xml
+++ b/itests/qtest-druid/pom.xml
@@ -106,6 +106,11 @@
         </dependency>
         <dependency>
           <groupId>org.apache.druid.extensions</groupId>
+          <artifactId>druid-bloom-filter</artifactId>
+          <version>${druid.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.druid.extensions</groupId>
           <artifactId>druid-kafka-indexing-service</artifactId>
           <version>${druid.version}</version>
         </dependency>
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 548dfac..3a880c8 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1894,7 +1894,8 @@ druid.query.files=druidmini_test1.q,\
   druidmini_extractTime.q,\
   druidmini_test_alter.q,\
   druidmini_floorTime.q, \
-  druidmini_masking.q
+  druidmini_masking.q, \
+  druidmini_semijoin_reduction_all_types.q
 
 druid.kafka.query.files=druidkafkamini_basic.q, \
   druidkafkamini_avro.q, \
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java
index c010b18..88a8ba9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
@@ -184,4 +185,16 @@ public interface HiveStorageHandler extends Configurable {
   default boolean addDynamicSplitPruningEdge(ExprNodeDesc syntheticFilterPredicate) {
     return false;
   }
+
+  /**
+   * Used to add additional operator specific information from storage handler during DESCRIBE EXTENDED statement.
+   *
+   * @param operatorDesc operatorDesc
+   * @param initialProps Map containing initial operator properties
+   * @return Map<String, String> containing additional operator specific information from storage handler
+   * OR `initialProps` if the storage handler choose to not provide any such information.
+   */
+  default Map<String, String> getOperatorDescProperties(OperatorDesc operatorDesc, Map<String, String> initialProps) {
+    return initialProps;
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
index 374004f..0f95d77 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
@@ -286,8 +286,10 @@ public class DynamicPartitionPruningOptimization implements NodeProcessor {
         andArgs.add(desc.getPredicate());
         andArgs.addAll(newBetweenNodes);
         andArgs.addAll(newBloomFilterNodes);
-        ExprNodeDesc andExpr = ExprNodeGenericFuncDesc.newInstance(
+        ExprNodeGenericFuncDesc andExpr = ExprNodeGenericFuncDesc.newInstance(
             FunctionRegistry.getFunctionInfo("and").getGenericUDF(), andArgs);
+        // Also pass in filter as tableScan filterExpr
+        ts.getConf().setFilterExpr(andExpr);
         desc.setPredicate(andExpr);
       }
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
index 192c04c..d2e22c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
@@ -18,16 +18,9 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.signature.Signature;
@@ -38,6 +31,14 @@ import org.apache.hadoop.hive.ql.plan.Explain.Vectorization;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
 /**
  * Table Scan Descriptor Currently, data is only read from a base source as part
  * of map-reduce framework. So, nothing is stored in the descriptor. But, more
@@ -470,12 +471,18 @@ public class TableScanDesc extends AbstractOperatorDesc implements IStatsGatherD
     return rtn;
   }
 
-  @Override
-  @Explain(displayName = "properties", explainLevels = { Level.DEFAULT, Level.USER, Level.EXTENDED })
-  public Map<String, String> getOpProps() {
+  @Override public Map<String, String> getOpProps() {
     return opProps;
   }
 
+  @Explain(displayName = "properties", explainLevels = { Level.DEFAULT, Level.USER, Level.EXTENDED })
+  public Map<String, String> getOpPropsWithStorageHandlerProps() {
+    HiveStorageHandler storageHandler = tableMetadata.getStorageHandler();
+    return storageHandler == null
+            ? opProps
+            : storageHandler.getOperatorDescProperties(this, opProps);
+  }
+
   public class TableScanOperatorExplainVectorization extends OperatorExplainVectorization {
 
     private final TableScanDesc tableScanDesc;
diff --git a/ql/src/test/queries/clientpositive/druidmini_expressions.q b/ql/src/test/queries/clientpositive/druidmini_expressions.q
index c1c0ff3..59ffb5b 100644
--- a/ql/src/test/queries/clientpositive/druidmini_expressions.q
+++ b/ql/src/test/queries/clientpositive/druidmini_expressions.q
@@ -155,8 +155,8 @@ SELECT SUM((`druid_table_alias`.`cdouble` * `druid_table_alias`.`cdouble`)) AS `
 FROM `default`.`druid_table_alltypesorc` `druid_table_alias`
 GROUP BY CAST(TRUNC(CAST(`druid_table_alias`.`__time` AS TIMESTAMP),'MM') AS DATE);
 
-explain SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2 from druid_table_alltypesorc  order by date_1, date_2 limit 3;
-SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2 from druid_table_alltypesorc  order by date_1, date_2 limit 3;
+explain SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2, cast(`__time` as date) as orig_date, CAST((cdouble / 1000) AS INT) as offset from druid_table_alltypesorc  order by date_1, date_2 limit 3;
+SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2, cast(`__time` as date) as orig_date, CAST((cdouble / 1000) AS INT) as offset from druid_table_alltypesorc  order by date_1, date_2 limit 3;
 
   -- Boolean Values
 -- Expected results of this query are wrong due to https://issues.apache.org/jira/browse/CALCITE-2319
diff --git a/ql/src/test/queries/clientpositive/druidmini_joins.q b/ql/src/test/queries/clientpositive/druidmini_joins.q
index a602887..b6876fe 100644
--- a/ql/src/test/queries/clientpositive/druidmini_joins.q
+++ b/ql/src/test/queries/clientpositive/druidmini_joins.q
@@ -39,7 +39,7 @@ druid_table_with_nulls `tbl1`
   )
   `tbl2`
     ON (`tbl1`.`username` = `tbl2`.`username`)
-GROUP BY `tbl1`.`username`;
+GROUP BY `tbl1`.`username` ORDER BY `sum_double1`;
 
 
 SELECT
@@ -58,4 +58,4 @@ druid_table_with_nulls `tbl1`
   )
   `tbl2`
     ON (`tbl1`.`username` = `tbl2`.`username`)
-GROUP BY `tbl1`.`username`;
\ No newline at end of file
+GROUP BY `tbl1`.`username` ORDER BY `sum_double1`;
\ No newline at end of file
diff --git a/ql/src/test/queries/clientpositive/druidmini_semijoin_reduction_all_types.q b/ql/src/test/queries/clientpositive/druidmini_semijoin_reduction_all_types.q
new file mode 100644
index 0000000..b0bf51c
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druidmini_semijoin_reduction_all_types.q
@@ -0,0 +1,129 @@
+--! qt:dataset:srcpart
+--! qt:dataset:druid_table_alltypesorc
+--! qt:dataset:alltypesorc
+
+set hive.compute.query.using.stats=false;
+set hive.mapred.mode=nonstrict;
+set hive.explain.user=false;
+set hive.optimize.ppd=true;
+set hive.ppd.remove.duplicatefilters=true;
+set hive.tez.dynamic.partition.pruning=true;
+set hive.tez.dynamic.semijoin.reduction=true;
+set hive.optimize.metadataonly=false;
+set hive.optimize.index.filter=true;
+set hive.stats.autogather=true;
+set hive.tez.bigtable.minsize.semijoin.reduction=1;
+set hive.tez.min.bloom.filter.entries=1;
+set hive.stats.fetch.column.stats=true;
+set hive.disable.unsafe.external.table.operations=false;
+set hive.tez.dynamic.semijoin.reduction.for.mapjoin=true;
+
+DROP TABLE IF EXISTS alltypesorc_small;
+CREATE TABLE alltypesorc_small(
+    ctinyint TINYINT,
+    csmallint SMALLINT,
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE,
+    cstring1 STRING,
+    cstring2 STRING,
+    ctimestamp1 TIMESTAMP,
+    cboolean1 BOOLEAN,
+    cboolean2 BOOLEAN)
+    STORED AS ORC;
+Insert into table alltypesorc_small
+Select ctinyint, csmallint, cint, cbigint, cfloat, cdouble, cstring1, cstring2, cast(`__time` as timestamp), cboolean1, cboolean2 from druid_table_alltypesorc where cstring2 like '%a%' and cstring1 like '%a%';
+Select count(*) from alltypesorc_small;
+Select count(*) from druid_table_alltypesorc;
+
+DESCRIBE druid_table_alltypesorc;
+DESCRIBE alltypesorc_small;
+
+-- Test Joins on all column types one by one
+-- String
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1);
+
+-- tinyint
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint);
+
+-- smallint
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint);
+
+-- int
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint);
+
+-- bigint
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint);
+
+-- float
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat);
+
+-- double
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble);
+set hive.disable.unsafe.external.table.operations=true;
+
+-- timestamp
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = druid_table_alltypesorc.`__time`);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp));
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp));
+
+-- boolean
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1);
+
+
+-- Test Casts
+
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cint as string) = druid_table_alltypesorc.cintstring);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cint as string) = druid_table_alltypesorc.cintstring);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cint as string) = druid_table_alltypesorc.cintstring);
+
+
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cdouble as string) = druid_table_alltypesorc.cdoublestring);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cdouble as string) = druid_table_alltypesorc.cdoublestring);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cdouble as string) = druid_table_alltypesorc.cdoublestring);
+
+
+set hive.disable.unsafe.external.table.operations=false;
+EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cfloat as string) = druid_table_alltypesorc.cfloatstring);
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cfloat as string) = druid_table_alltypesorc.cfloatstring);
+set hive.disable.unsafe.external.table.operations=true;
+select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cfloat as string) = druid_table_alltypesorc.cfloatstring);
diff --git a/ql/src/test/results/clientpositive/druid/druid_timestamptz.q.out b/ql/src/test/results/clientpositive/druid/druid_timestamptz.q.out
index 003b4d6..7338559 100644
--- a/ql/src/test/results/clientpositive/druid/druid_timestamptz.q.out
+++ b/ql/src/test/results/clientpositive/druid/druid_timestamptz.q.out
@@ -48,8 +48,12 @@ POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@tstz1_n1
 PREHOOK: query: EXPLAIN select `__time` from tstz1_n0
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select `__time` from tstz1_n0
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -81,8 +85,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:26:34.0 US/Pacific
 PREHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -114,8 +122,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:26:34
 PREHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0 where `__time` >= cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone)
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0 where `__time` >= cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -147,8 +159,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:26:34
 PREHOOK: query: EXPLAIN SELECT EXTRACT(HOUR FROM CAST(`__time` AS timestamp)) FROM tstz1_n0
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT EXTRACT(HOUR FROM CAST(`__time` AS timestamp)) FROM tstz1_n0
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -180,8 +196,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 12
 PREHOOK: query: EXPLAIN SELECT FLOOR(CAST(`__time` AS timestamp) to HOUR) FROM tstz1_n0
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT FLOOR(CAST(`__time` AS timestamp) to HOUR) FROM tstz1_n0
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -213,8 +233,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:00:00
 PREHOOK: query: EXPLAIN SELECT `__time`, max(v) FROM tstz1_n0 GROUP BY `__time`
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT `__time`, max(v) FROM tstz1_n0 GROUP BY `__time`
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -246,8 +270,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:26:34.0 US/Pacific	10
 PREHOOK: query: EXPLAIN select `__time` from tstz1_n1
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n1
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select `__time` from tstz1_n1
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -279,8 +307,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:26:34
 PREHOOK: query: EXPLAIN SELECT EXTRACT(HOUR FROM CAST(`__time` AS timestamp)) FROM tstz1_n1
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n1
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT EXTRACT(HOUR FROM CAST(`__time` AS timestamp)) FROM tstz1_n1
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -312,8 +344,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 12
 PREHOOK: query: EXPLAIN  SELECT FLOOR(CAST(`__time` AS timestamp) to HOUR) FROM tstz1_n1
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n1
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN  SELECT FLOOR(CAST(`__time` AS timestamp) to HOUR) FROM tstz1_n1
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -345,8 +381,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:00:00
 PREHOOK: query: EXPLAIN SELECT `__time`, max(v) FROM tstz1_n1 GROUP BY `__time`
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n1
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT `__time`, max(v) FROM tstz1_n1 GROUP BY `__time`
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -378,8 +418,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:26:34	10
 PREHOOK: query: EXPLAIN select `__time` from tstz1_n0
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select `__time` from tstz1_n0
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -411,8 +455,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 20:26:34.0 UTC
 PREHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -444,8 +492,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 20:26:34
 PREHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0 where `__time` >= cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone)
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0 where `__time` >= cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -477,8 +529,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 20:26:34
 PREHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0 where `__time` >= cast('2016-01-03 20:26:34' as timestamp)
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0 where `__time` >= cast('2016-01-03 20:26:34' as timestamp)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -510,8 +566,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 20:26:34
 PREHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0 where `__time` >= cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone) AND `__time` <= cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone)
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select cast(`__time` as timestamp) from tstz1_n0 where `__time` >= cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone) AND `__time` <= cast('2016-01-03 12:26:34 America/Los_Angeles' as timestamp with local time zone)
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -543,8 +603,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 20:26:34
 PREHOOK: query: EXPLAIN SELECT EXTRACT(HOUR FROM CAST(`__time` AS timestamp)) FROM tstz1_n0
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT EXTRACT(HOUR FROM CAST(`__time` AS timestamp)) FROM tstz1_n0
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -576,8 +640,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 20
 PREHOOK: query: EXPLAIN  SELECT FLOOR(CAST(`__time` AS timestamp) to HOUR) FROM tstz1_n0
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN  SELECT FLOOR(CAST(`__time` AS timestamp) to HOUR) FROM tstz1_n0
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -609,8 +677,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 20:00:00
 PREHOOK: query: EXPLAIN SELECT `__time`, max(v) FROM tstz1_n0 GROUP BY `__time`
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n0
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT `__time`, max(v) FROM tstz1_n0 GROUP BY `__time`
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n0
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -642,8 +714,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 20:26:34.0 UTC	10
 PREHOOK: query: EXPLAIN select `__time` from tstz1_n1
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n1
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN select `__time` from tstz1_n1
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -675,8 +751,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:26:34
 PREHOOK: query: EXPLAIN SELECT EXTRACT(HOUR FROM CAST(`__time` AS timestamp)) FROM tstz1_n1
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n1
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT EXTRACT(HOUR FROM CAST(`__time` AS timestamp)) FROM tstz1_n1
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -708,8 +788,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 12
 PREHOOK: query: EXPLAIN  SELECT FLOOR(CAST(`__time` AS timestamp) to HOUR) FROM tstz1_n1
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n1
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN  SELECT FLOOR(CAST(`__time` AS timestamp) to HOUR) FROM tstz1_n1
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
@@ -741,8 +825,12 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 2016-01-03 12:00:00
 PREHOOK: query: EXPLAIN SELECT `__time`, max(v) FROM tstz1_n1 GROUP BY `__time`
 PREHOOK: type: QUERY
+PREHOOK: Input: default@tstz1_n1
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT `__time`, max(v) FROM tstz1_n1 GROUP BY `__time`
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tstz1_n1
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out b/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
index 902b4c9..e05bc13 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
@@ -10,7 +10,7 @@ SELECT count(*) FROM druid_table_alltypesorc WHERE character_length(CAST(ctinyin
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-837
+2081
 PREHOOK: query: SELECT count(*) FROM druid_table_alltypesorc WHERE character_length(CAST(ctinyint AS STRING)) > 1 AND char_length(CAST(ctinyint AS STRING)) < 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -19,7 +19,7 @@ POSTHOOK: query: SELECT count(*) FROM druid_table_alltypesorc WHERE character_le
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-4838
+7602
 PREHOOK: query: SELECT count(*) FROM druid_table_alltypesorc WHERE power(cfloat, 2) * pow(csmallint, 3) > 1 AND SQRT(ABS(ctinyint)) > 3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -28,7 +28,7 @@ POSTHOOK: query: SELECT count(*) FROM druid_table_alltypesorc WHERE power(cfloat
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-837
+2081
 PREHOOK: query: SELECT  SUM(cfloat + 1), CAST(SUM(cdouble + ctinyint) AS INTEGER), SUM(ctinyint) + 1 , CAST(SUM(csmallint) + SUM(cint) AS DOUBLE), SUM(cint), SUM(cbigint)
 FROM druid_table_alltypesorc WHERE ceil(cfloat) > 0 AND floor(cdouble) * 2 < 1000 OR ln(cdouble) / log10(10) > 0 AND COS(cint) > 0 OR SIN(cdouble) > 1
 PREHOOK: type: QUERY
@@ -39,7 +39,7 @@ FROM druid_table_alltypesorc WHERE ceil(cfloat) > 0 AND floor(cdouble) * 2 < 100
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-19884.646918177605	27373419	14472	8.51628242804E11	851620413654	68151649880
+35881.09400522709	23378637	28946	7.57221208318E11	757217375155	68151649880
 PREHOOK: query: SELECT  SUM(cfloat + 1), CAST(SUM(cdouble + ctinyint) AS INTEGER), SUM(ctinyint) + 1 , CAST(SUM(csmallint) + SUM(cint) AS DOUBLE), SUM(cint), SUM(cbigint)
 FROM druid_table_alltypesorc WHERE ceil(cfloat) > 0 AND floor(cdouble) * 2 < 1000
 PREHOOK: type: QUERY
@@ -50,7 +50,7 @@ FROM druid_table_alltypesorc WHERE ceil(cfloat) > 0 AND floor(cdouble) * 2 < 100
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-54430.269943237305	-3740445	51268	1.31919188502E11	131922984948	92160895030
+78620.21200561523	-9973483	74628	3.54463845353E11	354473882701	92859255924
 PREHOOK: query: SELECT  SUM(cfloat + 1), CAST(SUM(cdouble + ctinyint) AS INTEGER), SUM(ctinyint) + 1 , CAST(SUM(csmallint) + SUM(cint) AS DOUBLE), SUM(cint), SUM(cbigint)
 FROM druid_table_alltypesorc WHERE  ln(cdouble) / log10(10) > 0 AND COS(cint) > 0 OR SIN(cdouble) > 1
 PREHOOK: type: QUERY
@@ -61,7 +61,7 @@ FROM druid_table_alltypesorc WHERE  ln(cdouble) / log10(10) > 0 AND COS(cint) >
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
--2389.517022252083	27640645	-5707	7.19705549994E11	719697428706	13774723379
+-2195.4589961767197	27912403	-6517	5.33299976914E11	533291579903	13774723379
 PREHOOK: query: SELECT  SUM(cfloat + 1), CAST(SUM(cdouble + ctinyint) AS INTEGER), SUM(ctinyint) + 1 , CAST(SUM(csmallint) + SUM(cint) AS DOUBLE), SUM(cint), SUM(cbigint)
 FROM druid_table_alltypesorc WHERE  SIN(cdouble) > 1
 PREHOOK: type: QUERY
@@ -243,9 +243,9 @@ STAGE PLANS:
                     bucket_count -1
                     bucketing_version 2
                     column.name.delimiter ,
-                    columns __time,cstring1,cstring2,cdouble,cfloat,ctinyint,csmallint,cint,cbigint,cboolean1,cboolean2
+                    columns __time,cstring1,cstring2,cdouble,cfloat,ctinyint,csmallint,cint,cbigint,cboolean1,cboolean2,cintstring,cfloatstring,cdoublestring
                     columns.comments 
-                    columns.types timestamp with local time zone:string:string:double:float:tinyint:smallint:int:bigint:boolean:boolean
+                    columns.types timestamp with local time zone:string:string:double:float:tinyint:smallint:int:bigint:boolean:boolean:string:string:string
                     druid.datasource default.druid_table_alltypesorc
                     druid.fieldNames vc
                     druid.fieldTypes int
@@ -259,7 +259,7 @@ STAGE PLANS:
                     numFiles 0
                     numRows 9173
                     rawDataSize 0
-                    serialization.ddl struct druid_table_alltypesorc { timestamp with local time zone __time, string cstring1, string cstring2, double cdouble, float cfloat, byte ctinyint, i16 csmallint, i32 cint, i64 cbigint, bool cboolean1, bool cboolean2}
+                    serialization.ddl struct druid_table_alltypesorc { timestamp with local time zone __time, string cstring1, string cstring2, double cdouble, float cfloat, byte ctinyint, i16 csmallint, i32 cint, i64 cbigint, bool cboolean1, bool cboolean2, string cintstring, string cfloatstring, string cdoublestring}
                     serialization.format 1
                     serialization.lib org.apache.hadoop.hive.druid.serde.DruidSerDe
                     storage_handler org.apache.hadoop.hive.druid.DruidStorageHandler
@@ -275,9 +275,9 @@ STAGE PLANS:
                       bucket_count -1
                       bucketing_version 2
                       column.name.delimiter ,
-                      columns __time,cstring1,cstring2,cdouble,cfloat,ctinyint,csmallint,cint,cbigint,cboolean1,cboolean2
+                      columns __time,cstring1,cstring2,cdouble,cfloat,ctinyint,csmallint,cint,cbigint,cboolean1,cboolean2,cintstring,cfloatstring,cdoublestring
                       columns.comments 
-                      columns.types timestamp with local time zone:string:string:double:float:tinyint:smallint:int:bigint:boolean:boolean
+                      columns.types timestamp with local time zone:string:string:double:float:tinyint:smallint:int:bigint:boolean:boolean:string:string:string
                       druid.datasource default.druid_table_alltypesorc
                       druid.fieldNames vc
                       druid.fieldTypes int
@@ -291,7 +291,7 @@ STAGE PLANS:
                       numFiles 0
                       numRows 9173
                       rawDataSize 0
-                      serialization.ddl struct druid_table_alltypesorc { timestamp with local time zone __time, string cstring1, string cstring2, double cdouble, float cfloat, byte ctinyint, i16 csmallint, i32 cint, i64 cbigint, bool cboolean1, bool cboolean2}
+                      serialization.ddl struct druid_table_alltypesorc { timestamp with local time zone __time, string cstring1, string cstring2, double cdouble, float cfloat, byte ctinyint, i16 csmallint, i32 cint, i64 cbigint, bool cboolean1, bool cboolean2, string cintstring, string cfloatstring, string cdoublestring}
                       serialization.format 1
                       serialization.lib org.apache.hadoop.hive.druid.serde.DruidSerDe
                       storage_handler org.apache.hadoop.hive.druid.DruidStorageHandler
@@ -1088,7 +1088,7 @@ POSTHOOK: query: select count(DISTINCT cstring2), sum(cdouble) FROM druid_table_
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-6078	2.7308662793799996E7
+6078	2.73086627938E7
 PREHOOK: query: select count(distinct cstring2), sum(2 * cdouble) FROM druid_table_alltypesorc GROUP  BY floor_year(`__time`)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -1097,7 +1097,7 @@ POSTHOOK: query: select count(distinct cstring2), sum(2 * cdouble) FROM druid_ta
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-6078	5.461732558759999E7
+6078	5.46173255876E7
 PREHOOK: query: select count(DISTINCT cstring2) FROM druid_table_alltypesorc
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -1188,7 +1188,7 @@ POSTHOOK: query: select count(DISTINCT cstring2), sum(cdouble) FROM druid_table_
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-6078	2.7308662793799996E7
+6078	2.73086627938E7
 PREHOOK: query: select count(distinct cstring2 || '_'|| cstring1), sum(cdouble), min(cint) FROM druid_table_alltypesorc
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -1215,7 +1215,7 @@ POSTHOOK: query: select count(*) from (select `__time` from druid_table_alltypes
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-6105
+9173
 PREHOOK: query: select count(`__time`) from (select `__time` from druid_table_alltypesorc limit 200) as src
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -1794,12 +1794,12 @@ GROUP BY CAST(TRUNC(CAST(`druid_table_alias`.`__time` AS TIMESTAMP),'MM') AS DAT
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-3.83294831503382E14	1969-12-01
-PREHOOK: query: explain SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2 from druid_table_alltypesorc  order by date_1, date_2 limit 3
+1.91216621994122E14	1969-12-01
+PREHOOK: query: explain SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2, cast(`__time` as date) as orig_date, CAST((cdouble / 1000) AS INT) as offset from druid_table_alltypesorc  order by date_1, date_2 limit 3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
 PREHOOK: Output: hdfs://### HDFS PATH ###
-POSTHOOK: query: explain SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2 from druid_table_alltypesorc  order by date_1, date_2 limit 3
+POSTHOOK: query: explain SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2, cast(`__time` as date) as orig_date, CAST((cdouble / 1000) AS INT) as offset from druid_table_alltypesorc  order by date_1, date_2 limit 3
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
@@ -1820,35 +1820,36 @@ STAGE PLANS:
                 TableScan
                   alias: druid_table_alltypesorc
                   properties:
-                    druid.fieldNames vc,vc0
-                    druid.fieldTypes date,date
-                    druid.query.json {"queryType":"scan","dataSource":"default.druid_table_alltypesorc","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"timestamp_shift(timestamp_floor(timestamp_parse(timestamp_format(\"__time\",'yyyy-MM-dd\\u0027T\\u0027HH:mm:ss.SSS\\u0027Z\\u0027','US/Pacific'),'','UTC'),'P1D','','UTC'),'P1D',CAST((\"cdouble\" / 1000), 'LONG'),'UTC')","outputType":"LONG"},{"type":"express [...]
+                    druid.fieldNames vc,vc0,vc1,vc2
+                    druid.fieldTypes date,date,date,int
+                    druid.query.json {"queryType":"scan","dataSource":"default.druid_table_alltypesorc","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"timestamp_shift(timestamp_floor(timestamp_parse(timestamp_format(\"__time\",'yyyy-MM-dd\\u0027T\\u0027HH:mm:ss.SSS\\u0027Z\\u0027','US/Pacific'),'','UTC'),'P1D','','UTC'),'P1D',CAST((\"cdouble\" / 1000), 'LONG'),'UTC')","outputType":"LONG"},{"type":"express [...]
                     druid.query.type scan
-                  Statistics: Num rows: 9173 Data size: 976192 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 9173 Data size: 1499152 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: vc (type: date), vc0 (type: date)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 9173 Data size: 976192 Basic stats: COMPLETE Column stats: NONE
+                    expressions: vc (type: date), vc0 (type: date), vc1 (type: date), vc2 (type: int)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 9173 Data size: 1499152 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: date), _col1 (type: date)
                       sort order: ++
-                      Statistics: Num rows: 9173 Data size: 976192 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 9173 Data size: 1499152 Basic stats: COMPLETE Column stats: NONE
                       TopN Hash Memory Usage: 0.1
+                      value expressions: _col2 (type: date), _col3 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: date), KEY.reducesinkkey1 (type: date)
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 9173 Data size: 976192 Basic stats: COMPLETE Column stats: NONE
+                expressions: KEY.reducesinkkey0 (type: date), KEY.reducesinkkey1 (type: date), VALUE._col0 (type: date), VALUE._col1 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 9173 Data size: 1499152 Basic stats: COMPLETE Column stats: NONE
                 Limit
                   Number of rows: 3
-                  Statistics: Num rows: 3 Data size: 318 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 3 Data size: 489 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 3 Data size: 318 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 3 Data size: 489 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1860,17 +1861,17 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-PREHOOK: query: SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2 from druid_table_alltypesorc  order by date_1, date_2 limit 3
+PREHOOK: query: SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2, cast(`__time` as date) as orig_date, CAST((cdouble / 1000) AS INT) as offset from druid_table_alltypesorc  order by date_1, date_2 limit 3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
 PREHOOK: Output: hdfs://### HDFS PATH ###
-POSTHOOK: query: SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2 from druid_table_alltypesorc  order by date_1, date_2 limit 3
+POSTHOOK: query: SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1,  DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2, cast(`__time` as date) as orig_date, CAST((cdouble / 1000) AS INT) as offset from druid_table_alltypesorc  order by date_1, date_2 limit 3
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1969-02-26	1970-11-04
-1969-03-19	1970-10-14
-1969-11-13	1970-02-17
+1969-12-15	1970-01-16	1969-12-31	-16
+1969-12-15	1970-01-16	1969-12-31	-16
+1969-12-15	1970-01-16	1969-12-31	-16
 PREHOOK: query: -- Boolean Values
 
 
@@ -1913,8 +1914,8 @@ POSTHOOK: query: SELECT cboolean2, count(*) from druid_table_alltypesorc GROUP B
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-false	3148
-true	2957
+false	6214
+true	2959
 PREHOOK: query: -- Expected results of this query are wrong due to https://issues.apache.org/jira/browse/CALCITE-2319
   -- It should get fixed once we upgrade calcite
  SELECT ctinyint > 2, count(*) from druid_table_alltypesorc GROUP BY ctinyint > 2
@@ -1927,8 +1928,8 @@ POSTHOOK: query: -- Expected results of this query are wrong due to https://issu
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-false	2653
-true	3452
+false	4280
+true	4893
 PREHOOK: query: EXPLAIN SELECT ctinyint > 2, count(*) from druid_table_alltypesorc GROUP BY ctinyint > 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -1993,7 +1994,7 @@ POSTHOOK: query: SELECT sum(cfloat) FROM druid_table_alltypesorc WHERE cstring1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
--39590.24724686146
+-39590.24699282646
 PREHOOK: query: EXPLAIN SELECT sum(cfloat) FROM druid_table_alltypesorc WHERE cstring1 != 'en' group by 1.011, 3.40
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -2030,7 +2031,7 @@ POSTHOOK: query: SELECT sum(cfloat) FROM druid_table_alltypesorc WHERE cstring1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
--39590.24724686146
+-39590.24699282646
 PREHOOK: query: EXPLAIN SELECT sum(cint) FROM druid_table_alltypesorc WHERE cfloat= 0.011 group by cfloat
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -2468,7 +2469,7 @@ POSTHOOK: query: select count(cstring1) from (select `cfloat`, `cstring1` from d
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-3036
+6059
 PREHOOK: query: explain select count(cstring1) from (select `cfloat`, `cstring1`, `cint` from druid_table_alltypesorc limit 90000) as src
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -2541,7 +2542,7 @@ POSTHOOK: query: select max(cint * cdouble) from (select `cfloat`, `cstring1`, `
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-8.256991041892764E15
+5.16019886323666E15
 PREHOOK: query: explain select max(cint * cfloat) from (select `cfloat`, `cstring1`, `cint`, `cdouble` from druid_table_alltypesorc limit 90000) as src
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_extractTime.q.out b/ql/src/test/results/clientpositive/druid/druidmini_extractTime.q.out
index 281cb53..475a40e 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_extractTime.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_extractTime.q.out
@@ -725,8 +725,8 @@ POSTHOOK: query: SELECT CAST(`__time` AS TIMESTAMP) AS `x_time`, SUM(cfloat)  FR
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1969-12-31 15:59:00	-4532.570196151733
-1969-12-31 16:00:00	-35057.677050709724
+1969-12-31 15:59:00	-4532.570000171661
+1969-12-31 16:00:00	-35057.6769926548
 PREHOOK: query: explain SELECT CAST(`__time` AS DATE) AS `x_date`, SUM(cfloat)  FROM druid_table_alltypesorc GROUP BY CAST(`__time` AS DATE) ORDER BY `x_date` LIMIT 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -763,7 +763,7 @@ POSTHOOK: query: SELECT CAST(`__time` AS DATE) AS `x_date`, SUM(cfloat)  FROM dr
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1969-12-31	-39590.24724686146
+1969-12-31	-39590.24699282646
 PREHOOK: query: SELECT CAST(`__time` AS DATE) AS `x_date` FROM druid_table_alltypesorc ORDER BY `x_date` LIMIT 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_joins.q.out b/ql/src/test/results/clientpositive/druid/druidmini_joins.q.out
index de0f2d8..6c9a964 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_joins.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_joins.q.out
@@ -62,8 +62,10 @@ druid_table_with_nulls `tbl1`
   )
   `tbl2`
     ON (`tbl1`.`username` = `tbl2`.`username`)
-GROUP BY `tbl1`.`username`
+GROUP BY `tbl1`.`username` ORDER BY `sum_double1`
 PREHOOK: type: QUERY
+PREHOOK: Input: default@druid_table_with_nulls
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: EXPLAIN SELECT
 username AS `username`,
 SUM(double1) AS `sum_double1`
@@ -80,8 +82,10 @@ druid_table_with_nulls `tbl1`
   )
   `tbl2`
     ON (`tbl1`.`username` = `tbl2`.`username`)
-GROUP BY `tbl1`.`username`
+GROUP BY `tbl1`.`username` ORDER BY `sum_double1`
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@druid_table_with_nulls
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-0 depends on stages: Stage-1
@@ -91,8 +95,9 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -117,9 +122,9 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 4 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: tbl1
@@ -135,7 +140,7 @@ STAGE PLANS:
                     Map-reduce partition columns: username (type: string)
                     Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: NONE
                     value expressions: double1 (type: double)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap
@@ -151,6 +156,7 @@ STAGE PLANS:
                 Group By Operator
                   aggregations: sum(_col1)
                   keys: _col0 (type: string)
+                  minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE
@@ -161,7 +167,7 @@ STAGE PLANS:
                     Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col1 (type: double)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 aggregations: sum(VALUE._col0)
@@ -169,6 +175,18 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 633 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: double)
+                  sort order: +
+                  Statistics: Num rows: 3 Data size: 633 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: string)
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: double)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 3 Data size: 633 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   Statistics: Num rows: 3 Data size: 633 Basic stats: COMPLETE Column stats: NONE
@@ -199,7 +217,7 @@ druid_table_with_nulls `tbl1`
   )
   `tbl2`
     ON (`tbl1`.`username` = `tbl2`.`username`)
-GROUP BY `tbl1`.`username`
+GROUP BY `tbl1`.`username` ORDER BY `sum_double1`
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_with_nulls
 PREHOOK: Output: hdfs://### HDFS PATH ###
@@ -219,12 +237,12 @@ druid_table_with_nulls `tbl1`
   )
   `tbl2`
     ON (`tbl1`.`username` = `tbl2`.`username`)
-GROUP BY `tbl1`.`username`
+GROUP BY `tbl1`.`username` ORDER BY `sum_double1`
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_with_nulls
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-alfred	10.300000190734863
-bob	3.140000104904175
 bonnie	0.0
-charlie	25.600000381469727
 calvin	0.0
+bob	3.14
+alfred	10.3
+charlie	25.6
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_masking.q.out b/ql/src/test/results/clientpositive/druid/druidmini_masking.q.out
index e3b50a4..0262e9b 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_masking.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_masking.q.out
@@ -42,8 +42,12 @@ POSTHOOK: Lineage: masking_test_druid.key SCRIPT []
 POSTHOOK: Lineage: masking_test_druid.username SCRIPT []
 PREHOOK: query: explain select username, key from masking_test_druid
 PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_druid
+PREHOOK: Output: hdfs://### HDFS PATH ###
 POSTHOOK: query: explain select username, key from masking_test_druid
 POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_druid
+POSTHOOK: Output: hdfs://### HDFS PATH ###
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
 
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_semijoin_reduction_all_types.q.out b/ql/src/test/results/clientpositive/druid/druidmini_semijoin_reduction_all_types.q.out
new file mode 100644
index 0000000..daf1e56
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid/druidmini_semijoin_reduction_all_types.q.out
@@ -0,0 +1,1973 @@
+PREHOOK: query: DROP TABLE IF EXISTS alltypesorc_small
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE IF EXISTS alltypesorc_small
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE alltypesorc_small(
+    ctinyint TINYINT,
+    csmallint SMALLINT,
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE,
+    cstring1 STRING,
+    cstring2 STRING,
+    ctimestamp1 TIMESTAMP,
+    cboolean1 BOOLEAN,
+    cboolean2 BOOLEAN)
+    STORED AS ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alltypesorc_small
+POSTHOOK: query: CREATE TABLE alltypesorc_small(
+    ctinyint TINYINT,
+    csmallint SMALLINT,
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE,
+    cstring1 STRING,
+    cstring2 STRING,
+    ctimestamp1 TIMESTAMP,
+    cboolean1 BOOLEAN,
+    cboolean2 BOOLEAN)
+    STORED AS ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alltypesorc_small
+PREHOOK: query: Insert into table alltypesorc_small
+Select ctinyint, csmallint, cint, cbigint, cfloat, cdouble, cstring1, cstring2, cast(`__time` as timestamp), cboolean1, cboolean2 from druid_table_alltypesorc where cstring2 like '%a%' and cstring1 like '%a%'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: default@alltypesorc_small
+POSTHOOK: query: Insert into table alltypesorc_small
+Select ctinyint, csmallint, cint, cbigint, cfloat, cdouble, cstring1, cstring2, cast(`__time` as timestamp), cboolean1, cboolean2 from druid_table_alltypesorc where cstring2 like '%a%' and cstring1 like '%a%'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: default@alltypesorc_small
+POSTHOOK: Lineage: alltypesorc_small.cbigint SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.cboolean1 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cboolean1, type:boolean, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.cboolean2 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cboolean2, type:boolean, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.cdouble SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cdouble, type:double, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.cfloat SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cfloat, type:float, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.cint SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cint, type:int, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.csmallint SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:csmallint, type:smallint, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.cstring1 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cstring1, type:string, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.cstring2 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cstring2, type:string, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.ctimestamp1 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:vc, type:timestamp, comment:from deserializer), ]
+POSTHOOK: Lineage: alltypesorc_small.ctinyint SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:ctinyint, type:tinyint, comment:from deserializer), ]
+PREHOOK: query: Select count(*) from alltypesorc_small
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: Select count(*) from alltypesorc_small
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+69
+PREHOOK: query: Select count(*) from druid_table_alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: Select count(*) from druid_table_alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+9173
+PREHOOK: query: DESCRIBE druid_table_alltypesorc
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: query: DESCRIBE druid_table_alltypesorc
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_alltypesorc
+__time              	timestamp with local time zone	from deserializer   
+cstring1            	string              	from deserializer   
+cstring2            	string              	from deserializer   
+cdouble             	double              	from deserializer   
+cfloat              	float               	from deserializer   
+ctinyint            	tinyint             	from deserializer   
+csmallint           	smallint            	from deserializer   
+cint                	int                 	from deserializer   
+cbigint             	bigint              	from deserializer   
+cboolean1           	boolean             	from deserializer   
+cboolean2           	boolean             	from deserializer   
+cintstring          	string              	from deserializer   
+cfloatstring        	string              	from deserializer   
+cdoublestring       	string              	from deserializer   
+PREHOOK: query: DESCRIBE alltypesorc_small
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@alltypesorc_small
+POSTHOOK: query: DESCRIBE alltypesorc_small
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@alltypesorc_small
+ctinyint            	tinyint             	                    
+csmallint           	smallint            	                    
+cint                	int                 	                    
+cbigint             	bigint              	                    
+cfloat              	float               	                    
+cdouble             	double              	                    
+cstring1            	string              	                    
+cstring2            	string              	                    
+ctimestamp1         	timestamp           	                    
+cboolean1           	boolean             	                    
+cboolean2           	boolean             	                    
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: cstring1 is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: cstring1 is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cstring1 (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (cstring1 BETWEEN DynamicValue(RS_4_alltypesorc_small_cstring1_min) AND DynamicValue(RS_4_alltypesorc_small_cstring1_max) and in_bloom_filter(cstring1, DynamicValue(RS_4_alltypesorc_small_cstring1_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames cstring1
+                    druid.fieldTypes string
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cstring1","lower":"DynamicValue(RS_4_alltypesorc_small_cstring1_min)","upp [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cstring1 BETWEEN DynamicValue(RS_4_alltypesorc_small_cstring1_min) AND DynamicValue(RS_4_alltypesorc_small_cstring1_max) and in_bloom_filter(cstring1, DynamicValue(RS_4_alltypesorc_small_cstring1_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: cstring1 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: cstring1 (type: string)
+                      Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 cstring1 (type: string)
+                Statistics: Num rows: 10090 Data size: 1764118 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+69
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+69
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: ctinyint is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: ctinyint is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: tinyint)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: tinyint)
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: tinyint)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=3)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: tinyint), _col1 (type: tinyint), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (ctinyint BETWEEN DynamicValue(RS_4_alltypesorc_small_ctinyint_min) AND DynamicValue(RS_4_alltypesorc_small_ctinyint_max) and in_bloom_filter(ctinyint, DynamicValue(RS_4_alltypesorc_small_ctinyint_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames ctinyint
+                    druid.fieldTypes tinyint
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"ctinyint","lower":"DynamicValue(RS_4_alltypesorc_small_ctinyint_min)","upp [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (ctinyint BETWEEN DynamicValue(RS_4_alltypesorc_small_ctinyint_min) AND DynamicValue(RS_4_alltypesorc_small_ctinyint_max) and in_bloom_filter(ctinyint, DynamicValue(RS_4_alltypesorc_small_ctinyint_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: ctinyint (type: tinyint)
+                      sort order: +
+                      Map-reduce partition columns: ctinyint (type: tinyint)
+                      Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: tinyint)
+                  1 ctinyint (type: tinyint)
+                Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=3)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: tinyint), _col1 (type: tinyint), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+73895
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+73895
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: csmallint is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: csmallint is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: csmallint (type: smallint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: smallint)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: smallint)
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: smallint)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: smallint), _col1 (type: smallint), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (csmallint BETWEEN DynamicValue(RS_4_alltypesorc_small_csmallint_min) AND DynamicValue(RS_4_alltypesorc_small_csmallint_max) and in_bloom_filter(csmallint, DynamicValue(RS_4_alltypesorc_small_csmallint_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames csmallint
+                    druid.fieldTypes smallint
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"csmallint","lower":"DynamicValue(RS_4_alltypesorc_small_csmallint_min)","u [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (csmallint BETWEEN DynamicValue(RS_4_alltypesorc_small_csmallint_min) AND DynamicValue(RS_4_alltypesorc_small_csmallint_max) and in_bloom_filter(csmallint, DynamicValue(RS_4_alltypesorc_small_csmallint_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: csmallint (type: smallint)
+                      sort order: +
+                      Map-reduce partition columns: csmallint (type: smallint)
+                      Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: smallint)
+                  1 csmallint (type: smallint)
+                Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: smallint), _col1 (type: smallint), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+211968
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+211968
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: cint is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: cint is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: int)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (cint BETWEEN DynamicValue(RS_4_alltypesorc_small_cint_min) AND DynamicValue(RS_4_alltypesorc_small_cint_max) and in_bloom_filter(cint, DynamicValue(RS_4_alltypesorc_small_cint_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames cint
+                    druid.fieldTypes int
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cint","lower":"DynamicValue(RS_4_alltypesorc_small_cint_min)","upper":"Dyn [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN DynamicValue(RS_4_alltypesorc_small_cint_min) AND DynamicValue(RS_4_alltypesorc_small_cint_max) and in_bloom_filter(cint, DynamicValue(RS_4_alltypesorc_small_cint_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: cint (type: int)
+                      sort order: +
+                      Map-reduce partition columns: cint (type: int)
+                      Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 cint (type: int)
+                Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+69
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+69
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: cbigint is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: cbigint is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cbigint (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: bigint)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: bigint)
+                        Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: bigint)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (cbigint BETWEEN DynamicValue(RS_4_alltypesorc_small_cbigint_min) AND DynamicValue(RS_4_alltypesorc_small_cbigint_max) and in_bloom_filter(cbigint, DynamicValue(RS_4_alltypesorc_small_cbigint_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames cbigint
+                    druid.fieldTypes bigint
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cbigint","lower":"DynamicValue(RS_4_alltypesorc_small_cbigint_min)","upper [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cbigint BETWEEN DynamicValue(RS_4_alltypesorc_small_cbigint_min) AND DynamicValue(RS_4_alltypesorc_small_cbigint_max) and in_bloom_filter(cbigint, DynamicValue(RS_4_alltypesorc_small_cbigint_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: cbigint (type: bigint)
+                      sort order: +
+                      Map-reduce partition columns: cbigint (type: bigint)
+                      Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: bigint)
+                  1 cbigint (type: bigint)
+                Statistics: Num rows: 10090 Data size: 76700 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+69
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+69
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: cfloat is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: cfloat is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cfloat (type: float)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: float)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: float)
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: float)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=3)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: float), _col1 (type: float), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (cfloat BETWEEN DynamicValue(RS_4_alltypesorc_small_cfloat_min) AND DynamicValue(RS_4_alltypesorc_small_cfloat_max) and in_bloom_filter(cfloat, DynamicValue(RS_4_alltypesorc_small_cfloat_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames cfloat
+                    druid.fieldTypes float
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"not","field":{"type":"selector","dimension":"cfloat","value":null,"extractionFn":null}},"columns":["cfloat"],"legacy":null,"context":null,"des [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cfloat BETWEEN DynamicValue(RS_4_alltypesorc_small_cfloat_min) AND DynamicValue(RS_4_alltypesorc_small_cfloat_max) and in_bloom_filter(cfloat, DynamicValue(RS_4_alltypesorc_small_cfloat_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: cfloat (type: float)
+                      sort order: +
+                      Map-reduce partition columns: cfloat (type: float)
+                      Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: float)
+                  1 cfloat (type: float)
+                Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=3)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: float), _col1 (type: float), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+73895
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+73895
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: cdouble is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: cdouble is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cdouble (type: double)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: double)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: double)
+                        Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: double)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (cdouble BETWEEN DynamicValue(RS_4_alltypesorc_small_cdouble_min) AND DynamicValue(RS_4_alltypesorc_small_cdouble_max) and in_bloom_filter(cdouble, DynamicValue(RS_4_alltypesorc_small_cdouble_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames cdouble
+                    druid.fieldTypes double
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cdouble","lower":"DynamicValue(RS_4_alltypesorc_small_cdouble_min)","upper [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cdouble BETWEEN DynamicValue(RS_4_alltypesorc_small_cdouble_min) AND DynamicValue(RS_4_alltypesorc_small_cdouble_max) and in_bloom_filter(cdouble, DynamicValue(RS_4_alltypesorc_small_cdouble_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: cdouble (type: double)
+                      sort order: +
+                      Map-reduce partition columns: cdouble (type: double)
+                      Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: double)
+                  1 cdouble (type: double)
+                Statistics: Num rows: 10090 Data size: 76700 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+211968
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = druid_table_alltypesorc.`__time`)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = druid_table_alltypesorc.`__time`)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: CAST( ctimestamp1 AS timestamp with local time zone) is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: CAST( ctimestamp1 AS timestamp with local time zone) is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: CAST( ctimestamp1 AS timestamp with local time zone) (type: timestamp with local time zone)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: timestamp with local time zone)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: timestamp with local time zone)
+                        Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: timestamp with local time zone)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: timestamp with local time zone), _col1 (type: timestamp with local time zone), _col2 (type: binary)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (vc BETWEEN DynamicValue(RS_4_alltypesorc_small_ctimestamp1_min) AND DynamicValue(RS_4_alltypesorc_small_ctimestamp1_max) and in_bloom_filter(vc, DynamicValue(RS_4_alltypesorc_small_ctimestamp1_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames vc
+                    druid.fieldTypes timestamp with local time zone
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[{"type":"expression","name":"vc","expression":"\"__time\"","outputType":"LONG"}],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":null,"columns":["vc"],"legacy":null,"context":null,"descending":false,"g [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (vc BETWEEN DynamicValue(RS_4_alltypesorc_small_ctimestamp1_min) AND DynamicValue(RS_4_alltypesorc_small_ctimestamp1_max) and in_bloom_filter(vc, DynamicValue(RS_4_alltypesorc_small_ctimestamp1_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: vc (type: timestamp with local time zone)
+                      sort order: +
+                      Map-reduce partition columns: vc (type: timestamp with local time zone)
+                      Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: timestamp with local time zone)
+                  1 vc (type: timestamp with local time zone)
+                Statistics: Num rows: 10090 Data size: 383504 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: timestamp with local time zone), _col1 (type: timestamp with local time zone), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+434493
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+434493
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: cboolean1 is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: cboolean1 is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cboolean1 (type: boolean)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: boolean)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: boolean), _col1 (type: boolean), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (cboolean1 BETWEEN DynamicValue(RS_4_alltypesorc_small_cboolean1_min) AND DynamicValue(RS_4_alltypesorc_small_cboolean1_max) and in_bloom_filter(cboolean1, DynamicValue(RS_4_alltypesorc_small_cboolean1_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames cboolean1
+                    druid.fieldTypes boolean
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"not","field":{"type":"selector","dimension":"cboolean1","value":null,"extractionFn":null}},"columns":["cboolean1"],"legacy":null,"context":nul [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cboolean1 BETWEEN DynamicValue(RS_4_alltypesorc_small_cboolean1_min) AND DynamicValue(RS_4_alltypesorc_small_cboolean1_max) and in_bloom_filter(cboolean1, DynamicValue(RS_4_alltypesorc_small_cboolean1_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: cboolean1 (type: boolean)
+                      sort order: +
+                      Map-reduce partition columns: cboolean1 (type: boolean)
+                      Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: boolean)
+                  1 cboolean1 (type: boolean)
+                Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: boolean), _col1 (type: boolean), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+418071
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+418071
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cint as string) = druid_table_alltypesorc.cintstring)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cint as string) = druid_table_alltypesorc.cintstring)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: CAST( cint AS STRING) is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: CAST( cint AS STRING) is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: CAST( cint AS STRING) (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 12696 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 69 Data size: 12696 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 12696 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (cintstring BETWEEN DynamicValue(RS_4_alltypesorc_small_cint_min) AND DynamicValue(RS_4_alltypesorc_small_cint_max) and in_bloom_filter(cintstring, DynamicValue(RS_4_alltypesorc_small_cint_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames cintstring
+                    druid.fieldTypes string
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cintstring","lower":"DynamicValue(RS_4_alltypesorc_small_cint_min)","upper [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cintstring BETWEEN DynamicValue(RS_4_alltypesorc_small_cint_min) AND DynamicValue(RS_4_alltypesorc_small_cint_max) and in_bloom_filter(cintstring, DynamicValue(RS_4_alltypesorc_small_cint_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: cintstring (type: string)
+                      sort order: +
+                      Map-reduce partition columns: cintstring (type: string)
+                      Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 cintstring (type: string)
+                Statistics: Num rows: 10090 Data size: 1764118 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cint as string) = druid_table_alltypesorc.cintstring)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cint as string) = druid_table_alltypesorc.cintstring)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+69
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cint as string) = druid_table_alltypesorc.cintstring)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cint as string) = druid_table_alltypesorc.cintstring)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+69
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cdouble as string) = druid_table_alltypesorc.cdoublestring)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cdouble as string) = druid_table_alltypesorc.cdoublestring)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: CAST( cdouble AS STRING) is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: CAST( cdouble AS STRING) is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: CAST( cdouble AS STRING) (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 12696 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 69 Data size: 12696 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 12696 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (cdoublestring BETWEEN DynamicValue(RS_4_alltypesorc_small_cdouble_min) AND DynamicValue(RS_4_alltypesorc_small_cdouble_max) and in_bloom_filter(cdoublestring, DynamicValue(RS_4_alltypesorc_small_cdouble_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames cdoublestring
+                    druid.fieldTypes string
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cdoublestring","lower":"DynamicValue(RS_4_alltypesorc_small_cdouble_min)", [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cdoublestring BETWEEN DynamicValue(RS_4_alltypesorc_small_cdouble_min) AND DynamicValue(RS_4_alltypesorc_small_cdouble_max) and in_bloom_filter(cdoublestring, DynamicValue(RS_4_alltypesorc_small_cdouble_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: cdoublestring (type: string)
+                      sort order: +
+                      Map-reduce partition columns: cdoublestring (type: string)
+                      Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 cdoublestring (type: string)
+                Statistics: Num rows: 10090 Data size: 1764118 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cdouble as string) = druid_table_alltypesorc.cdoublestring)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cdouble as string) = druid_table_alltypesorc.cdoublestring)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+0
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cdouble as string) = druid_table_alltypesorc.cdoublestring)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cdouble as string) = druid_table_alltypesorc.cdoublestring)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+0
+PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cfloat as string) = druid_table_alltypesorc.cfloatstring)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cfloat as string) = druid_table_alltypesorc.cfloatstring)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: alltypesorc_small
+                  filterExpr: CAST( cfloat AS STRING) is not null (type: boolean)
+                  Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: CAST( cfloat AS STRING) is not null (type: boolean)
+                    Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: CAST( cfloat AS STRING) (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 69 Data size: 12696 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 69 Data size: 12696 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 69 Data size: 12696 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=3)
+                          minReductionHashAggr: 0.98550725
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: druid_table_alltypesorc
+                  filterExpr: (cfloatstring BETWEEN DynamicValue(RS_4_alltypesorc_small_cfloat_min) AND DynamicValue(RS_4_alltypesorc_small_cfloat_max) and in_bloom_filter(cfloatstring, DynamicValue(RS_4_alltypesorc_small_cfloat_bloom_filter))) (type: boolean)
+                  properties:
+                    druid.fieldNames cfloatstring
+                    druid.fieldTypes string
+                    druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cfloatstring","lower":"DynamicValue(RS_4_alltypesorc_small_cfloat_min)","u [...]
+                    druid.query.type scan
+                  Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cfloatstring BETWEEN DynamicValue(RS_4_alltypesorc_small_cfloat_min) AND DynamicValue(RS_4_alltypesorc_small_cfloat_max) and in_bloom_filter(cfloatstring, DynamicValue(RS_4_alltypesorc_small_cfloat_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: cfloatstring (type: string)
+                      sort order: +
+                      Map-reduce partition columns: cfloatstring (type: string)
+                      Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 cfloatstring (type: string)
+                Statistics: Num rows: 10090 Data size: 1764118 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=3)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cfloat as string) = druid_table_alltypesorc.cfloatstring)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cfloat as string) = druid_table_alltypesorc.cfloatstring)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+73895
+PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cfloat as string) = druid_table_alltypesorc.cfloatstring)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_small
+PREHOOK: Input: default@druid_table_alltypesorc
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (cast(alltypesorc_small.cfloat as string) = druid_table_alltypesorc.cfloatstring)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_small
+POSTHOOK: Input: default@druid_table_alltypesorc
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+73895
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out b/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out
index 395a69b..d38e3df 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out
@@ -34,7 +34,7 @@ POSTHOOK: query: SELECT count(*) FROM druid_table_alltypesorc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-6105
+9173
 PREHOOK: query: EXPLAIN SELECT floor_year(`__time`), SUM(cfloat), SUM(cdouble), SUM(ctinyint), SUM(csmallint),SUM(cint), SUM(cbigint)
 FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 PREHOOK: type: QUERY
@@ -75,7 +75,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1969-01-01 00:00:00.0 US/Pacific	-39590.24724686146	2.7308662793799996E7	-39967	7781089	1408069801800	10992545287
+1969-01-01 00:00:00.0 US/Pacific	-39590.24699282646	2.73086627938E7	-39967	7781089	1408069801800	10992545287
 PREHOOK: query: EXPLAIN SELECT floor_year(`__time`), MIN(cfloat), MIN(cdouble), MIN(ctinyint), MIN(csmallint),MIN(cint), MIN(cbigint)
 FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 PREHOOK: type: QUERY
@@ -116,7 +116,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1969-01-01 00:00:00.0 US/Pacific	-1790.7781	-308691.8399999999	2	14255	-1073279343	-8577981133
+1969-01-01 00:00:00.0 US/Pacific	-64.0	-16373.0	-64	-16373	-1073279343	-2147311592
 PREHOOK: query: EXPLAIN SELECT floor_year(`__time`), MAX(cfloat), MAX(cdouble), MAX(ctinyint), MAX(csmallint),MAX(cint), MAX(cbigint)
 FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 PREHOOK: type: QUERY
@@ -157,7 +157,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1969-01-01 00:00:00.0 US/Pacific	769.16394	1.9565517977799997E7	-45	-8101	1276572707	4923772860
+1969-01-01 00:00:00.0 US/Pacific	79.553	9763215.5639	62	16370	1073680599	2145498388
 PREHOOK: query: EXPLAIN SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc GROUP BY cstring1 ORDER BY s ASC LIMIT 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -195,7 +195,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 1cGVWH7n1QU	-596096.6799999999
-821UdmGbkEf4j	-14161.82699999999
+821UdmGbkEf4j	-14161.826999999997
 00iT08	0.0
 02v8WnLuYDos3Cq	0.0
 yv1js	0.0
@@ -240,7 +240,7 @@ POSTHOOK: query: SELECT cstring2, MAX(cdouble) FROM druid_table_alltypesorc GROU
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-NULL	1.9565517977799997E7
+NULL	9763215.5639
 0034fkcXMQI3	15601.0
 004J8y	0.0
 00GNm	-200.0
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out b/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out
index 917e22a..ea893b9 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out
@@ -6,7 +6,7 @@ POSTHOOK: query: SELECT count(*) FROM druid_table_alltypesorc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-6105
+9173
 PREHOOK: query: SELECT floor_year(`__time`), SUM(cfloat), SUM(cdouble), SUM(ctinyint), SUM(csmallint),SUM(cint), SUM(cbigint)
 FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 PREHOOK: type: QUERY
@@ -17,7 +17,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1969-01-01 00:00:00.0 US/Pacific	-39590.24724686146	2.7308662793799996E7	-39967	7781089	1408069801800	10992545287
+1969-01-01 00:00:00.0 US/Pacific	-39590.24699282646	2.73086627938E7	-39967	7781089	1408069801800	10992545287
 PREHOOK: query: SELECT floor_year(`__time`), MIN(cfloat), MIN(cdouble), MIN(ctinyint), MIN(csmallint),MIN(cint), MIN(cbigint)
 FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 PREHOOK: type: QUERY
@@ -28,7 +28,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1969-01-01 00:00:00.0 US/Pacific	-1790.7781	-308691.8399999999	2	14255	-1073279343	-8577981133
+1969-01-01 00:00:00.0 US/Pacific	-64.0	-16373.0	-64	-16373	-1073279343	-2147311592
 PREHOOK: query: SELECT floor_year(`__time`), MAX(cfloat), MAX(cdouble), MAX(ctinyint), MAX(csmallint),MAX(cint), MAX(cbigint)
 FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 PREHOOK: type: QUERY
@@ -39,7 +39,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`)
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1969-01-01 00:00:00.0 US/Pacific	769.16394	1.9565517977799997E7	-45	-8101	1276572707	4923772860
+1969-01-01 00:00:00.0 US/Pacific	79.553	9763215.5639	62	16370	1073680599	2145498388
 PREHOOK: query: SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc GROUP BY cstring1 ORDER BY s ASC LIMIT 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@druid_table_alltypesorc
@@ -49,7 +49,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 1cGVWH7n1QU	-596096.6799999999
-821UdmGbkEf4j	-14161.82699999999
+821UdmGbkEf4j	-14161.826999999997
 00iT08	0.0
 02v8WnLuYDos3Cq	0.0
 yv1js	0.0
@@ -66,7 +66,7 @@ POSTHOOK: query: SELECT cstring2, MAX(cdouble) FROM druid_table_alltypesorc GROU
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@druid_table_alltypesorc
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-NULL	1.9565517977799997E7
+NULL	9763215.5639
 0034fkcXMQI3	15601.0
 004J8y	0.0
 00GNm	-200.0
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
index f62788f..6cf964e 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
@@ -67,6 +67,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: t
+                  filterExpr: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
@@ -389,6 +390,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: t
+                  filterExpr: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
index ec6ffea..fb12c2c 100644
--- a/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
+++ b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
@@ -3265,6 +3265,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: t
+                  filterExpr: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
@@ -3606,6 +3607,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: t
+                  filterExpr: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (a BETWEEN DynamicValue(RS_3_s_a_min) AND DynamicValue(RS_3_s_a_max) and in_bloom_filter(a, DynamicValue(RS_3_s_a_bloom_filter))) (type: boolean)
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/query1b.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/query1b.q.out
index 72661a7..bef5ef1 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/query1b.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/query1b.q.out
@@ -131,6 +131,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: customer
+                  filterExpr: (c_customer_sk BETWEEN DynamicValue(RS_47_store_returns_sr_customer_sk_min) AND DynamicValue(RS_47_store_returns_sr_customer_sk_max) and in_bloom_filter(c_customer_sk, DynamicValue(RS_47_store_returns_sr_customer_sk_bloom_filter))) (type: boolean)
                   Statistics: Num rows: 80000000 Data size: 8320000000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (c_customer_sk BETWEEN DynamicValue(RS_47_store_returns_sr_customer_sk_min) AND DynamicValue(RS_47_store_returns_sr_customer_sk_max) and in_bloom_filter(c_customer_sk, DynamicValue(RS_47_store_returns_sr_customer_sk_bloom_filter))) (type: boolean)