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

kgyrtkirk 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 160f467  HIVE-23462: Add option to rewrite CUME_DIST to sketch 
functions (#1031)
160f467 is described below

commit 160f467011a489bb27fdd17d228b9120e18f50e6
Author: Zoltan Haindrich <zhaindr...@cloudera.com>
AuthorDate: Wed Jun 10 13:40:33 2020 +0200

    HIVE-23462: Add option to rewrite CUME_DIST to sketch functions (#1031)
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   13 +-
 .../hadoop/hive/ql/exec/DataSketchesFunctions.java |   72 +-
 .../hive/ql/optimizer/calcite/HiveRelBuilder.java  |    7 +
 .../rules/HiveRewriteToDataSketchesRules.java      |  290 +++++-
 .../calcite/translator/SqlFunctionConverter.java   |    3 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java       |   13 +-
 .../hive/ql/parse/type/HiveFunctionHelper.java     |    3 +-
 .../hive/ql/exec/TestDataSketchesFunctions.java    |   38 +
 .../sketches_materialized_view_cume_dist.q         |   54 +
 .../clientpositive/sketches_rewrite_cume_dist.q    |   47 +
 .../sketches_rewrite_cume_dist_partition_by.q      |   27 +
 .../clientpositive/llap/cbo_rp_windowing_2.q.out   |  104 +-
 .../sketches_materialized_view_cume_dist.q.out     | 1054 ++++++++++++++++++++
 .../llap/sketches_rewrite_cume_dist.q.out          |  775 ++++++++++++++
 .../sketches_rewrite_cume_dist_partition_by.q.out  |  258 +++++
 15 files changed, 2635 insertions(+), 123 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 085ab4a..8cdb2eb 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2495,19 +2495,22 @@ public class HiveConf extends Configuration {
     
HIVE_OPTIMIZE_BI_REWRITE_COUNTDISTINCT_ENABLED("hive.optimize.bi.rewrite.countdistinct.enabled",
         true,
         "Enables to rewrite COUNT(DISTINCT(X)) queries to be rewritten to use 
sketch functions."),
-    HIVE_OPTIMIZE_BI_REWRITE_COUNT_DISTINCT_SKETCH(
-        "hive.optimize.bi.rewrite.countdistinct.sketch", "hll",
+    
HIVE_OPTIMIZE_BI_REWRITE_COUNT_DISTINCT_SKETCH("hive.optimize.bi.rewrite.countdistinct.sketch",
 "hll",
         new StringSet("hll"),
         "Defines which sketch type to use when rewriting COUNT(DISTINCT(X)) 
expressions. "
             + "Distinct counting can be done with: hll"),
     
HIVE_OPTIMIZE_BI_REWRITE_PERCENTILE_DISC_ENABLED("hive.optimize.bi.rewrite.percentile_disc.enabled",
         true,
         "Enables to rewrite PERCENTILE_DISC(X) queries to be rewritten to use 
sketch functions."),
-    HIVE_OPTIMIZE_BI_REWRITE_PERCENTILE_DISC_SKETCH(
-        "hive.optimize.bi.rewrite.percentile_disc.sketch", "kll",
+    
HIVE_OPTIMIZE_BI_REWRITE_PERCENTILE_DISC_SKETCH("hive.optimize.bi.rewrite.percentile_disc.sketch",
 "kll",
         new StringSet("kll"),
         "Defines which sketch type to use when rewriting PERCENTILE_DISC 
expressions. Options: kll"),
-
+    
HIVE_OPTIMIZE_BI_REWRITE_CUME_DIST_ENABLED("hive.optimize.bi.rewrite.cume_dist.enabled",
+        true,
+        "Enables to rewrite CUME_DIST(X) queries to be rewritten to use sketch 
functions."),
+    
HIVE_OPTIMIZE_BI_REWRITE_CUME_DIST_SKETCH("hive.optimize.bi.rewrite.cume_dist.sketch",
 "kll",
+        new StringSet("kll"),
+        "Defines which sketch type to use when rewriting CUME_DIST 
expressions. Options: kll"),
 
     // Statistics
     HIVE_STATS_ESTIMATE_STATS("hive.stats.estimate", true,
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/DataSketchesFunctions.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/DataSketchesFunctions.java
index cc48d5b..3a450a9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DataSketchesFunctions.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DataSketchesFunctions.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import java.lang.reflect.Method;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -61,8 +63,8 @@ public final class DataSketchesFunctions implements 
HiveUDFPlugin {
   private static final String SKETCH_TO_STRING = "stringify";
   private static final String UNION_SKETCH = "union";
   private static final String UNION_SKETCH1 = "union_f";
-  private static final String GET_N = "n";
-  private static final String GET_CDF = "cdf";
+  public static final String GET_N = "n";
+  public static final String GET_CDF = "cdf";
   private static final String GET_PMF = "pmf";
   private static final String GET_QUANTILES = "quantiles";
   public static final String GET_QUANTILE = "quantile";
@@ -123,13 +125,17 @@ public final class DataSketchesFunctions implements 
HiveUDFPlugin {
 
   private void buildCalciteFns() {
     for (SketchDescriptor sd : sketchClasses.values()) {
+
+      registerAsHiveFunction(sd.fnMap.get(SKETCH_TO_ESTIMATE));
+      registerAsHiveFunction(sd.fnMap.get(GET_QUANTILE));
+      registerAsHiveFunction(sd.fnMap.get(GET_CDF));
+      registerAsHiveFunction(sd.fnMap.get(GET_N));
+
       // Mergability is exposed to Calcite; which enables to use it during 
rollup.
       RelProtoDataType sketchType = RelDataTypeImpl.proto(SqlTypeName.BINARY, 
true);
 
       SketchFunctionDescriptor sketchSFD = sd.fnMap.get(DATA_TO_SKETCH);
       SketchFunctionDescriptor unionSFD = sd.fnMap.get(UNION_SKETCH);
-      SketchFunctionDescriptor estimateSFD = sd.fnMap.get(SKETCH_TO_ESTIMATE);
-      SketchFunctionDescriptor quantileSFD = sd.fnMap.get(GET_QUANTILE);
 
       if (sketchSFD == null || unionSFD == null) {
         continue;
@@ -152,33 +158,24 @@ public final class DataSketchesFunctions implements 
HiveUDFPlugin {
 
       unionSFD.setCalciteFunction(unionFn);
       sketchSFD.setCalciteFunction(sketchFn);
-      if (estimateSFD != null && 
estimateSFD.getReturnRelDataType().isPresent()) {
-
-        SqlFunction estimateFn = new HiveSqlFunction(estimateSFD.name,
-            SqlKind.OTHER_FUNCTION,
-            
ReturnTypes.explicit(estimateSFD.getReturnRelDataType().get().getSqlTypeName()),
-            InferTypes.ANY_NULLABLE,
-            OperandTypes.family(),
-            SqlFunctionCategory.USER_DEFINED_FUNCTION,
-            true,
-            false);
-
-        estimateSFD.setCalciteFunction(estimateFn);
-      }
 
-      if (quantileSFD != null && 
quantileSFD.getReturnRelDataType().isPresent()) {
-        SqlFunction quantileFn = new HiveSqlFunction(quantileSFD.name,
-            SqlKind.OTHER_FUNCTION,
-            
ReturnTypes.explicit(quantileSFD.getReturnRelDataType().get().getSqlTypeName()),
-            InferTypes.ANY_NULLABLE,
-            OperandTypes.family(),
-            SqlFunctionCategory.USER_DEFINED_FUNCTION,
-            true,
-            false);
 
-        quantileSFD.setCalciteFunction(quantileFn);
+    }
+  }
 
-      }
+  private void registerAsHiveFunction(SketchFunctionDescriptor sfd) {
+    if (sfd != null && sfd.getReturnRelDataType().isPresent()) {
+      SqlFunction cdfFn =
+          new HiveSqlFunction(sfd.name,
+              SqlKind.OTHER_FUNCTION,
+              ReturnTypes.explicit(sfd.getReturnRelDataType().get()),
+              InferTypes.ANY_NULLABLE,
+              OperandTypes.family(),
+              SqlFunctionCategory.USER_DEFINED_FUNCTION,
+              true,
+              false);
+
+      sfd.setCalciteFunction(cdfFn);
     }
   }
 
@@ -209,11 +206,11 @@ public final class DataSketchesFunctions implements 
HiveUDFPlugin {
 
   }
 
-  private static class SketchFunctionDescriptor implements 
HiveUDFPlugin.UDFDescriptor {
+  static class SketchFunctionDescriptor implements HiveUDFPlugin.UDFDescriptor 
{
     String name;
     Class<?> udfClass;
     private SqlFunction calciteFunction;
-    private Class<?> returnType;
+    private Type returnType;
 
     public SketchFunctionDescriptor(String name, Class<?> udfClass) {
       this.name = name;
@@ -235,12 +232,21 @@ public final class DataSketchesFunctions implements 
HiveUDFPlugin {
         return Optional.empty();
       } else {
         JavaTypeFactoryImpl typeFactory = new JavaTypeFactoryImpl(new 
HiveTypeSystemImpl());
+        if (returnType instanceof ParameterizedType) {
+          ParameterizedType parameterizedType = (ParameterizedType) returnType;
+          if (parameterizedType.getRawType() == List.class) {
+            final RelDataType componentRelType = 
typeFactory.createType(parameterizedType.getActualTypeArguments()[0]);
+            return Optional
+                
.of(typeFactory.createArrayType(typeFactory.createTypeWithNullability(componentRelType,
 true), -1));
+          }
+          return Optional.empty();
+        }
         return Optional.of(typeFactory.createType(returnType));
       }
     }
 
-    public void setReturnType(Class<?> returnType) {
-      this.returnType = returnType;
+    public void setReturnType(Type type) {
+      this.returnType = type;
     }
 
     @Override
@@ -272,7 +278,7 @@ public final class DataSketchesFunctions implements 
HiveUDFPlugin {
       if (UDF.class.isAssignableFrom(clazz)) {
         Optional<Method> evaluateMethod = getEvaluateMethod(clazz);
         if (evaluateMethod.isPresent()) {
-          value.setReturnType(evaluateMethod.get().getReturnType());
+          value.setReturnType(evaluateMethod.get().getGenericReturnType());
         }
       }
       fnMap.put(name, value);
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
index 184a026..4e184dc 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
@@ -42,6 +42,7 @@ import 
org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlSumAggFuncti
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlSumEmptyIsZeroAggFunction;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFloorDate;
 
+import com.google.common.collect.ImmutableList;
 
 /**
  * Builder for relational expressions in Hive.
@@ -165,4 +166,10 @@ public class HiveRelBuilder extends RelBuilder {
     return false;
   }
 
+  /** Make the method visible */
+  @Override
+  public AggCall aggregateCall(SqlAggFunction aggFunction, boolean distinct, 
boolean approximate, boolean ignoreNulls,
+      RexNode filter, ImmutableList<RexNode> orderKeys, String alias, 
ImmutableList<RexNode> operands) {
+    return super.aggregateCall(aggFunction, distinct, approximate, 
ignoreNulls, filter, orderKeys, alias, operands);
+  }
 }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRewriteToDataSketchesRules.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRewriteToDataSketchesRules.java
index 0123137..6cc638b 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRewriteToDataSketchesRules.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRewriteToDataSketchesRules.java
@@ -20,29 +20,43 @@ import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexFieldCollation;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexWindow;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilder.AggCall;
 import org.apache.hadoop.hive.ql.exec.DataSketchesFunctions;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelBuilder;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import 
org.apache.hadoop.hive.ql.optimizer.calcite.translator.SqlFunctionConverter;
 import org.apache.hive.plugin.api.HiveUDFPlugin.UDFDescriptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -56,30 +70,13 @@ import com.google.common.collect.Lists;
  * <br/>
  * Currently it can rewrite:
  * <ul>
- *  <li>{@code count(distinct(x))} to distinct counting sketches
- *    <pre>
- *     SELECT COUNT(DISTINCT id) FROM sketch_input;
- *       ⇒ SELECT ROUND(ds_hll_estimate(ds_hll_sketch(id))) FROM sketch_input;
- *    </pre>
+ *  <li>{@code count(distinct(x))} using {@code CountDistinctRewrite}
+ *  </li>
+ *  <li>{@code percentile_disc(0.2) within group (order by id)} using {@code 
PercentileDiscRewrite}
  *  </li>
- *  <li>{@code percentile_disc(0.2) within group (order by id)}
- *    <pre>
- *     SELECT PERCENTILE_DISC(0.2) WITHIN GROUP(ORDER BY ID) FROM sketch_input;
- *       ⇒ SELECT ds_kll_quantile(ds_kll_sketch(CAST(id AS FLOAT)), 0.2) FROM 
sketch_input;
- *    </pre>
+ *  <li>{@code cume_dist() over (order by id)} using {@code CumeDistRewrite}
  *  </li>
  *  </ul>
- *
- * <p>
- *   The transformation here works on Aggregate nodes; the operations done are 
the following:
- * </p>
- * <ol>
- * <li>Identify candidate aggregate calls</li>
- * <li>A new Project is inserted below the Aggregate; to help with data 
pre-processing</li>
- * <li>A new Aggregate is created in which the aggregation is done by the 
sketch function</li>
- * <li>A new Project is inserted on top of the Aggregate; which unwraps the 
resulting
- *    count-distinct estimation from the sketch representation</li>
- * </ol>
  */
 public final class HiveRewriteToDataSketchesRules {
 
@@ -87,6 +84,15 @@ public final class HiveRewriteToDataSketchesRules {
 
   /**
    * Generic support for rewriting an Aggregate into a chain of 
Project->Aggregate->Project.
+   * <p>
+   *   The transformation here works on Aggregate nodes; the operations done 
are the following:
+   * </p>
+   * <ol>
+   * <li>Identify candidate aggregate calls</li>
+   * <li>A new Project is inserted below the Aggregate; to help with data 
pre-processing</li>
+   * <li>A new Aggregate is created in which the aggregation is done by the 
sketch function</li>
+   * <li>A new Project is inserted on top of the Aggregate; which unwraps the 
resulting estimation from the sketch representation</li>
+   * </ol>
    */
   private static abstract class AggregateToProjectAggregateProject extends 
RelOptRule {
 
@@ -204,14 +210,21 @@ public final class HiveRewriteToDataSketchesRules {
         return fn.getCalciteFunction().get();
       }
 
-      abstract void rewrite(AggregateCall aggCall);
-
       abstract boolean isApplicable(AggregateCall aggCall);
 
-    }
+      abstract void rewrite(AggregateCall aggCall);
 
+    }
   };
 
+  /**
+   * Rewrites {@code count(distinct(x))} to distinct counting sketches.
+   *
+   * <pre>
+   *   SELECT COUNT(DISTINCT id) FROM sketch_input;
+   *   ⇒ SELECT ROUND(ds_hll_estimate(ds_hll_sketch(id))) FROM sketch_input;
+   * </pre>
+   */
   public static class CountDistinctRewrite extends 
AggregateToProjectAggregateProject {
 
     private final String sketchType;
@@ -256,7 +269,7 @@ public final class HiveRewriteToDataSketchesRules {
 
         SqlAggFunction aggFunction = (SqlAggFunction) 
getSqlOperator(DataSketchesFunctions.DATA_TO_SKETCH);
         boolean distinct = false;
-        boolean approximate = true;
+        boolean approximate = false;
         boolean ignoreNulls = true;
         List<Integer> argList = Lists.newArrayList(newProjectsBelow.size() - 
1);
         int filterArg = aggCall.filterArg;
@@ -279,6 +292,14 @@ public final class HiveRewriteToDataSketchesRules {
     }
   }
 
+  /**
+   * Rewrites {@code percentile_disc(0.2) within group (order by id)}.
+   *
+   * <pre>
+   *   SELECT PERCENTILE_DISC(0.2) WITHIN GROUP(ORDER BY ID) FROM sketch_input;
+   *   ⇒ SELECT ds_kll_quantile(ds_kll_sketch(CAST(id AS FLOAT)), 0.2) FROM 
sketch_input;
+   * </pre>
+   */
   public static class PercentileDiscRewrite extends 
AggregateToProjectAggregateProject {
 
     private final String sketchType;
@@ -343,7 +364,7 @@ public final class HiveRewriteToDataSketchesRules {
 
         SqlAggFunction aggFunction = (SqlAggFunction) 
getSqlOperator(DataSketchesFunctions.DATA_TO_SKETCH);
         boolean distinct = false;
-        boolean approximate = true;
+        boolean approximate = false;
         boolean ignoreNulls = true;
         List<Integer> argList = Lists.newArrayList(newProjectsBelow.size() - 
1);
         int filterArg = aggCall.filterArg;
@@ -368,4 +389,221 @@ public final class HiveRewriteToDataSketchesRules {
       }
     }
   }
+
+  /**
+   * Generic support for rewriting Windowing expression into a different form 
usually using joins.
+   */
+  private static abstract class WindowingToProjectAggregateJoinProject extends 
RelOptRule {
+
+    protected final String sketchType;
+
+    public WindowingToProjectAggregateJoinProject(String sketchType) {
+      super(operand(HiveProject.class, any()), HiveRelFactories.HIVE_BUILDER, 
null);
+      this.sketchType = sketchType;
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      final Project project = call.rel(0);
+
+      VbuilderPAP vb = buildProcessor(call);
+      RelNode newProject = vb.processProject(project);
+
+      if (newProject == project) {
+        return;
+      } else {
+        call.transformTo(newProject);
+      }
+    }
+
+    protected abstract VbuilderPAP buildProcessor(RelOptRuleCall call);
+
+    protected static abstract class VbuilderPAP {
+      private final String sketchClass;
+      protected final RelBuilder relBuilder;
+      protected final RexBuilder rexBuilder;
+
+      protected VbuilderPAP(String sketchClass, RelBuilder relBuilder) {
+        this.sketchClass = sketchClass;
+        this.relBuilder = relBuilder;
+        rexBuilder = relBuilder.getRexBuilder();
+      }
+
+      final class ProcessShuttle extends RexShuttle {
+        public RexNode visitOver(RexOver over) {
+          return processCall(over);
+        }
+      };
+
+      protected final RelNode processProject(Project project) {
+        RelNode origInput = project.getInput();
+        relBuilder.push(origInput);
+        RexShuttle shuttle = new ProcessShuttle();
+        List<RexNode> newProjects = new ArrayList<RexNode>();
+        for (RexNode expr : project.getChildExps()) {
+          newProjects.add(expr.accept(shuttle));
+        }
+        if (relBuilder.peek() == origInput) {
+          relBuilder.clear();
+          return project;
+        }
+        relBuilder.project(newProjects);
+        return relBuilder.build();
+      }
+
+      private final RexNode processCall(RexNode expr) {
+        if (expr instanceof RexOver) {
+          RexOver over = (RexOver) expr;
+          if (isApplicable(over)) {
+            return rewrite(over);
+          }
+        }
+        return expr;
+      }
+
+      protected final SqlOperator getSqlOperator(String fnName) {
+        UDFDescriptor fn = 
DataSketchesFunctions.INSTANCE.getSketchFunction(sketchClass, fnName);
+        if (!fn.getCalciteFunction().isPresent()) {
+          throw new RuntimeException(fn.toString() + " doesn't have a Calcite 
function associated with it");
+        }
+        return fn.getCalciteFunction().get();
+      }
+
+      /**
+       * Do the rewrite for the given expression.
+       *
+       * When this method is invoked the {@link #relBuilder} will only contain 
the current input.
+       * Expectation is to leave the new input there after the method finishes.
+       */
+      abstract RexNode rewrite(RexOver expr);
+
+      abstract boolean isApplicable(RexOver expr);
+
+    }
+  }
+
+  /**
+   * Rewrites {@code cume_dist() over (order by id)}.
+   *
+   *  <pre>
+   *   SELECT id, CUME_DIST() OVER (ORDER BY id) FROM sketch_input;
+   *     ⇒ SELECT id, 1.0-ds_kll_cdf(ds, CAST(-id AS FLOAT) )[0]
+   *       FROM sketch_input JOIN (
+   *         SELECT ds_kll_sketch(CAST(-id AS FLOAT)) AS ds FROM sketch_input
+   *       ) q;
+   *  </pre>
+   */
+  public static class CumeDistRewrite extends 
WindowingToProjectAggregateJoinProject {
+
+    public CumeDistRewrite(String sketchType) {
+      super(sketchType);
+    }
+
+    @Override
+    protected VbuilderPAP buildProcessor(RelOptRuleCall call) {
+      return new VB(sketchType, call.builder());
+    }
+
+    private static class VB extends VbuilderPAP {
+
+      protected VB(String sketchClass, RelBuilder relBuilder) {
+        super(sketchClass, relBuilder);
+      }
+
+      @Override
+      boolean isApplicable(RexOver over) {
+        SqlAggFunction aggOp = over.getAggOperator();
+        RexWindow window = over.getWindow();
+        if (aggOp.getName().equalsIgnoreCase("cume_dist") && 
window.orderKeys.size() == 1
+            && window.getLowerBound().isUnbounded() && 
window.getUpperBound().isUnbounded()) {
+          return true;
+        }
+        return false;
+      }
+
+      @Override
+      RexNode rewrite(RexOver over) {
+        RexWindow w = over.getWindow();
+        RexFieldCollation orderKey = w.orderKeys.get(0);
+        // we don't really support nulls in aggregate/etc...they are actually 
ignored
+        // so some hack will be needed for NULLs anyway..
+        ImmutableList<RexNode> partitionKeys = w.partitionKeys;
+
+        relBuilder.push(relBuilder.peek());
+        // the CDF function utilizes the '<' operator;
+        // negating the input will mirror the values on the x axis
+        // by using 1-CDF(-x) we could get a <= operator
+        RexNode key = orderKey.getKey();
+        key = rexBuilder.makeCall(SqlStdOperatorTable.UNARY_MINUS, key);
+        key = rexBuilder.makeCast(getFloatType(), key);
+
+        AggCall aggCall = ((HiveRelBuilder) relBuilder).aggregateCall(
+            (SqlAggFunction) 
getSqlOperator(DataSketchesFunctions.DATA_TO_SKETCH),
+            /* distinct */ false,
+            /* approximate */ false,
+            /* ignoreNulls */ true,
+            null,
+            ImmutableList.of(),
+            null,
+            ImmutableList.of(key));
+
+        relBuilder.aggregate(relBuilder.groupKey(partitionKeys), aggCall);
+
+        List<RexNode> joinConditions;
+        joinConditions = Ord.zip(partitionKeys).stream().map(o -> {
+          RexNode f = relBuilder.field(2, 1, o.i);
+          return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, 
o.e, f);
+        }).collect(Collectors.toList());
+        relBuilder.join(JoinRelType.INNER, joinConditions);
+
+        int sketchFieldIndex = relBuilder.peek().getRowType().getFieldCount() 
- 1;
+        RexInputRef sketchInputRef = relBuilder.field(sketchFieldIndex);
+        SqlOperator projectOperator = 
getSqlOperator(DataSketchesFunctions.GET_CDF);
+
+        // NULLs will be replaced by this value - to be before / after the 
other values
+        // note: the sketch will ignore NULLs entirely but they will be placed 
at 0.0 or 1.0
+        final RexNode nullReplacement =
+            relBuilder.literal(orderKey.getNullDirection() == 
NullDirection.FIRST ? Float.MAX_VALUE : -Float.MAX_VALUE);
+
+        // long story short: CAST(1.0f-CDF(CAST(COALESCE(-X, nullReplacement) 
AS FLOAT))[0] AS targetType)
+        RexNode projRex = key;
+        projRex = rexBuilder.makeCall(SqlStdOperatorTable.COALESCE, key, 
nullReplacement);
+        projRex = rexBuilder.makeCast(getFloatType(), projRex);
+        projRex = rexBuilder.makeCall(projectOperator, 
ImmutableList.of(sketchInputRef, projRex));
+        projRex = makeItemCall(projRex, relBuilder.literal(0));
+        projRex = rexBuilder.makeCall(SqlStdOperatorTable.MINUS, 
relBuilder.literal(1.0f), projRex);
+        projRex = rexBuilder.makeCast(over.getType(), projRex);
+
+        return projRex;
+      }
+
+      private RexNode makeItemCall(RexNode arr, RexNode offset) {
+        if(getClass().desiredAssertionStatus()) {
+          try {
+            SqlKind.class.getField("ITEM");
+            throw new RuntimeException("bind SqlKind.ITEM instead of this 
workaround - C1.23 a02155a70a");
+           } catch(NoSuchFieldException e) {
+             // ignore
+          }
+        }
+
+        try {
+        SqlOperator indexFn = SqlFunctionConverter.getCalciteFn("index",
+            ImmutableList.of(arr.getType(),offset.getType()),
+            arr.getType().getComponentType(), true, false);
+          RexNode call = rexBuilder.makeCall(indexFn, arr, offset);
+          return call;
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      private RelDataType getFloatType() {
+        RelDataTypeFactory typeFactory = rexBuilder.getTypeFactory();
+        RelDataType notNullFloatType = 
typeFactory.createSqlType(SqlTypeName.FLOAT);
+        RelDataType floatType = 
typeFactory.createTypeWithNullability(notNullFloatType, true);
+        return floatType;
+      }
+    }
+  }
 }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index 9819f4a..f4608c5 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -392,10 +392,11 @@ public class SqlFunctionConverter {
       registerFunction("istrue", SqlStdOperatorTable.IS_TRUE, 
hToken(HiveParser.Identifier, "istrue"));
       registerFunction("isnotfalse", SqlStdOperatorTable.IS_NOT_FALSE, 
hToken(HiveParser.Identifier, "isnotfalse"));
       registerFunction("isfalse", SqlStdOperatorTable.IS_FALSE, 
hToken(HiveParser.Identifier, "isfalse"));
-      registerFunction("is not distinct from", 
SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, hToken(HiveParser.EQUAL_NS, "<=>"));
+      registerFunction("<=>", SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, 
hToken(HiveParser.EQUAL_NS, "<=>"));
       registerFunction("when", SqlStdOperatorTable.CASE, 
hToken(HiveParser.Identifier, "when"));
       registerDuplicateFunction("case", SqlStdOperatorTable.CASE, 
hToken(HiveParser.Identifier, "when"));
       registerFunction("coalesce", SqlStdOperatorTable.COALESCE, 
hToken(HiveParser.Identifier, "coalesce"));
+
       // timebased
       registerFunction("year", HiveExtractDate.YEAR,
           hToken(HiveParser.Identifier, "year"));
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 377e828..2396641 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -1974,13 +1974,18 @@ public class CalcitePlanner extends SemanticAnalyzer {
       if (!isMaterializedViewMaintenance() && 
conf.getBoolVar(ConfVars.HIVE_OPTIMIZE_BI_ENABLED)) {
         // Rewrite to datasketches if enabled
         if 
(conf.getBoolVar(ConfVars.HIVE_OPTIMIZE_BI_REWRITE_COUNTDISTINCT_ENABLED)) {
-          String countDistinctSketchType = 
conf.getVar(ConfVars.HIVE_OPTIMIZE_BI_REWRITE_COUNT_DISTINCT_SKETCH);
-          RelOptRule rule = new 
HiveRewriteToDataSketchesRules.CountDistinctRewrite(countDistinctSketchType);
+          String sketchType = 
conf.getVar(ConfVars.HIVE_OPTIMIZE_BI_REWRITE_COUNT_DISTINCT_SKETCH);
+          RelOptRule rule = new 
HiveRewriteToDataSketchesRules.CountDistinctRewrite(sketchType);
           generatePartialProgram(program, true, HepMatchOrder.TOP_DOWN, rule);
         }
         if 
(conf.getBoolVar(ConfVars.HIVE_OPTIMIZE_BI_REWRITE_PERCENTILE_DISC_ENABLED)) {
-          String percentileDiscSketchType = 
conf.getVar(ConfVars.HIVE_OPTIMIZE_BI_REWRITE_PERCENTILE_DISC_SKETCH);
-          RelOptRule rule = new 
HiveRewriteToDataSketchesRules.PercentileDiscRewrite(percentileDiscSketchType);
+          String sketchType = 
conf.getVar(ConfVars.HIVE_OPTIMIZE_BI_REWRITE_PERCENTILE_DISC_SKETCH);
+          RelOptRule rule = new 
HiveRewriteToDataSketchesRules.PercentileDiscRewrite(sketchType);
+          generatePartialProgram(program, true, HepMatchOrder.TOP_DOWN, rule);
+        }
+        if 
(conf.getBoolVar(ConfVars.HIVE_OPTIMIZE_BI_REWRITE_CUME_DIST_ENABLED)) {
+          String sketchType = 
conf.getVar(ConfVars.HIVE_OPTIMIZE_BI_REWRITE_CUME_DIST_SKETCH);
+          RelOptRule rule = new 
HiveRewriteToDataSketchesRules.CumeDistRewrite(sketchType);
           generatePartialProgram(program, true, HepMatchOrder.TOP_DOWN, rule);
         }
       }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/HiveFunctionHelper.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/HiveFunctionHelper.java
index 07813b9..c91eb31 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/HiveFunctionHelper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/HiveFunctionHelper.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hive.ql.exec.FunctionInfo;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRexExecutorImpl;
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExtractDate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFloorDate;
@@ -389,7 +388,7 @@ public class HiveFunctionHelper implements FunctionHelper {
 
     if (FunctionRegistry.isRankingFunction(aggregateName)) {
       // Rank functions type is 'int'/'double'
-      if (aggregateName.equalsIgnoreCase("percent_rank")) {
+      if (aggregateName.equalsIgnoreCase("percent_rank") || 
aggregateName.equalsIgnoreCase("cume_dist")) {
         returnType = TypeInfoFactory.doubleTypeInfo;
       } else {
         returnType = TypeInfoFactory.intTypeInfo;
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestDataSketchesFunctions.java 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestDataSketchesFunctions.java
new file mode 100644
index 0000000..d446105
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestDataSketchesFunctions.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.Optional;
+
+import org.apache.calcite.rel.type.RelDataType;
+import 
org.apache.hadoop.hive.ql.exec.DataSketchesFunctions.SketchFunctionDescriptor;
+import org.junit.Test;
+
+public final class TestDataSketchesFunctions {
+
+  @Test
+  public void testKllGetCdfReturnType() {
+    SketchFunctionDescriptor cf =
+        DataSketchesFunctions.INSTANCE.getSketchFunction("kll", 
DataSketchesFunctions.GET_CDF);
+    Optional<RelDataType> retType = cf.getReturnRelDataType();
+    assertTrue(retType.get().getComponentType() != null);
+  }
+}
diff --git 
a/ql/src/test/queries/clientpositive/sketches_materialized_view_cume_dist.q 
b/ql/src/test/queries/clientpositive/sketches_materialized_view_cume_dist.q
new file mode 100644
index 0000000..e8e1795
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/sketches_materialized_view_cume_dist.q
@@ -0,0 +1,54 @@
+--! qt:transactional
+set hive.fetch.task.conversion=none;
+
+create table sketch_input (id int, category char(1))
+STORED AS ORC
+TBLPROPERTIES ('transactional'='true');
+
+insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b')
+; 
+
+-- create an mv for the intermediate results
+create  materialized view mv_1 as
+  select category,ds_kll_sketch(cast(-id as float)) from sketch_input group by 
category;
+
+-- bi mode on
+set hive.optimize.bi.enabled=true;
+
+explain
+select 'rewrite; mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id;
+select 'rewrite; mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id;
+
+set hive.optimize.bi.enabled=false;
+
+explain
+select 'no rewrite; no mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id;
+select 'no rewrite; no mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id;
+
+set hive.optimize.bi.enabled=true;
+
+insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b')
+;
+
+explain
+select 'rewrite; no mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id;
+select 'rewrite; no mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id;
+
+explain
+alter materialized view mv_1 rebuild;
+alter materialized view mv_1 rebuild;
+
+explain
+select 'rewrite; mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id;
+select 'rewrite; mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id;
+
+-- rewrite+mv matching with rollup
+explain
+select 'rewrite; mv matching', category, id, cume_dist() over (partition by 
category order by id) from sketch_input order by category,id;
+select 'rewrite; mv matching', category, id, cume_dist() over (partition by 
category order by id) from sketch_input order by category,id;
+
+drop materialized view mv_1;
diff --git a/ql/src/test/queries/clientpositive/sketches_rewrite_cume_dist.q 
b/ql/src/test/queries/clientpositive/sketches_rewrite_cume_dist.q
new file mode 100644
index 0000000..c7b0837
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/sketches_rewrite_cume_dist.q
@@ -0,0 +1,47 @@
+--! qt:transactional
+
+
+create table sketch_input (id int, category char(1))
+STORED AS ORC
+TBLPROPERTIES ('transactional'='true');
+
+insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b')
+; 
+
+select id,cume_dist() over (order by id) from sketch_input;
+
+select id,cume_dist() over (order by id),1.0-ds_kll_cdf(ds, CAST(-id AS FLOAT) 
)[0]
+from sketch_input
+join ( select ds_kll_sketch(cast(-id as float)) as ds from sketch_input ) q
+order by id;
+
+set hive.optimize.bi.enabled=true;
+
+-- see if rewrite happens
+explain
+select id,'rewrite',cume_dist() over (order by id) from sketch_input order by 
id;
+
+select id,'rewrite',cume_dist() over (order by id) from sketch_input order by 
id;
+
+-- see if rewrite happens in nested expressions
+explain
+select id,'rewrite',count(id) over ()*cume_dist() over (order by id) from 
sketch_input order by id;
+
+select id,'rewrite',count(id) over ()*cume_dist() over (order by id) from 
sketch_input order by id;
+
+
+insert into sketch_input values (null,'a'),(null,'b');
+
+explain
+select id,'rewrite',cume_dist() over (order by id nulls first) from 
sketch_input order by id nulls first;
+
+select id,'rewrite',cume_dist() over (order by id nulls first) from 
sketch_input order by id nulls first;
+
+explain
+select id,'rewrite',cume_dist() over (order by id nulls last) from 
sketch_input order by id nulls last;
+
+select id,'rewrite',cume_dist() over (order by id nulls last) from 
sketch_input order by id nulls last;
+
+select id,cume_dist() over (order by id) from sketch_input order by id;
diff --git 
a/ql/src/test/queries/clientpositive/sketches_rewrite_cume_dist_partition_by.q 
b/ql/src/test/queries/clientpositive/sketches_rewrite_cume_dist_partition_by.q
new file mode 100644
index 0000000..7931a59
--- /dev/null
+++ 
b/ql/src/test/queries/clientpositive/sketches_rewrite_cume_dist_partition_by.q
@@ -0,0 +1,27 @@
+--! qt:transactional
+
+create table sketch_input (id int, category char(1))
+STORED AS ORC
+TBLPROPERTIES ('transactional'='true');
+
+insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b'),
+  (1,null),(2,null),(10,null),(13,null);
+; 
+
+select id,category,cume_dist() over (partition by category order by id) from 
sketch_input order by category,id;
+
+select id,category,cume_dist() over (partition by category order by 
id),1.0-ds_kll_cdf(ds, CAST(-id AS FLOAT))[0]
+from sketch_input
+join ( select category as c,ds_kll_sketch(cast(-id as float)) as ds from 
sketch_input group by category) q on (q.c=category)
+order by category,id;
+
+set hive.optimize.bi.enabled=true;
+
+-- see if rewrite happens
+explain
+select id,'rewrite',cume_dist() over (partition by category order by id) from 
sketch_input order by category,id;
+
+select id,'rewrite',cume_dist() over (partition by category order by id) from 
sketch_input order by category,id;
+
diff --git a/ql/src/test/results/clientpositive/llap/cbo_rp_windowing_2.q.out 
b/ql/src/test/results/clientpositive/llap/cbo_rp_windowing_2.q.out
index 2ff927c..6ec2d60 100644
--- a/ql/src/test/results/clientpositive/llap/cbo_rp_windowing_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/cbo_rp_windowing_2.q.out
@@ -625,32 +625,32 @@ window w1 as (distribute by p_mfgr sort by p_mfgr, p_name 
rows between 2 precedi
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
-Manufacturer#1 almond antique burnished rose metallic  2       1       1       
0       0.0     1       2       2.0     0.0     2       2       2
-Manufacturer#1 almond antique burnished rose metallic  2       1       1       
0       0.0     1       2       2.0     0.0     2       2       2
-Manufacturer#1 almond antique chartreuse lavender yellow       34      3       
2       0       0.4     2       3       12.666666666666666      
15.084944665313014      2       34      2
-Manufacturer#1 almond antique salmon chartreuse burlywood      6       4       
3       0       0.6     2       4       11.0    13.379088160259652      2       
6       2
-Manufacturer#1 almond aquamarine burnished black steel 28      5       4       
0       0.8     3       5       14.4    13.763720427268202      2       28      
34
-Manufacturer#1 almond aquamarine pink moccasin thistle 42      6       5       
1       1.0     3       6       19.0    16.237815945091466      2       42      
6
-Manufacturer#2 almond antique violet chocolate turquoise       14      1       
1       0       0.0     1       1       14.0    0.0     4       14      14
-Manufacturer#2 almond antique violet turquoise frosted 40      2       2       
0       0.25    1       2       27.0    13.0    4       40      14
-Manufacturer#2 almond aquamarine midnight light salmon 2       3       3       
0       0.5     2       3       18.666666666666668      15.86050300449376       
4       2       14
-Manufacturer#2 almond aquamarine rose maroon antique   25      4       4       
0       0.75    2       4       20.25   14.00669482783144       4       25      
40
-Manufacturer#2 almond aquamarine sandy cyan gainsboro  18      5       5       
1       1.0     3       5       19.8    12.560254774486067      4       18      
2
-Manufacturer#3 almond antique chartreuse khaki white   17      1       1       
0       0.0     1       1       17.0    0.0     2       17      17
-Manufacturer#3 almond antique forest lavender goldenrod        14      2       
2       0       0.25    1       2       15.5    1.5     2       14      17
-Manufacturer#3 almond antique metallic orange dim      19      3       3       
0       0.5     2       3       16.666666666666668      2.0548046676563256      
2       19      17
-Manufacturer#3 almond antique misty red olive  1       4       4       0       
0.75    2       4       12.75   7.013380069552769       2       1       14
-Manufacturer#3 almond antique olive coral navajo       45      5       5       
1       1.0     3       5       19.2    14.344336861632886      2       45      
19
-Manufacturer#4 almond antique gainsboro frosted violet 10      1       1       
0       0.0     1       1       10.0    0.0     0       10      10
-Manufacturer#4 almond antique violet mint lemon        39      2       2       
0       0.25    1       2       24.5    14.5    0       39      10
-Manufacturer#4 almond aquamarine floral ivory bisque   27      3       3       
0       0.5     2       3       25.333333333333332      11.897712198383164      
0       27      10
-Manufacturer#4 almond aquamarine yellow dodger mint    7       4       4       
0       0.75    2       4       20.75   13.007209539328564      0       7       
39
-Manufacturer#4 almond azure aquamarine papaya violet   12      5       5       
1       1.0     3       5       19.0    12.149074038789951      0       12      
27
-Manufacturer#5 almond antique blue firebrick mint      31      1       1       
0       0.0     1       1       31.0    0.0     1       31      31
-Manufacturer#5 almond antique medium spring khaki      6       2       2       
0       0.25    1       2       18.5    12.5    1       6       31
-Manufacturer#5 almond antique sky peru orange  2       3       3       0       
0.5     2       3       13.0    12.832251036613439      1       2       31
-Manufacturer#5 almond aquamarine dodger light gainsboro        46      4       
4       0       0.75    2       4       21.25   18.102140757380052      1       
46      6
-Manufacturer#5 almond azure blanched chiffon midnight  23      5       5       
1       1.0     3       5       21.6    16.206171663906314      1       23      
2
+Manufacturer#1 almond antique burnished rose metallic  2       1       1       
0.3333333333333333      0.0     1       2       2.0     0.0     2       2       
2
+Manufacturer#1 almond antique burnished rose metallic  2       1       1       
0.3333333333333333      0.0     1       2       2.0     0.0     2       2       
2
+Manufacturer#1 almond antique chartreuse lavender yellow       34      3       
2       0.5     0.4     2       3       12.666666666666666      
15.084944665313014      2       34      2
+Manufacturer#1 almond antique salmon chartreuse burlywood      6       4       
3       0.6666666666666666      0.6     2       4       11.0    
13.379088160259652      2       6       2
+Manufacturer#1 almond aquamarine burnished black steel 28      5       4       
0.8333333333333334      0.8     3       5       14.4    13.763720427268202      
2       28      34
+Manufacturer#1 almond aquamarine pink moccasin thistle 42      6       5       
1.0     1.0     3       6       19.0    16.237815945091466      2       42      
6
+Manufacturer#2 almond antique violet chocolate turquoise       14      1       
1       0.2     0.0     1       1       14.0    0.0     4       14      14
+Manufacturer#2 almond antique violet turquoise frosted 40      2       2       
0.4     0.25    1       2       27.0    13.0    4       40      14
+Manufacturer#2 almond aquamarine midnight light salmon 2       3       3       
0.6     0.5     2       3       18.666666666666668      15.86050300449376       
4       2       14
+Manufacturer#2 almond aquamarine rose maroon antique   25      4       4       
0.8     0.75    2       4       20.25   14.00669482783144       4       25      
40
+Manufacturer#2 almond aquamarine sandy cyan gainsboro  18      5       5       
1.0     1.0     3       5       19.8    12.560254774486067      4       18      
2
+Manufacturer#3 almond antique chartreuse khaki white   17      1       1       
0.2     0.0     1       1       17.0    0.0     2       17      17
+Manufacturer#3 almond antique forest lavender goldenrod        14      2       
2       0.4     0.25    1       2       15.5    1.5     2       14      17
+Manufacturer#3 almond antique metallic orange dim      19      3       3       
0.6     0.5     2       3       16.666666666666668      2.0548046676563256      
2       19      17
+Manufacturer#3 almond antique misty red olive  1       4       4       0.8     
0.75    2       4       12.75   7.013380069552769       2       1       14
+Manufacturer#3 almond antique olive coral navajo       45      5       5       
1.0     1.0     3       5       19.2    14.344336861632886      2       45      
19
+Manufacturer#4 almond antique gainsboro frosted violet 10      1       1       
0.2     0.0     1       1       10.0    0.0     0       10      10
+Manufacturer#4 almond antique violet mint lemon        39      2       2       
0.4     0.25    1       2       24.5    14.5    0       39      10
+Manufacturer#4 almond aquamarine floral ivory bisque   27      3       3       
0.6     0.5     2       3       25.333333333333332      11.897712198383164      
0       27      10
+Manufacturer#4 almond aquamarine yellow dodger mint    7       4       4       
0.8     0.75    2       4       20.75   13.007209539328564      0       7       
39
+Manufacturer#4 almond azure aquamarine papaya violet   12      5       5       
1.0     1.0     3       5       19.0    12.149074038789951      0       12      
27
+Manufacturer#5 almond antique blue firebrick mint      31      1       1       
0.2     0.0     1       1       31.0    0.0     1       31      31
+Manufacturer#5 almond antique medium spring khaki      6       2       2       
0.4     0.25    1       2       18.5    12.5    1       6       31
+Manufacturer#5 almond antique sky peru orange  2       3       3       0.6     
0.5     2       3       13.0    12.832251036613439      1       2       31
+Manufacturer#5 almond aquamarine dodger light gainsboro        46      4       
4       0.8     0.75    2       4       21.25   18.102140757380052      1       
46      6
+Manufacturer#5 almond azure blanched chiffon midnight  23      5       5       
1.0     1.0     3       5       21.6    16.206171663906314      1       23      
2
 PREHOOK: query: select  p_mfgr,p_name, p_size,  
   rank() over(distribute by p_mfgr sort by p_name) as r, 
   dense_rank() over(distribute by p_mfgr sort by p_name) as dr, 
@@ -675,32 +675,32 @@ window w1 as (distribute by p_mfgr sort by p_mfgr, p_name 
rows between 2 precedi
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
-Manufacturer#1 almond antique burnished rose metallic  2       1       1       
0       4       4       2
-Manufacturer#1 almond antique burnished rose metallic  2       1       1       
0       4       4       2
-Manufacturer#1 almond antique chartreuse lavender yellow       34      3       
2       0       38      34      2
-Manufacturer#1 almond antique salmon chartreuse burlywood      6       4       
3       0       44      10      2
-Manufacturer#1 almond aquamarine burnished black steel 28      5       4       
0       72      28      34
-Manufacturer#1 almond aquamarine pink moccasin thistle 42      6       5       
1       114     42      6
-Manufacturer#2 almond antique violet chocolate turquoise       14      1       
1       0       14      14      14
-Manufacturer#2 almond antique violet turquoise frosted 40      2       2       
0       54      40      14
-Manufacturer#2 almond aquamarine midnight light salmon 2       3       3       
0       56      2       14
-Manufacturer#2 almond aquamarine rose maroon antique   25      4       4       
0       81      25      40
-Manufacturer#2 almond aquamarine sandy cyan gainsboro  18      5       5       
1       99      32      2
-Manufacturer#3 almond antique chartreuse khaki white   17      1       1       
0       17      31      17
-Manufacturer#3 almond antique forest lavender goldenrod        14      2       
2       0       31      14      17
-Manufacturer#3 almond antique metallic orange dim      19      3       3       
0       50      50      17
-Manufacturer#3 almond antique misty red olive  1       4       4       0       
51      1       14
-Manufacturer#3 almond antique olive coral navajo       45      5       5       
1       96      45      19
-Manufacturer#4 almond antique gainsboro frosted violet 10      1       1       
0       10      17      10
-Manufacturer#4 almond antique violet mint lemon        39      2       2       
0       49      39      10
-Manufacturer#4 almond aquamarine floral ivory bisque   27      3       3       
0       76      27      10
-Manufacturer#4 almond aquamarine yellow dodger mint    7       4       4       
0       83      7       39
-Manufacturer#4 almond azure aquamarine papaya violet   12      5       5       
1       95      29      27
-Manufacturer#5 almond antique blue firebrick mint      31      1       1       
0       31      31      31
-Manufacturer#5 almond antique medium spring khaki      6       2       2       
0       37      8       31
-Manufacturer#5 almond antique sky peru orange  2       3       3       0       
39      2       31
-Manufacturer#5 almond aquamarine dodger light gainsboro        46      4       
4       0       85      46      6
-Manufacturer#5 almond azure blanched chiffon midnight  23      5       5       
1       108     23      2
+Manufacturer#1 almond antique burnished rose metallic  2       1       1       
0.3333333333333333      4       4       2
+Manufacturer#1 almond antique burnished rose metallic  2       1       1       
0.3333333333333333      4       4       2
+Manufacturer#1 almond antique chartreuse lavender yellow       34      3       
2       0.5     38      34      2
+Manufacturer#1 almond antique salmon chartreuse burlywood      6       4       
3       0.6666666666666666      44      10      2
+Manufacturer#1 almond aquamarine burnished black steel 28      5       4       
0.8333333333333334      72      28      34
+Manufacturer#1 almond aquamarine pink moccasin thistle 42      6       5       
1.0     114     42      6
+Manufacturer#2 almond antique violet chocolate turquoise       14      1       
1       0.2     14      14      14
+Manufacturer#2 almond antique violet turquoise frosted 40      2       2       
0.4     54      40      14
+Manufacturer#2 almond aquamarine midnight light salmon 2       3       3       
0.6     56      2       14
+Manufacturer#2 almond aquamarine rose maroon antique   25      4       4       
0.8     81      25      40
+Manufacturer#2 almond aquamarine sandy cyan gainsboro  18      5       5       
1.0     99      32      2
+Manufacturer#3 almond antique chartreuse khaki white   17      1       1       
0.2     17      31      17
+Manufacturer#3 almond antique forest lavender goldenrod        14      2       
2       0.4     31      14      17
+Manufacturer#3 almond antique metallic orange dim      19      3       3       
0.6     50      50      17
+Manufacturer#3 almond antique misty red olive  1       4       4       0.8     
51      1       14
+Manufacturer#3 almond antique olive coral navajo       45      5       5       
1.0     96      45      19
+Manufacturer#4 almond antique gainsboro frosted violet 10      1       1       
0.2     10      17      10
+Manufacturer#4 almond antique violet mint lemon        39      2       2       
0.4     49      39      10
+Manufacturer#4 almond aquamarine floral ivory bisque   27      3       3       
0.6     76      27      10
+Manufacturer#4 almond aquamarine yellow dodger mint    7       4       4       
0.8     83      7       39
+Manufacturer#4 almond azure aquamarine papaya violet   12      5       5       
1.0     95      29      27
+Manufacturer#5 almond antique blue firebrick mint      31      1       1       
0.2     31      31      31
+Manufacturer#5 almond antique medium spring khaki      6       2       2       
0.4     37      8       31
+Manufacturer#5 almond antique sky peru orange  2       3       3       0.6     
39      2       31
+Manufacturer#5 almond aquamarine dodger light gainsboro        46      4       
4       0.8     85      46      6
+Manufacturer#5 almond azure blanched chiffon midnight  23      5       5       
1.0     108     23      2
 PREHOOK: query: select  p_mfgr,p_name, p_size,
 count(*) over(distribute by p_mfgr sort by p_name ) as c, 
 count(p_size) over(distribute by p_mfgr sort by p_name) as ca, 
diff --git 
a/ql/src/test/results/clientpositive/llap/sketches_materialized_view_cume_dist.q.out
 
b/ql/src/test/results/clientpositive/llap/sketches_materialized_view_cume_dist.q.out
new file mode 100644
index 0000000..ae5c3a6
--- /dev/null
+++ 
b/ql/src/test/results/clientpositive/llap/sketches_materialized_view_cume_dist.q.out
@@ -0,0 +1,1054 @@
+PREHOOK: query: create table sketch_input (id int, category char(1))
+STORED AS ORC
+TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@sketch_input
+POSTHOOK: query: create table sketch_input (id int, category char(1))
+STORED AS ORC
+TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@sketch_input
+PREHOOK: query: insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@sketch_input
+POSTHOOK: query: insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@sketch_input
+POSTHOOK: Lineage: sketch_input.category SCRIPT []
+POSTHOOK: Lineage: sketch_input.id SCRIPT []
+PREHOOK: query: create  materialized view mv_1 as
+  select category,ds_kll_sketch(cast(-id as float)) from sketch_input group by 
category
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
+PREHOOK: Input: default@sketch_input
+PREHOOK: Output: database:default
+PREHOOK: Output: default@mv_1
+POSTHOOK: query: create  materialized view mv_1 as
+  select category,ds_kll_sketch(cast(-id as float)) from sketch_input group by 
category
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
+POSTHOOK: Input: default@sketch_input
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@mv_1
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: explain
+select 'rewrite; mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mv_1
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select 'rewrite; mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mv_1
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: id (type: int), 
UDFToFloat(COALESCE(UDFToFloat((- id)),-3.4028234663852886E38D)) (type: float)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 22 Data size: 176 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 22 Data size: 176 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: float)
+            Execution mode: vectorized, llap
+            LLAP IO: may be used (ACID table)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: default.mv_1
+                  Statistics: Num rows: 2 Data size: 240 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: _c1 (type: binary)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 2 Data size: 240 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: ds_kll_union(_col0)
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 22 Data size: 3344 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), (1.0D - ds_kll_cdf(_col2, 
_col1)[0]) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 22 Data size: 264 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Statistics: Num rows: 22 Data size: 264 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: double)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: 'rewrite; mv matching' (type: string), 
KEY.reducesinkkey0 (type: int), VALUE._col0 (type: double)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 22 Data size: 2552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 22 Data size: 2552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: ds_kll_union(VALUE._col0)
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 148 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: binary)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 144 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 144 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: select 'rewrite; mv matching', id, cume_dist() over (order by 
id) from sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mv_1
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select 'rewrite; mv matching', id, cume_dist() over (order by 
id) from sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mv_1
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+rewrite; mv matching   1       0.09090909090909094
+rewrite; mv matching   1       0.09090909090909094
+rewrite; mv matching   2       0.13636363636363635
+rewrite; mv matching   3       0.18181818181818177
+rewrite; mv matching   4       0.2272727272727273
+rewrite; mv matching   5       0.2727272727272727
+rewrite; mv matching   6       0.40909090909090906
+rewrite; mv matching   6       0.40909090909090906
+rewrite; mv matching   6       0.40909090909090906
+rewrite; mv matching   7       0.5
+rewrite; mv matching   7       0.5
+rewrite; mv matching   8       0.5909090909090908
+rewrite; mv matching   8       0.5909090909090908
+rewrite; mv matching   9       0.6818181818181819
+rewrite; mv matching   9       0.6818181818181819
+rewrite; mv matching   10      0.7727272727272727
+rewrite; mv matching   10      0.7727272727272727
+rewrite; mv matching   11      0.8181818181818181
+rewrite; mv matching   12      0.8636363636363636
+rewrite; mv matching   13      0.9090909090909091
+rewrite; mv matching   14      0.9545454545454546
+rewrite; mv matching   15      1.0
+PREHOOK: query: explain
+select 'no rewrite; no mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select 'no rewrite; no mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: 0 (type: int), id (type: int)
+                    null sort order: az
+                    sort order: ++
+                    Map-reduce partition columns: 0 (type: int)
+                    Statistics: Num rows: 22 Data size: 88 Basic stats: 
COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: may be used (ACID table)
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey1 (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 22 Data size: 5984 Basic stats: COMPLETE 
Column stats: COMPLETE
+                PTF Operator
+                  Function definitions:
+                      Input definition
+                        input alias: ptf_0
+                        output shape: _col0: int
+                        type: WINDOWING
+                      Windowing table definition
+                        input alias: ptf_1
+                        name: windowingtablefunction
+                        order by: _col0 ASC NULLS LAST
+                        partition by: 0
+                        raw input shape:
+                        window functions:
+                            window function definition
+                              alias: cume_dist_window_0
+                              arguments: _col0
+                              name: cume_dist
+                              window function: GenericUDAFCumeDistEvaluator
+                              window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
+                              isPivotResult: true
+                  Statistics: Num rows: 22 Data size: 5984 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: int), cume_dist_window_0 (type: 
double)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 22 Data size: 264 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Statistics: Num rows: 22 Data size: 264 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: double)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: 'no rewrite; no mv matching' (type: string), 
KEY.reducesinkkey0 (type: int), VALUE._col0 (type: double)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 22 Data size: 2684 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 22 Data size: 2684 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select 'no rewrite; no mv matching', id, cume_dist() over 
(order by id) from sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select 'no rewrite; no mv matching', id, cume_dist() over 
(order by id) from sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+no rewrite; no mv matching     1       0.09090909090909091
+no rewrite; no mv matching     1       0.09090909090909091
+no rewrite; no mv matching     2       0.13636363636363635
+no rewrite; no mv matching     3       0.18181818181818182
+no rewrite; no mv matching     4       0.22727272727272727
+no rewrite; no mv matching     5       0.2727272727272727
+no rewrite; no mv matching     6       0.4090909090909091
+no rewrite; no mv matching     6       0.4090909090909091
+no rewrite; no mv matching     6       0.4090909090909091
+no rewrite; no mv matching     7       0.5
+no rewrite; no mv matching     7       0.5
+no rewrite; no mv matching     8       0.5909090909090909
+no rewrite; no mv matching     8       0.5909090909090909
+no rewrite; no mv matching     9       0.6818181818181818
+no rewrite; no mv matching     9       0.6818181818181818
+no rewrite; no mv matching     10      0.7727272727272727
+no rewrite; no mv matching     10      0.7727272727272727
+no rewrite; no mv matching     11      0.8181818181818182
+no rewrite; no mv matching     12      0.8636363636363636
+no rewrite; no mv matching     13      0.9090909090909091
+no rewrite; no mv matching     14      0.9545454545454546
+no rewrite; no mv matching     15      1.0
+PREHOOK: query: insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@sketch_input
+POSTHOOK: query: insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@sketch_input
+POSTHOOK: Lineage: sketch_input.category SCRIPT []
+POSTHOOK: Lineage: sketch_input.id SCRIPT []
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: explain
+select 'rewrite; no mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select 'rewrite; no mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 44 Data size: 176 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: id (type: int), 
UDFToFloat(COALESCE(UDFToFloat((- id)),-3.4028234663852886E38D)) (type: float)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 44 Data size: 352 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 44 Data size: 352 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: float)
+                  Select Operator
+                    expressions: UDFToFloat((- id)) (type: float)
+                    outputColumnNames: _col1
+                    Statistics: Num rows: 44 Data size: 176 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: ds_kll_sketch(_col1)
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.97727275
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: may be used (ACID table)
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 44 Data size: 6688 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), (1.0D - ds_kll_cdf(_col2, 
_col1)[0]) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 44 Data size: 528 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Statistics: Num rows: 44 Data size: 528 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: double)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: 'rewrite; no mv matching' (type: string), 
KEY.reducesinkkey0 (type: int), VALUE._col0 (type: double)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 44 Data size: 5236 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 44 Data size: 5236 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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: ds_kll_sketch(VALUE._col0)
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 148 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: binary)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 144 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 144 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: select 'rewrite; no mv matching', id, cume_dist() over (order 
by id) from sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select 'rewrite; no mv matching', id, cume_dist() over (order 
by id) from sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+rewrite; no mv matching        1       0.09090909090909094
+rewrite; no mv matching        1       0.09090909090909094
+rewrite; no mv matching        1       0.09090909090909094
+rewrite; no mv matching        1       0.09090909090909094
+rewrite; no mv matching        2       0.13636363636363635
+rewrite; no mv matching        2       0.13636363636363635
+rewrite; no mv matching        3       0.18181818181818177
+rewrite; no mv matching        3       0.18181818181818177
+rewrite; no mv matching        4       0.2272727272727273
+rewrite; no mv matching        4       0.2272727272727273
+rewrite; no mv matching        5       0.2727272727272727
+rewrite; no mv matching        5       0.2727272727272727
+rewrite; no mv matching        6       0.40909090909090906
+rewrite; no mv matching        6       0.40909090909090906
+rewrite; no mv matching        6       0.40909090909090906
+rewrite; no mv matching        6       0.40909090909090906
+rewrite; no mv matching        6       0.40909090909090906
+rewrite; no mv matching        6       0.40909090909090906
+rewrite; no mv matching        7       0.5
+rewrite; no mv matching        7       0.5
+rewrite; no mv matching        7       0.5
+rewrite; no mv matching        7       0.5
+rewrite; no mv matching        8       0.5909090909090908
+rewrite; no mv matching        8       0.5909090909090908
+rewrite; no mv matching        8       0.5909090909090908
+rewrite; no mv matching        8       0.5909090909090908
+rewrite; no mv matching        9       0.6818181818181819
+rewrite; no mv matching        9       0.6818181818181819
+rewrite; no mv matching        9       0.6818181818181819
+rewrite; no mv matching        9       0.6818181818181819
+rewrite; no mv matching        10      0.7727272727272727
+rewrite; no mv matching        10      0.7727272727272727
+rewrite; no mv matching        10      0.7727272727272727
+rewrite; no mv matching        10      0.7727272727272727
+rewrite; no mv matching        11      0.8181818181818181
+rewrite; no mv matching        11      0.8181818181818181
+rewrite; no mv matching        12      0.8636363636363636
+rewrite; no mv matching        12      0.8636363636363636
+rewrite; no mv matching        13      0.9090909090909091
+rewrite; no mv matching        13      0.9090909090909091
+rewrite; no mv matching        14      0.9545454545454546
+rewrite; no mv matching        14      0.9545454545454546
+rewrite; no mv matching        15      1.0
+rewrite; no mv matching        15      1.0
+PREHOOK: query: explain
+alter materialized view mv_1 rebuild
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mv_1
+PREHOOK: Input: default@sketch_input
+PREHOOK: Output: default@mv_1
+POSTHOOK: query: explain
+alter materialized view mv_1 rebuild
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mv_1
+POSTHOOK: Input: default@sketch_input
+POSTHOOK: Output: default@mv_1
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+  Stage-3 depends on stages: Stage-0
+  Stage-4 depends on stages: Stage-3
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 6 <- Union 3 (CONTAINS)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Union 3 (CONTAINS)
+        Reducer 4 <- Union 3 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  filterExpr: (ROW__ID.writeid > 1L) (type: boolean)
+                  Statistics: Num rows: 44 Data size: 3916 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (ROW__ID.writeid > 1L) (type: boolean)
+                    Statistics: Num rows: 14 Data size: 1246 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: category (type: char(1)), UDFToFloat((- 
id)) (type: float)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 14 Data size: 1246 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: ds_kll_sketch(_col1)
+                        keys: _col0 (type: char(1))
+                        minReductionHashAggr: 0.85714287
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 2 Data size: 458 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: char(1))
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: char(1))
+                          Statistics: Num rows: 2 Data size: 458 Basic stats: 
COMPLETE Column stats: COMPLETE
+                          value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: may be used (ACID table)
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: default.mv_1
+                  Statistics: Num rows: 2 Data size: 410 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: category (type: char(1)), _c1 (type: binary)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 2 Data size: 410 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: ds_kll_union(_col1)
+                      keys: _col0 (type: char(1))
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 2 Data size: 458 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(1))
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: char(1))
+                        Statistics: Num rows: 2 Data size: 458 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: ds_kll_sketch(VALUE._col0)
+                keys: KEY._col0 (type: char(1))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 2 Data size: 458 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Group By Operator
+                  aggregations: ds_kll_union(_col1)
+                  keys: _col0 (type: char(1))
+                  minReductionHashAggr: 0.5
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 2 Data size: 458 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(1))
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(1))
+                    Statistics: Num rows: 2 Data size: 458 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: binary)
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: ds_kll_union(VALUE._col0)
+                keys: KEY._col0 (type: char(1))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 2 Data size: 458 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 2 Data size: 458 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  table:
+                      input format: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: 
org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.mv_1
+                Select Operator
+                  expressions: _col0 (type: char(1)), _col1 (type: binary)
+                  outputColumnNames: category, _c1
+                  Statistics: Num rows: 2 Data size: 458 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: max(length(category)), 
avg(COALESCE(length(category),0)), count(1), count(category), 
compute_bit_vector(category, 'hll'), max(length(_c1)), 
avg(COALESCE(length(_c1),0)), count(_c1)
+                    minReductionHashAggr: 0.5
+                    mode: hash
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6, _col7
+                    Statistics: Num rows: 1 Data size: 328 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 328 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: 
struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: 
bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: 
struct<count:bigint,sum:double,input:int>), _col7 (type: bigint)
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: max(VALUE._col0), avg(VALUE._col1), 
count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), 
max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, 
_col6, _col7
+                Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: 'STRING' (type: string), 
UDFToLong(COALESCE(_col0,0)) (type: bigint), COALESCE(_col1,0) (type: double), 
(_col2 - _col3) (type: bigint), COALESCE(ndv_compute_bit_vector(_col4),0) 
(type: bigint), _col4 (type: binary), 'BINARY' (type: string), 
UDFToLong(COALESCE(_col5,0)) (type: bigint), COALESCE(_col6,0) (type: double), 
(_col2 - _col7) (type: bigint)
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, 
_col6, _col7, _col8, _col9
+                  Statistics: Num rows: 1 Data size: 380 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 380 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    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
+        Union 3 
+            Vertex: Union 3
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: true
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.mv_1
+
+  Stage: Stage-3
+    Stats Work
+      Basic Stats Work:
+      Column Stats Desc:
+          Columns: category, _c1
+          Column Types: char(1), binary
+          Table: default.mv_1
+
+  Stage: Stage-4
+    Materialized View Update
+      name: default.mv_1
+      update creation metadata: true
+
+PREHOOK: query: alter materialized view mv_1 rebuild
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mv_1
+PREHOOK: Input: default@sketch_input
+PREHOOK: Output: default@mv_1
+POSTHOOK: query: alter materialized view mv_1 rebuild
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mv_1
+POSTHOOK: Input: default@sketch_input
+POSTHOOK: Output: default@mv_1
+POSTHOOK: Lineage: mv_1._c1 EXPRESSION 
[(sketch_input)sketch_input.FieldSchema(name:id, type:int, comment:null), 
(mv_1)default.mv_1.FieldSchema(name:_c1, type:binary, comment:null), ]
+POSTHOOK: Lineage: mv_1.category EXPRESSION 
[(sketch_input)sketch_input.FieldSchema(name:category, type:char(1), 
comment:null), (mv_1)default.mv_1.FieldSchema(name:category, type:char(1), 
comment:null), ]
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: explain
+select 'rewrite; mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mv_1
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select 'rewrite; mv matching', id, cume_dist() over (order by id) from 
sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mv_1
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 44 Data size: 176 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: id (type: int), 
UDFToFloat(COALESCE(UDFToFloat((- id)),-3.4028234663852886E38D)) (type: float)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 44 Data size: 352 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 44 Data size: 352 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: float)
+            Execution mode: vectorized, llap
+            LLAP IO: may be used (ACID table)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: default.mv_1
+                  Statistics: Num rows: 2 Data size: 320 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: _c1 (type: binary)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 2 Data size: 320 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: ds_kll_union(_col0)
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 44 Data size: 6688 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), (1.0D - ds_kll_cdf(_col2, 
_col1)[0]) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 44 Data size: 528 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Statistics: Num rows: 44 Data size: 528 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: double)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: 'rewrite; mv matching' (type: string), 
KEY.reducesinkkey0 (type: int), VALUE._col0 (type: double)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 44 Data size: 5104 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 44 Data size: 5104 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: ds_kll_union(VALUE._col0)
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 148 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: binary)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 144 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 144 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: select 'rewrite; mv matching', id, cume_dist() over (order by 
id) from sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mv_1
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select 'rewrite; mv matching', id, cume_dist() over (order by 
id) from sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mv_1
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+rewrite; mv matching   1       0.09090909090909094
+rewrite; mv matching   1       0.09090909090909094
+rewrite; mv matching   1       0.09090909090909094
+rewrite; mv matching   1       0.09090909090909094
+rewrite; mv matching   2       0.13636363636363635
+rewrite; mv matching   2       0.13636363636363635
+rewrite; mv matching   3       0.18181818181818177
+rewrite; mv matching   3       0.18181818181818177
+rewrite; mv matching   4       0.2272727272727273
+rewrite; mv matching   4       0.2272727272727273
+rewrite; mv matching   5       0.2727272727272727
+rewrite; mv matching   5       0.2727272727272727
+rewrite; mv matching   6       0.40909090909090906
+rewrite; mv matching   6       0.40909090909090906
+rewrite; mv matching   6       0.40909090909090906
+rewrite; mv matching   6       0.40909090909090906
+rewrite; mv matching   6       0.40909090909090906
+rewrite; mv matching   6       0.40909090909090906
+rewrite; mv matching   7       0.5
+rewrite; mv matching   7       0.5
+rewrite; mv matching   7       0.5
+rewrite; mv matching   7       0.5
+rewrite; mv matching   8       0.5909090909090908
+rewrite; mv matching   8       0.5909090909090908
+rewrite; mv matching   8       0.5909090909090908
+rewrite; mv matching   8       0.5909090909090908
+rewrite; mv matching   9       0.6818181818181819
+rewrite; mv matching   9       0.6818181818181819
+rewrite; mv matching   9       0.6818181818181819
+rewrite; mv matching   9       0.6818181818181819
+rewrite; mv matching   10      0.7727272727272727
+rewrite; mv matching   10      0.7727272727272727
+rewrite; mv matching   10      0.7727272727272727
+rewrite; mv matching   10      0.7727272727272727
+rewrite; mv matching   11      0.8181818181818181
+rewrite; mv matching   11      0.8181818181818181
+rewrite; mv matching   12      0.8636363636363636
+rewrite; mv matching   12      0.8636363636363636
+rewrite; mv matching   13      0.9090909090909091
+rewrite; mv matching   13      0.9090909090909091
+rewrite; mv matching   14      0.9545454545454546
+rewrite; mv matching   14      0.9545454545454546
+rewrite; mv matching   15      1.0
+rewrite; mv matching   15      1.0
+PREHOOK: query: explain
+select 'rewrite; mv matching', category, id, cume_dist() over (partition by 
category order by id) from sketch_input order by category,id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mv_1
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select 'rewrite; mv matching', category, id, cume_dist() over (partition by 
category order by id) from sketch_input order by category,id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mv_1
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 44 Data size: 3916 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: id (type: int), category (type: char(1)), 
UDFToFloat(COALESCE(UDFToFloat((- id)),-3.4028234663852886E38D)) (type: float)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 44 Data size: 4092 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col1 (type: char(1))
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: char(1))
+                      Statistics: Num rows: 44 Data size: 4092 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col2 (type: float)
+            Execution mode: vectorized, llap
+            LLAP IO: may be used (ACID table)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: default.mv_1
+                  Statistics: Num rows: 2 Data size: 490 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: category (type: char(1))
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: category (type: char(1))
+                    Statistics: Num rows: 2 Data size: 490 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _c1 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col1 (type: char(1))
+                  1 category (type: char(1))
+                nullSafes: [true]
+                outputColumnNames: _col0, _col1, _col2, _col4
+                Statistics: Num rows: 44 Data size: 11132 Basic stats: 
COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: char(1)), _col0 (type: int), (1.0D 
- ds_kll_cdf(_col4, _col2)[0]) (type: double)
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 44 Data size: 4268 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(1)), _col1 (type: int)
+                    null sort order: zz
+                    sort order: ++
+                    Statistics: Num rows: 44 Data size: 4268 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col2 (type: double)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: 'rewrite; mv matching' (type: string), 
KEY.reducesinkkey0 (type: char(1)), KEY.reducesinkkey1 (type: int), VALUE._col0 
(type: double)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 44 Data size: 8844 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 44 Data size: 8844 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select 'rewrite; mv matching', category, id, cume_dist() over 
(partition by category order by id) from sketch_input order by category,id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@mv_1
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select 'rewrite; mv matching', category, id, cume_dist() over 
(partition by category order by id) from sketch_input order by category,id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@mv_1
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+rewrite; mv matching   a       1       0.18181818181818177
+rewrite; mv matching   a       1       0.18181818181818177
+rewrite; mv matching   a       1       0.18181818181818177
+rewrite; mv matching   a       1       0.18181818181818177
+rewrite; mv matching   a       2       0.2727272727272727
+rewrite; mv matching   a       2       0.2727272727272727
+rewrite; mv matching   a       3       0.36363636363636365
+rewrite; mv matching   a       3       0.36363636363636365
+rewrite; mv matching   a       4       0.4545454545454546
+rewrite; mv matching   a       4       0.4545454545454546
+rewrite; mv matching   a       5       0.5454545454545454
+rewrite; mv matching   a       5       0.5454545454545454
+rewrite; mv matching   a       6       0.6363636363636364
+rewrite; mv matching   a       6       0.6363636363636364
+rewrite; mv matching   a       7       0.7272727272727273
+rewrite; mv matching   a       7       0.7272727272727273
+rewrite; mv matching   a       8       0.8181818181818181
+rewrite; mv matching   a       8       0.8181818181818181
+rewrite; mv matching   a       9       0.9090909090909091
+rewrite; mv matching   a       9       0.9090909090909091
+rewrite; mv matching   a       10      1.0
+rewrite; mv matching   a       10      1.0
+rewrite; mv matching   b       6       0.18181818181818177
+rewrite; mv matching   b       6       0.18181818181818177
+rewrite; mv matching   b       6       0.18181818181818177
+rewrite; mv matching   b       6       0.18181818181818177
+rewrite; mv matching   b       7       0.2727272727272727
+rewrite; mv matching   b       7       0.2727272727272727
+rewrite; mv matching   b       8       0.36363636363636365
+rewrite; mv matching   b       8       0.36363636363636365
+rewrite; mv matching   b       9       0.4545454545454546
+rewrite; mv matching   b       9       0.4545454545454546
+rewrite; mv matching   b       10      0.5454545454545454
+rewrite; mv matching   b       10      0.5454545454545454
+rewrite; mv matching   b       11      0.6363636363636364
+rewrite; mv matching   b       11      0.6363636363636364
+rewrite; mv matching   b       12      0.7272727272727273
+rewrite; mv matching   b       12      0.7272727272727273
+rewrite; mv matching   b       13      0.8181818181818181
+rewrite; mv matching   b       13      0.8181818181818181
+rewrite; mv matching   b       14      0.9090909090909091
+rewrite; mv matching   b       14      0.9090909090909091
+rewrite; mv matching   b       15      1.0
+rewrite; mv matching   b       15      1.0
+PREHOOK: query: drop materialized view mv_1
+PREHOOK: type: DROP_MATERIALIZED_VIEW
+PREHOOK: Input: default@mv_1
+PREHOOK: Output: default@mv_1
+POSTHOOK: query: drop materialized view mv_1
+POSTHOOK: type: DROP_MATERIALIZED_VIEW
+POSTHOOK: Input: default@mv_1
+POSTHOOK: Output: default@mv_1
diff --git 
a/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist.q.out 
b/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist.q.out
new file mode 100644
index 0000000..372a0b0
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist.q.out
@@ -0,0 +1,775 @@
+PREHOOK: query: create table sketch_input (id int, category char(1))
+STORED AS ORC
+TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@sketch_input
+POSTHOOK: query: create table sketch_input (id int, category char(1))
+STORED AS ORC
+TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@sketch_input
+PREHOOK: query: insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@sketch_input
+POSTHOOK: query: insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@sketch_input
+POSTHOOK: Lineage: sketch_input.category SCRIPT []
+POSTHOOK: Lineage: sketch_input.id SCRIPT []
+PREHOOK: query: select id,cume_dist() over (order by id) from sketch_input
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,cume_dist() over (order by id) from sketch_input
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+1      0.09090909090909091
+1      0.09090909090909091
+2      0.13636363636363635
+3      0.18181818181818182
+4      0.22727272727272727
+5      0.2727272727272727
+6      0.4090909090909091
+6      0.4090909090909091
+6      0.4090909090909091
+7      0.5
+7      0.5
+8      0.5909090909090909
+8      0.5909090909090909
+9      0.6818181818181818
+9      0.6818181818181818
+10     0.7727272727272727
+10     0.7727272727272727
+11     0.8181818181818182
+12     0.8636363636363636
+13     0.9090909090909091
+14     0.9545454545454546
+15     1.0
+Warning: Shuffle Join MERGEJOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: select id,cume_dist() over (order by id),1.0-ds_kll_cdf(ds, 
CAST(-id AS FLOAT) )[0]
+from sketch_input
+join ( select ds_kll_sketch(cast(-id as float)) as ds from sketch_input ) q
+order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,cume_dist() over (order by id),1.0-ds_kll_cdf(ds, 
CAST(-id AS FLOAT) )[0]
+from sketch_input
+join ( select ds_kll_sketch(cast(-id as float)) as ds from sketch_input ) q
+order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+1      0.09090909090909091     0.09090909090909094
+1      0.09090909090909091     0.09090909090909094
+2      0.13636363636363635     0.13636363636363635
+3      0.18181818181818182     0.18181818181818177
+4      0.22727272727272727     0.2272727272727273
+5      0.2727272727272727      0.2727272727272727
+6      0.4090909090909091      0.40909090909090906
+6      0.4090909090909091      0.40909090909090906
+6      0.4090909090909091      0.40909090909090906
+7      0.5     0.5
+7      0.5     0.5
+8      0.5909090909090909      0.5909090909090908
+8      0.5909090909090909      0.5909090909090908
+9      0.6818181818181818      0.6818181818181819
+9      0.6818181818181818      0.6818181818181819
+10     0.7727272727272727      0.7727272727272727
+10     0.7727272727272727      0.7727272727272727
+11     0.8181818181818182      0.8181818181818181
+12     0.8636363636363636      0.8636363636363636
+13     0.9090909090909091      0.9090909090909091
+14     0.9545454545454546      0.9545454545454546
+15     1.0     1.0
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: explain
+select id,'rewrite',cume_dist() over (order by id) from sketch_input order by 
id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select id,'rewrite',cume_dist() over (order by id) from sketch_input order by 
id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: id (type: int), 
UDFToFloat(COALESCE(UDFToFloat((- id)),-3.4028234663852886E38D)) (type: float)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 22 Data size: 176 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 22 Data size: 176 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: float)
+                  Select Operator
+                    expressions: UDFToFloat((- id)) (type: float)
+                    outputColumnNames: _col1
+                    Statistics: Num rows: 22 Data size: 88 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: ds_kll_sketch(_col1)
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.95454544
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: may be used (ACID table)
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 22 Data size: 3344 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), (1.0D - ds_kll_cdf(_col2, 
_col1)[0]) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 22 Data size: 264 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Statistics: Num rows: 22 Data size: 264 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: double)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: int), 'rewrite' (type: 
string), VALUE._col0 (type: double)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 22 Data size: 2266 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 22 Data size: 2266 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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: ds_kll_sketch(VALUE._col0)
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 148 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: binary)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 144 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 144 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: select id,'rewrite',cume_dist() over (order by id) from 
sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,'rewrite',cume_dist() over (order by id) from 
sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+1      rewrite 0.09090909090909094
+1      rewrite 0.09090909090909094
+2      rewrite 0.13636363636363635
+3      rewrite 0.18181818181818177
+4      rewrite 0.2272727272727273
+5      rewrite 0.2727272727272727
+6      rewrite 0.40909090909090906
+6      rewrite 0.40909090909090906
+6      rewrite 0.40909090909090906
+7      rewrite 0.5
+7      rewrite 0.5
+8      rewrite 0.5909090909090908
+8      rewrite 0.5909090909090908
+9      rewrite 0.6818181818181819
+9      rewrite 0.6818181818181819
+10     rewrite 0.7727272727272727
+10     rewrite 0.7727272727272727
+11     rewrite 0.8181818181818181
+12     rewrite 0.8636363636363636
+13     rewrite 0.9090909090909091
+14     rewrite 0.9545454545454546
+15     rewrite 1.0
+Warning: Shuffle Join MERGEJOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: explain
+select id,'rewrite',count(id) over ()*cume_dist() over (order by id) from 
sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select id,'rewrite',count(id) over ()*cume_dist() over (order by id) from 
sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: id (type: int), 
UDFToFloat(COALESCE(UDFToFloat((- id)),-3.4028234663852886E38D)) (type: float)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 22 Data size: 176 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 22 Data size: 176 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: float)
+                  Select Operator
+                    expressions: UDFToFloat((- id)) (type: float)
+                    outputColumnNames: _col1
+                    Statistics: Num rows: 22 Data size: 88 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: ds_kll_sketch(_col1)
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.95454544
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: may be used (ACID table)
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 22 Data size: 3344 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: 0 (type: int)
+                  null sort order: a
+                  sort order: +
+                  Map-reduce partition columns: 0 (type: int)
+                  Statistics: Num rows: 22 Data size: 3344 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col1 (type: float), 
_col2 (type: binary)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: int), VALUE._col1 (type: 
float), VALUE._col2 (type: binary)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 22 Data size: 3344 Basic stats: COMPLETE 
Column stats: COMPLETE
+                PTF Operator
+                  Function definitions:
+                      Input definition
+                        input alias: ptf_0
+                        output shape: _col0: int, _col1: float, _col2: binary
+                        type: WINDOWING
+                      Windowing table definition
+                        input alias: ptf_1
+                        name: windowingtablefunction
+                        order by: 0 ASC NULLS FIRST
+                        partition by: 0
+                        raw input shape:
+                        window functions:
+                            window function definition
+                              alias: count_window_0
+                              arguments: _col0
+                              name: count
+                              window function: GenericUDAFCountEvaluator
+                              window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
+                  Statistics: Num rows: 22 Data size: 3344 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: int), 
(UDFToDouble(count_window_0) * (1.0D - ds_kll_cdf(_col2, _col1)[0])) (type: 
double)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 22 Data size: 264 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Statistics: Num rows: 22 Data size: 264 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: double)
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: int), 'rewrite' (type: 
string), VALUE._col0 (type: double)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 22 Data size: 2266 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 22 Data size: 2266 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: ds_kll_sketch(VALUE._col0)
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 148 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: binary)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 144 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 144 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: select id,'rewrite',count(id) over ()*cume_dist() over (order 
by id) from sketch_input order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,'rewrite',count(id) over ()*cume_dist() over (order 
by id) from sketch_input order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+1      rewrite 2.000000000000001
+1      rewrite 2.000000000000001
+2      rewrite 3.0
+3      rewrite 3.999999999999999
+4      rewrite 5.0
+5      rewrite 6.0
+6      rewrite 9.0
+6      rewrite 9.0
+6      rewrite 9.0
+7      rewrite 11.0
+7      rewrite 11.0
+8      rewrite 12.999999999999998
+8      rewrite 12.999999999999998
+9      rewrite 15.000000000000002
+9      rewrite 15.000000000000002
+10     rewrite 17.0
+10     rewrite 17.0
+11     rewrite 18.0
+12     rewrite 19.0
+13     rewrite 20.0
+14     rewrite 21.0
+15     rewrite 22.0
+PREHOOK: query: insert into sketch_input values (null,'a'),(null,'b')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@sketch_input
+POSTHOOK: query: insert into sketch_input values (null,'a'),(null,'b')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@sketch_input
+POSTHOOK: Lineage: sketch_input.category SCRIPT []
+POSTHOOK: Lineage: sketch_input.id EXPRESSION []
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: explain
+select id,'rewrite',cume_dist() over (order by id nulls first) from 
sketch_input order by id nulls first
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select id,'rewrite',cume_dist() over (order by id nulls first) from 
sketch_input order by id nulls first
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 24 Data size: 92 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: id (type: int), 
UDFToFloat(COALESCE(UDFToFloat((- id)),3.4028234663852886E38D)) (type: float)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 24 Data size: 188 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 24 Data size: 188 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: float)
+                  Select Operator
+                    expressions: UDFToFloat((- id)) (type: float)
+                    outputColumnNames: _col1
+                    Statistics: Num rows: 24 Data size: 92 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: ds_kll_sketch(_col1)
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.9583333
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: may be used (ACID table)
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 24 Data size: 3644 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), (1.0D - ds_kll_cdf(_col2, 
_col1)[0]) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 24 Data size: 284 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: a
+                    sort order: +
+                    Statistics: Num rows: 24 Data size: 284 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: double)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: int), 'rewrite' (type: 
string), VALUE._col0 (type: double)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 24 Data size: 2468 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 24 Data size: 2468 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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: ds_kll_sketch(VALUE._col0)
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 148 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: binary)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 144 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 144 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: select id,'rewrite',cume_dist() over (order by id nulls first) 
from sketch_input order by id nulls first
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,'rewrite',cume_dist() over (order by id nulls 
first) from sketch_input order by id nulls first
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+NULL   rewrite 0.0
+NULL   rewrite 0.0
+1      rewrite 0.09090909090909094
+1      rewrite 0.09090909090909094
+2      rewrite 0.13636363636363635
+3      rewrite 0.18181818181818177
+4      rewrite 0.2272727272727273
+5      rewrite 0.2727272727272727
+6      rewrite 0.40909090909090906
+6      rewrite 0.40909090909090906
+6      rewrite 0.40909090909090906
+7      rewrite 0.5
+7      rewrite 0.5
+8      rewrite 0.5909090909090908
+8      rewrite 0.5909090909090908
+9      rewrite 0.6818181818181819
+9      rewrite 0.6818181818181819
+10     rewrite 0.7727272727272727
+10     rewrite 0.7727272727272727
+11     rewrite 0.8181818181818181
+12     rewrite 0.8636363636363636
+13     rewrite 0.9090909090909091
+14     rewrite 0.9545454545454546
+15     rewrite 1.0
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: explain
+select id,'rewrite',cume_dist() over (order by id nulls last) from 
sketch_input order by id nulls last
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select id,'rewrite',cume_dist() over (order by id nulls last) from 
sketch_input order by id nulls last
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 24 Data size: 92 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: id (type: int), 
UDFToFloat(COALESCE(UDFToFloat((- id)),-3.4028234663852886E38D)) (type: float)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 24 Data size: 188 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 24 Data size: 188 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: float)
+                  Select Operator
+                    expressions: UDFToFloat((- id)) (type: float)
+                    outputColumnNames: _col1
+                    Statistics: Num rows: 24 Data size: 92 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: ds_kll_sketch(_col1)
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.9583333
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 148 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: may be used (ACID table)
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 24 Data size: 3644 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), (1.0D - ds_kll_cdf(_col2, 
_col1)[0]) (type: double)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 24 Data size: 284 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Statistics: Num rows: 24 Data size: 284 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: double)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: int), 'rewrite' (type: 
string), VALUE._col0 (type: double)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 24 Data size: 2468 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 24 Data size: 2468 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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: ds_kll_sketch(VALUE._col0)
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 148 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: binary)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 144 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 144 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: select id,'rewrite',cume_dist() over (order by id nulls last) 
from sketch_input order by id nulls last
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,'rewrite',cume_dist() over (order by id nulls last) 
from sketch_input order by id nulls last
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+1      rewrite 0.09090909090909094
+1      rewrite 0.09090909090909094
+2      rewrite 0.13636363636363635
+3      rewrite 0.18181818181818177
+4      rewrite 0.2272727272727273
+5      rewrite 0.2727272727272727
+6      rewrite 0.40909090909090906
+6      rewrite 0.40909090909090906
+6      rewrite 0.40909090909090906
+7      rewrite 0.5
+7      rewrite 0.5
+8      rewrite 0.5909090909090908
+8      rewrite 0.5909090909090908
+9      rewrite 0.6818181818181819
+9      rewrite 0.6818181818181819
+10     rewrite 0.7727272727272727
+10     rewrite 0.7727272727272727
+11     rewrite 0.8181818181818181
+12     rewrite 0.8636363636363636
+13     rewrite 0.9090909090909091
+14     rewrite 0.9545454545454546
+15     rewrite 1.0
+NULL   rewrite 1.0
+NULL   rewrite 1.0
+Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
+PREHOOK: query: select id,cume_dist() over (order by id) from sketch_input 
order by id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,cume_dist() over (order by id) from sketch_input 
order by id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+1      0.09090909090909094
+1      0.09090909090909094
+2      0.13636363636363635
+3      0.18181818181818177
+4      0.2272727272727273
+5      0.2727272727272727
+6      0.40909090909090906
+6      0.40909090909090906
+6      0.40909090909090906
+7      0.5
+7      0.5
+8      0.5909090909090908
+8      0.5909090909090908
+9      0.6818181818181819
+9      0.6818181818181819
+10     0.7727272727272727
+10     0.7727272727272727
+11     0.8181818181818181
+12     0.8636363636363636
+13     0.9090909090909091
+14     0.9545454545454546
+15     1.0
+NULL   1.0
+NULL   1.0
diff --git 
a/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist_partition_by.q.out
 
b/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist_partition_by.q.out
new file mode 100644
index 0000000..19f9dbe
--- /dev/null
+++ 
b/ql/src/test/results/clientpositive/llap/sketches_rewrite_cume_dist_partition_by.q.out
@@ -0,0 +1,258 @@
+PREHOOK: query: create table sketch_input (id int, category char(1))
+STORED AS ORC
+TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@sketch_input
+POSTHOOK: query: create table sketch_input (id int, category char(1))
+STORED AS ORC
+TBLPROPERTIES ('transactional'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@sketch_input
+PREHOOK: query: insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b'),
+  (1,null),(2,null),(10,null),(13,null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@sketch_input
+POSTHOOK: query: insert into table sketch_input values
+  (1,'a'),(1, 'a'), (2, 'a'), (3, 'a'), (4, 'a'), (5, 'a'), (6, 'a'), (7, 
'a'), (8, 'a'), (9, 'a'), (10, 'a'),
+  (6,'b'),(6, 'b'), (7, 'b'), (8, 'b'), (9, 'b'), (10, 'b'), (11, 'b'), (12, 
'b'), (13, 'b'), (14, 'b'), (15, 'b'),
+  (1,null),(2,null),(10,null),(13,null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@sketch_input
+POSTHOOK: Lineage: sketch_input.category SCRIPT []
+POSTHOOK: Lineage: sketch_input.id SCRIPT []
+PREHOOK: query: select id,category,cume_dist() over (partition by category 
order by id) from sketch_input order by category,id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,category,cume_dist() over (partition by category 
order by id) from sketch_input order by category,id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+1      a       0.18181818181818182
+1      a       0.18181818181818182
+2      a       0.2727272727272727
+3      a       0.36363636363636365
+4      a       0.45454545454545453
+5      a       0.5454545454545454
+6      a       0.6363636363636364
+7      a       0.7272727272727273
+8      a       0.8181818181818182
+9      a       0.9090909090909091
+10     a       1.0
+6      b       0.18181818181818182
+6      b       0.18181818181818182
+7      b       0.2727272727272727
+8      b       0.36363636363636365
+9      b       0.45454545454545453
+10     b       0.5454545454545454
+11     b       0.6363636363636364
+12     b       0.7272727272727273
+13     b       0.8181818181818182
+14     b       0.9090909090909091
+15     b       1.0
+1      NULL    0.25
+2      NULL    0.5
+10     NULL    0.75
+13     NULL    1.0
+PREHOOK: query: select id,category,cume_dist() over (partition by category 
order by id),1.0-ds_kll_cdf(ds, CAST(-id AS FLOAT))[0]
+from sketch_input
+join ( select category as c,ds_kll_sketch(cast(-id as float)) as ds from 
sketch_input group by category) q on (q.c=category)
+order by category,id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,category,cume_dist() over (partition by category 
order by id),1.0-ds_kll_cdf(ds, CAST(-id AS FLOAT))[0]
+from sketch_input
+join ( select category as c,ds_kll_sketch(cast(-id as float)) as ds from 
sketch_input group by category) q on (q.c=category)
+order by category,id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+1      a       0.18181818181818182     0.18181818181818177
+1      a       0.18181818181818182     0.18181818181818177
+2      a       0.2727272727272727      0.2727272727272727
+3      a       0.36363636363636365     0.36363636363636365
+4      a       0.45454545454545453     0.4545454545454546
+5      a       0.5454545454545454      0.5454545454545454
+6      a       0.6363636363636364      0.6363636363636364
+7      a       0.7272727272727273      0.7272727272727273
+8      a       0.8181818181818182      0.8181818181818181
+9      a       0.9090909090909091      0.9090909090909091
+10     a       1.0     1.0
+6      b       0.18181818181818182     0.18181818181818177
+6      b       0.18181818181818182     0.18181818181818177
+7      b       0.2727272727272727      0.2727272727272727
+8      b       0.36363636363636365     0.36363636363636365
+9      b       0.45454545454545453     0.4545454545454546
+10     b       0.5454545454545454      0.5454545454545454
+11     b       0.6363636363636364      0.6363636363636364
+12     b       0.7272727272727273      0.7272727272727273
+13     b       0.8181818181818182      0.8181818181818181
+14     b       0.9090909090909091      0.9090909090909091
+15     b       1.0     1.0
+PREHOOK: query: explain
+select id,'rewrite',cume_dist() over (partition by category order by id) from 
sketch_input order by category,id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select id,'rewrite',cume_dist() over (partition by category order by id) from 
sketch_input order by category,id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+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:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 26 Data size: 2059 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: id (type: int), category (type: char(1)), 
UDFToFloat(COALESCE(UDFToFloat((- id)),-3.4028234663852886E38D)) (type: float)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 26 Data size: 2163 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col1 (type: char(1))
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: char(1))
+                      Statistics: Num rows: 26 Data size: 2163 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col2 (type: float)
+            Execution mode: vectorized, llap
+            LLAP IO: may be used (ACID table)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: sketch_input
+                  Statistics: Num rows: 26 Data size: 2059 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: category (type: char(1)), UDFToFloat((- id)) 
(type: float)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 26 Data size: 2059 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: ds_kll_sketch(_col1)
+                      keys: _col0 (type: char(1))
+                      minReductionHashAggr: 0.9230769
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 3 Data size: 687 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(1))
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: char(1))
+                        Statistics: Num rows: 3 Data size: 687 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: binary)
+            Execution mode: llap
+            LLAP IO: may be used (ACID table)
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col1 (type: char(1))
+                  1 _col0 (type: char(1))
+                nullSafes: [true]
+                outputColumnNames: _col0, _col1, _col2, _col4
+                Statistics: Num rows: 26 Data size: 5907 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), (1.0D - ds_kll_cdf(_col4, 
_col2)[0]) (type: double), _col1 (type: char(1))
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 26 Data size: 2267 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col2 (type: char(1)), _col0 (type: int)
+                    null sort order: zz
+                    sort order: ++
+                    Statistics: Num rows: 26 Data size: 2267 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: double)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey1 (type: int), 'rewrite' (type: 
string), VALUE._col0 (type: double)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 26 Data size: 2678 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 26 Data size: 2678 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  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 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: ds_kll_sketch(VALUE._col0)
+                keys: KEY._col0 (type: char(1))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 2 Data size: 458 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: char(1))
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: char(1))
+                  Statistics: Num rows: 2 Data size: 458 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  value expressions: _col1 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select id,'rewrite',cume_dist() over (partition by category 
order by id) from sketch_input order by category,id
+PREHOOK: type: QUERY
+PREHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+POSTHOOK: query: select id,'rewrite',cume_dist() over (partition by category 
order by id) from sketch_input order by category,id
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@sketch_input
+#### A masked pattern was here ####
+1      rewrite 0.18181818181818177
+1      rewrite 0.18181818181818177
+2      rewrite 0.2727272727272727
+3      rewrite 0.36363636363636365
+4      rewrite 0.4545454545454546
+5      rewrite 0.5454545454545454
+6      rewrite 0.6363636363636364
+7      rewrite 0.7272727272727273
+8      rewrite 0.8181818181818181
+9      rewrite 0.9090909090909091
+10     rewrite 1.0
+6      rewrite 0.18181818181818177
+6      rewrite 0.18181818181818177
+7      rewrite 0.2727272727272727
+8      rewrite 0.36363636363636365
+9      rewrite 0.4545454545454546
+10     rewrite 0.5454545454545454
+11     rewrite 0.6363636363636364
+12     rewrite 0.7272727272727273
+13     rewrite 0.8181818181818181
+14     rewrite 0.9090909090909091
+15     rewrite 1.0
+1      rewrite 0.25
+2      rewrite 0.5
+10     rewrite 0.75
+13     rewrite 1.0

Reply via email to