You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2016/12/16 13:58:27 UTC

[1/6] hive git commit: HIVE-14496: Enable Calcite rewriting with materialized views (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master bf37d787e -> 3da29fe7e


http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/subquery_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_views.q.out b/ql/src/test/results/clientpositive/subquery_views.q.out
index 610bf24..3c61924 100644
--- a/ql/src/test/results/clientpositive/subquery_views.q.out
+++ b/ql/src/test/results/clientpositive/subquery_views.q.out
@@ -45,7 +45,7 @@ from `default`.`src` `b`
 where exists	 	 
   (select `a`.`key` 	 	 
   from `default`.`src` `a` 	 	 
-  where `b`.`value` = `a`.`value`  and `a`.`key` = `b`.`key` and `a`.`value` > 'val_9'), tableType:VIRTUAL_VIEW)		 
+  where `b`.`value` = `a`.`value`  and `a`.`key` = `b`.`key` and `a`.`value` > 'val_9'), rewriteEnabled:false), tableType:VIRTUAL_VIEW)		 
 PREHOOK: query: select * 
 from cv1 where cv1.key in (select key from cv1 c where c.key > '95')
 PREHOOK: type: QUERY
@@ -110,7 +110,7 @@ where `b`.`key` not in
   (select `a`.`key` 	 	 
   from `default`.`src` `a` 	 	 
   where `b`.`value` = `a`.`value`  and `a`.`key` = `b`.`key` and `a`.`value` > 'val_11'	 	 
-  ), tableType:VIRTUAL_VIEW)		 
+  ), rewriteEnabled:false), tableType:VIRTUAL_VIEW)		 
 Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 Warning: Shuffle Join JOIN[40][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-6:MAPRED' is a cross product
 PREHOOK: query: explain
@@ -489,7 +489,7 @@ having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.k
 from `default`.`src` `b`	 	 
 where `b`.`key` in (select `src`.`key` from `default`.`src` where `src`.`key` > '8')	 	 
 group by `b`.`key`, `b`.`value`	 	 
-having count(*) in (select count(*) from `default`.`src` `s1` where `s1`.`key` > '9' group by `s1`.`key` ), tableType:VIRTUAL_VIEW)		 
+having count(*) in (select count(*) from `default`.`src` `s1` where `s1`.`key` > '9' group by `s1`.`key` ), rewriteEnabled:false), tableType:VIRTUAL_VIEW)		 
 PREHOOK: query: select * from cv3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@cv3

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out b/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out
index 304d74f..10609d9 100644
--- a/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/unionDistinct_2.q.out
@@ -365,7 +365,7 @@ union
 select `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
 union all	 	 
 select `u3`.`key` as `key`, `u3`.`value` from `default`.`u3`	 	 
-) `tab`, tableType:VIRTUAL_VIEW)		 
+) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -435,7 +435,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union  	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW)		 
+) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -500,7 +500,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union all 	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW)		 
+) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/unicode_comments.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/unicode_comments.q.out b/ql/src/test/results/clientpositive/unicode_comments.q.out
index 4872cd3..19413b9 100644
--- a/ql/src/test/results/clientpositive/unicode_comments.q.out
+++ b/ql/src/test/results/clientpositive/unicode_comments.q.out
@@ -142,6 +142,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select col1 from unicode_comments_tbl1	 
 View Expanded Text: 	SELECT `col1` AS `col1` FROM (select `unicode_comments_tbl1`.`col1` from `unicode_comments_db`.`unicode_comments_tbl1`) `unicode_comments_db.unicode_comments_view1`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: show formatted index on unicode_comments_tbl1
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: show formatted index on unicode_comments_tbl1

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/view_alias.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/view_alias.q.out b/ql/src/test/results/clientpositive/view_alias.q.out
index 78ff5e2..4e952bb 100644
--- a/ql/src/test/results/clientpositive/view_alias.q.out
+++ b/ql/src/test/results/clientpositive/view_alias.q.out
@@ -43,6 +43,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select key, '12' from src	 
 View Expanded Text: 	select `src`.`key`, '12' from `default`.`src`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select * from v order by `_c1` limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -107,6 +108,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select key as _c1, '12' from src	 
 View Expanded Text: 	select `src`.`key` as `_c1`, '12' from `default`.`src`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select * from v order by `_c1` limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -172,6 +174,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select *, '12' from src	 
 View Expanded Text: 	select `src`.`key`, `src`.`value`, '12' from `default`.`src`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select * from v order by `_c2` limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -237,6 +240,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select *, '12' as _c121 from src	 
 View Expanded Text: 	select `src`.`key`, `src`.`value`, '12' as `_c121` from `default`.`src`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select * from v order by `_c121` limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -301,6 +305,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select key, count(*) from src group by key	 
 View Expanded Text: 	select `src`.`key`, count(*) from `default`.`src` group by `src`.`key`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select * from v order by `_c1` limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -408,6 +413,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select '010', a.*, 121, b.*, 234 from a join b on a.ca = b.cb	 
 View Expanded Text: 	select '010', `a`.`ca`, `a`.`caa`, 121, `b`.`cb`, `b`.`cbb`, 234 from `default`.`a` join `default`.`b` on `a`.`ca` = `b`.`cb`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select * from v order by `_c3` limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@a

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index 70cb126..bc969f2 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -57,6 +57,9 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.coordinator.LlapCoordinator;
 import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl;
 import org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveMaterializedViewsRegistry;
 import org.apache.hadoop.hive.ql.session.ClearDanglingScratchDir;
 import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper;
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -149,6 +152,13 @@ public class HiveServer2 extends CompositeService {
         throw new RuntimeException(e);
       }
     }
+    // Create views registry
+    try {
+      Hive sessionHive = Hive.get(hiveConf);
+      HiveMaterializedViewsRegistry.get().init(sessionHive);
+    } catch (HiveException e) {
+      throw new RuntimeException("Failed to get metastore connection", e);
+    }
     // Setup web UI
     try {
       int webUIPort =


[4/6] hive git commit: HIVE-14496: Enable Calcite rewriting with materialized views (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewFilterScanRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewFilterScanRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewFilterScanRule.java
new file mode 100644
index 0000000..38d7906
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewFilterScanRule.java
@@ -0,0 +1,91 @@
+/**
+ * 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.optimizer.calcite.rules.views;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptMaterialization;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Planner rule that replaces (if possible)
+ * a {@link org.apache.calcite.rel.core.Project}
+ * on a {@link org.apache.calcite.rel.core.Filter}
+ * on a {@link org.apache.calcite.rel.core.TableScan}
+ * to use a Materialized View.
+ */
+public class HiveMaterializedViewFilterScanRule extends RelOptRule {
+
+  public static final HiveMaterializedViewFilterScanRule INSTANCE =
+      new HiveMaterializedViewFilterScanRule(HiveRelFactories.HIVE_BUILDER);
+
+
+  //~ Constructors -----------------------------------------------------------
+
+  /** Creates a HiveMaterializedViewFilterScanRule. */
+  protected HiveMaterializedViewFilterScanRule(RelBuilderFactory relBuilderFactory) {
+    super(operand(Project.class, operand(Filter.class, operand(TableScan.class, null, none()))),
+            relBuilderFactory, "MaterializedViewFilterScanRule");
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  public void onMatch(RelOptRuleCall call) {
+    final Project project = call.rel(0);
+    final Filter filter = call.rel(1);
+    final TableScan scan = call.rel(2);
+    apply(call, project, filter, scan);
+  }
+
+  protected void apply(RelOptRuleCall call, Project project, Filter filter, TableScan scan) {
+    RelOptPlanner planner = call.getPlanner();
+    List<RelOptMaterialization> materializations =
+        (planner instanceof VolcanoPlanner)
+            ? ((VolcanoPlanner) planner).getMaterializations()
+            : ImmutableList.<RelOptMaterialization>of();
+    if (!materializations.isEmpty()) {
+      RelNode root = project.copy(project.getTraitSet(), Collections.singletonList(
+          filter.copy(filter.getTraitSet(), Collections.singletonList(
+              (RelNode) scan))));
+      // Costing is done in transformTo(), so we call it repeatedly with all applicable
+      // materialized views and cheapest one will be picked
+      List<RelOptMaterialization> applicableMaterializations =
+          VolcanoPlanner.getApplicableMaterializations(root, materializations);
+      for (RelOptMaterialization materialization : applicableMaterializations) {
+        List<RelNode> subs = new MaterializedViewSubstitutionVisitor(
+            materialization.queryRel, root, relBuilderFactory).go(materialization.tableRel);
+        for (RelNode s : subs) {
+          call.transformTo(s);
+        }
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/MaterializedViewSubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/MaterializedViewSubstitutionVisitor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/MaterializedViewSubstitutionVisitor.java
new file mode 100644
index 0000000..e32f1a6
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/MaterializedViewSubstitutionVisitor.java
@@ -0,0 +1,292 @@
+/**
+ * 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.optimizer.calcite.rules.views;
+
+import java.util.List;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.tools.RelBuilderFactory;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Extension to {@link SubstitutionVisitor}.
+ *
+ * TODO: Remove when we upgrade to Calcite version using builders.
+ */
+public class MaterializedViewSubstitutionVisitor extends SubstitutionVisitor {
+  private static final ImmutableList<UnifyRule> EXTENDED_RULES =
+      ImmutableList.<UnifyRule>builder()
+          .addAll(DEFAULT_RULES)
+          .add(ProjectToProjectUnifyRule1.INSTANCE)
+          .add(FilterToFilterUnifyRule1.INSTANCE)
+          .add(FilterToProjectUnifyRule1.INSTANCE)
+          .build();
+
+  public MaterializedViewSubstitutionVisitor(RelNode target_, RelNode query_) {
+    super(target_, query_, EXTENDED_RULES);
+  }
+
+  public MaterializedViewSubstitutionVisitor(RelNode target_, RelNode query_,
+      RelBuilderFactory relBuilderFactory) {
+    super(target_, query_, EXTENDED_RULES, relBuilderFactory);
+  }
+
+  public List<RelNode> go(RelNode replacement_) {
+    return super.go(replacement_);
+  }
+
+  /**
+   * Implementation of {@link SubstitutionVisitor.UnifyRule} that matches a
+   * {@link SubstitutionVisitor.MutableProject} to a
+   * {@link SubstitutionVisitor.MutableProject} where the condition of the target
+   * relation is weaker.
+   *
+   * <p>Example: target has a weaker condition and contains all columns selected
+   * by query</p>
+   * <ul>
+   * <li>query:   Project(projects: [$2, $0])
+   *                Filter(condition: &gt;($1, 20))
+   *                  Scan(table: [hr, emps])</li>
+   * <li>target:  Project(projects: [$0, $1, $2])
+   *                Filter(condition: &gt;($1, 10))
+   *                  Scan(table: [hr, emps])</li>
+   * </ul>
+   */
+  private static class ProjectToProjectUnifyRule1 extends AbstractUnifyRule {
+    public static final ProjectToProjectUnifyRule1 INSTANCE =
+        new ProjectToProjectUnifyRule1();
+
+    private ProjectToProjectUnifyRule1() {
+      super(operand(MutableProject.class, query(0)),
+          operand(MutableProject.class, target(0)), 1);
+    }
+
+    @Override protected UnifyResult apply(UnifyRuleCall call) {
+      final MutableProject query = (MutableProject) call.query;
+
+      final List<RelDataTypeField> oldFieldList =
+          query.getInput().getRowType().getFieldList();
+      final List<RelDataTypeField> newFieldList =
+          call.target.getRowType().getFieldList();
+      List<RexNode> newProjects;
+      try {
+        newProjects = transformRex(query.getProjects(), oldFieldList, newFieldList);
+      } catch (MatchFailed e) {
+        return null;
+      }
+
+      final MutableProject newProject =
+          MutableProject.of(
+              query.getRowType(), call.target, newProjects);
+
+      final MutableRel newProject2 = MutableRels.strip(newProject);
+      return call.result(newProject2);
+    }
+
+    @Override protected UnifyRuleCall match(SubstitutionVisitor visitor,
+        MutableRel query, MutableRel target) {
+      assert query instanceof MutableProject && target instanceof MutableProject;
+
+      if (queryOperand.matches(visitor, query)) {
+        if (targetOperand.matches(visitor, target)) {
+          return null;
+        } else if (targetOperand.isWeaker(visitor, target)) {
+
+          final MutableProject queryProject = (MutableProject) query;
+          if (queryProject.getInput() instanceof MutableFilter) {
+            final MutableFilter innerFilter =
+                (MutableFilter) queryProject.getInput();
+            RexNode newCondition;
+            try {
+              newCondition = transformRex(innerFilter.getCondition(),
+                  innerFilter.getInput().getRowType().getFieldList(),
+                  target.getRowType().getFieldList());
+            } catch (MatchFailed e) {
+              return null;
+            }
+            final MutableFilter newFilter = MutableFilter.of(target,
+                newCondition);
+
+            return visitor.new UnifyRuleCall(this, query, newFilter,
+                copy(visitor.slots, slotCount));
+          }
+        }
+      }
+      return null;
+    }
+  }
+
+  /**
+   * Implementation of {@link SubstitutionVisitor.UnifyRule} that matches a
+   * {@link SubstitutionVisitor.MutableFilter} to a
+   * {@link SubstitutionVisitor.MutableFilter} where the condition of the target
+   * relation is weaker.
+   *
+   * <p>Example: target has a weaker condition</p>
+   * <ul>
+   * <li>query:   Filter(condition: &gt;($1, 20))
+   *                Scan(table: [hr, emps])</li>
+   * <li>target:  Filter(condition: &gt;($1, 10))
+   *                Scan(table: [hr, emps])</li>
+   * </ul>
+   */
+  private static class FilterToFilterUnifyRule1 extends AbstractUnifyRule {
+    public static final FilterToFilterUnifyRule1 INSTANCE =
+        new FilterToFilterUnifyRule1();
+
+    private FilterToFilterUnifyRule1() {
+      super(operand(MutableFilter.class, query(0)),
+          operand(MutableFilter.class, target(0)), 1);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      final MutableFilter query = (MutableFilter) call.query;
+      final MutableFilter target = (MutableFilter) call.target;
+      final MutableFilter newFilter = MutableFilter.of(target, query.getCondition());
+      return call.result(newFilter);
+    }
+
+    @Override protected UnifyRuleCall match(SubstitutionVisitor visitor,
+        MutableRel query, MutableRel target) {
+      if (queryOperand.matches(visitor, query)) {
+        if (targetOperand.matches(visitor, target)) {
+          if (visitor.isWeaker(query, target)) {
+            return visitor.new UnifyRuleCall(this, query, target,
+                copy(visitor.slots, slotCount));
+          }
+        }
+      }
+      return null;
+    }
+  }
+
+  /**
+   * Implementation of {@link SubstitutionVisitor.UnifyRule} that matches a
+   * {@link SubstitutionVisitor.MutableFilter} to a
+   * {@link SubstitutionVisitor.MutableProject} on top of a
+   * {@link SubstitutionVisitor.MutableFilter} where the condition of the target
+   * relation is weaker.
+   *
+   * <p>Example: target has a weaker condition and is a permutation projection of
+   * its child relation</p>
+   * <ul>
+   * <li>query:   Filter(condition: &gt;($1, 20))
+   *                Scan(table: [hr, emps])</li>
+   * <li>target:  Project(projects: [$1, $0, $2, $3, $4])
+   *                Filter(condition: &gt;($1, 10))
+   *                  Scan(table: [hr, emps])</li>
+   * </ul>
+   */
+  private static class FilterToProjectUnifyRule1 extends AbstractUnifyRule {
+    public static final FilterToProjectUnifyRule1 INSTANCE =
+        new FilterToProjectUnifyRule1();
+
+    private FilterToProjectUnifyRule1() {
+      super(
+          operand(MutableFilter.class, query(0)),
+          operand(MutableProject.class,
+              operand(MutableFilter.class, target(0))), 1);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      final MutableRel query = call.query;
+
+      final List<RelDataTypeField> oldFieldList =
+          query.getRowType().getFieldList();
+      final List<RelDataTypeField> newFieldList =
+          call.target.getRowType().getFieldList();
+      List<RexNode> newProjects;
+      try {
+        newProjects = transformRex(
+            (List<RexNode>) call.getCluster().getRexBuilder().identityProjects(
+                query.getRowType()),
+            oldFieldList, newFieldList);
+      } catch (MatchFailed e) {
+        return null;
+      }
+
+      final MutableProject newProject =
+          MutableProject.of(
+              query.getRowType(), call.target, newProjects);
+
+      final MutableRel newProject2 = MutableRels.strip(newProject);
+      return call.result(newProject2);
+    }
+
+    @Override protected UnifyRuleCall match(SubstitutionVisitor visitor,
+        MutableRel query, MutableRel target) {
+      assert query instanceof MutableFilter && target instanceof MutableProject;
+
+      if (queryOperand.matches(visitor, query)) {
+        if (targetOperand.matches(visitor, target)) {
+          if (visitor.isWeaker(query, ((MutableProject) target).getInput())) {
+            final MutableFilter filter = (MutableFilter) query;
+            RexNode newCondition;
+            try {
+              newCondition = transformRex(filter.getCondition(),
+                  filter.getInput().getRowType().getFieldList(),
+                  target.getRowType().getFieldList());
+            } catch (MatchFailed e) {
+              return null;
+            }
+            final MutableFilter newFilter = MutableFilter.of(target,
+                newCondition);
+            return visitor.new UnifyRuleCall(this, query, newFilter,
+                copy(visitor.slots, slotCount));
+          }
+        }
+      }
+      return null;
+    }
+  }
+
+  private static RexNode transformRex(RexNode node,
+      final List<RelDataTypeField> oldFields,
+      final List<RelDataTypeField> newFields) {
+    List<RexNode> nodes =
+        transformRex(ImmutableList.of(node), oldFields, newFields);
+    return nodes.get(0);
+  }
+
+  private static List<RexNode> transformRex(
+      List<RexNode> nodes,
+      final List<RelDataTypeField> oldFields,
+      final List<RelDataTypeField> newFields) {
+    RexShuttle shuttle = new RexShuttle() {
+      @Override public RexNode visitInputRef(RexInputRef ref) {
+        RelDataTypeField f = oldFields.get(ref.getIndex());
+        for (int index = 0; index < newFields.size(); index++) {
+          RelDataTypeField newf = newFields.get(index);
+          if (f.getKey().equals(newf.getKey())
+              && f.getValue() == newf.getValue()) {
+            return new RexInputRef(index, f.getValue());
+          }
+        }
+        throw MatchFailed.INSTANCE;
+      }
+    };
+    return shuttle.apply(nodes);
+  }
+}
+
+// End MaterializedViewSubstitutionVisitor.java


[2/6] hive git commit: HIVE-14496: Enable Calcite rewriting with materialized views (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
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 16df496..07155fd 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
@@ -46,9 +46,9 @@ import org.apache.calcite.adapter.druid.DruidRules;
 import org.apache.calcite.adapter.druid.DruidSchema;
 import org.apache.calcite.adapter.druid.DruidTable;
 import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptMaterialization;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptPlanner.Executor;
-import org.apache.calcite.plan.RelOptQuery;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptSchema;
 import org.apache.calcite.plan.RelOptUtil;
@@ -70,6 +70,7 @@ import org.apache.calcite.rel.core.SetOp;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
@@ -108,7 +109,6 @@ import org.apache.calcite.util.CompositeList;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
-import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -126,6 +126,7 @@ import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
@@ -193,6 +194,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortRemoveRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortUnionReduceRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveUnionPullUpConstantsRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveWindowingFixRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewFilterScanRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.HiveOpConverter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.JoinCondTypeCheckProcFactory;
@@ -240,7 +242,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList.Builder;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
-import com.google.common.math.IntMath;
 
 public class CalcitePlanner extends SemanticAnalyzer {
 
@@ -276,6 +277,33 @@ public class CalcitePlanner extends SemanticAnalyzer {
     }
   }
 
+  /**
+   * This method is useful if we want to obtain the logical plan after being parsed and
+   * optimized by Calcite.
+   *
+   * @return the Calcite plan for the query, null if it could not be generated
+   */
+  public RelNode genLogicalPlan(ASTNode ast) throws SemanticException {
+    LOG.info("Starting generating logical plan");
+    PreCboCtx cboCtx = new PreCboCtx();
+    if (!genResolvedParseTree(ast, cboCtx)) {
+      return null;
+    }
+    ASTNode queryForCbo = ast;
+    if (cboCtx.type == PreCboCtx.Type.CTAS_OR_MV) {
+      queryForCbo = cboCtx.nodeOfInterest; // nodeOfInterest is the query
+    }
+    runCBO = canCBOHandleAst(queryForCbo, getQB(), cboCtx);
+    if (!runCBO) {
+      return null;
+    }
+    profilesCBO = obtainCBOProfiles(queryProperties);
+    disableJoinMerge = true;
+    final RelNode resPlan = logicalPlan();
+    LOG.info("Finished generating logical plan");
+    return resPlan;
+  }
+
   @Override
   @SuppressWarnings("rawtypes")
   Operator genOPTree(ASTNode ast, PlannerContext plannerCtx) throws SemanticException {
@@ -758,13 +786,12 @@ public class CalcitePlanner extends SemanticAnalyzer {
   }
 
   /**
-   * Get Optimized AST for the given QB tree in the semAnalyzer.
+   * Get optimized logical plan for the given QB tree in the semAnalyzer.
    *
-   * @return Optimized operator tree translated in to Hive AST
+   * @return
    * @throws SemanticException
    */
-  ASTNode getOptimizedAST() throws SemanticException {
-    ASTNode optiqOptimizedAST = null;
+  RelNode logicalPlan() throws SemanticException {
     RelNode optimizedOptiqPlan = null;
 
     CalcitePlannerAction calcitePlannerAction = null;
@@ -780,9 +807,19 @@ public class CalcitePlanner extends SemanticAnalyzer {
       rethrowCalciteException(e);
       throw new AssertionError("rethrowCalciteException didn't throw for " + e.getMessage());
     }
-    optiqOptimizedAST = ASTConverter.convert(optimizedOptiqPlan, resultSchema,
-            HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_COLUMN_ALIGNMENT));
+    return optimizedOptiqPlan;
+  }
 
+  /**
+   * Get Optimized AST for the given QB tree in the semAnalyzer.
+   *
+   * @return Optimized operator tree translated in to Hive AST
+   * @throws SemanticException
+   */
+  ASTNode getOptimizedAST() throws SemanticException {
+    RelNode optimizedOptiqPlan = logicalPlan();
+    ASTNode optiqOptimizedAST = ASTConverter.convert(optimizedOptiqPlan, resultSchema,
+            HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_COLUMN_ALIGNMENT));
     return optiqOptimizedAST;
   }
 
@@ -990,11 +1027,10 @@ public class CalcitePlanner extends SemanticAnalyzer {
       HiveRulesRegistry registry = new HiveRulesRegistry();
       HivePlannerContext confContext = new HivePlannerContext(algorithmsConf, registry);
       RelOptPlanner planner = HiveVolcanoPlanner.createPlanner(confContext);
-      final RelOptQuery query = new RelOptQuery(planner);
       final RexBuilder rexBuilder = cluster.getRexBuilder();
-      cluster = query.createCluster(rexBuilder.getTypeFactory(), rexBuilder);
+      final RelOptCluster optCluster = RelOptCluster.create(planner, rexBuilder);
 
-      this.cluster = cluster;
+      this.cluster = optCluster;
       this.relOptSchema = relOptSchema;
 
       PerfLogger perfLogger = SessionState.getPerfLogger();
@@ -1014,7 +1050,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
 
       // We need to get the ColumnAccessInfo and viewToTableSchema for views.
       HiveRelFieldTrimmer fieldTrimmer = new HiveRelFieldTrimmer(null,
-          HiveRelFactories.HIVE_BUILDER.create(cluster, null), this.columnAccessInfo,
+          HiveRelFactories.HIVE_BUILDER.create(optCluster, null), this.columnAccessInfo,
           this.viewProjectToTableSchema);
       fieldTrimmer.trim(calciteGenPlan);
 
@@ -1024,7 +1060,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
               JaninoRelMetadataProvider.of(mdProvider.getMetadataProvider()));
 
       // Create executor
-      Executor executorProvider = new HiveRexExecutorImpl(cluster);
+      Executor executorProvider = new HiveRexExecutorImpl(optCluster);
 
       // 2. Apply pre-join order optimizations
       calcitePreCboPlan = applyPreJoinOrderingTransforms(calciteGenPlan,
@@ -1038,7 +1074,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         try {
           List<RelMetadataProvider> list = Lists.newArrayList();
           list.add(mdProvider.getMetadataProvider());
-          RelTraitSet desiredTraits = cluster
+          RelTraitSet desiredTraits = optCluster
               .traitSetOf(HiveRelNode.CONVENTION, RelCollations.EMPTY);
 
           HepProgramBuilder hepPgmBldr = new HepProgramBuilder().addMatchOrder(HepMatchOrder.BOTTOM_UP);
@@ -1050,7 +1086,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
 
           hepPlanner.registerMetadataProviders(list);
           RelMetadataProvider chainedProvider = ChainedRelMetadataProvider.of(list);
-          cluster.setMetadataProvider(new CachingRelMetadataProvider(chainedProvider, hepPlanner));
+          optCluster.setMetadataProvider(new CachingRelMetadataProvider(chainedProvider, hepPlanner));
 
           RelNode rootRel = calcitePreCboPlan;
           hepPlanner.setRoot(rootRel);
@@ -1085,27 +1121,70 @@ public class CalcitePlanner extends SemanticAnalyzer {
               HiveJoinCommuteRule.INSTANCE);
       perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Optimizations without stats");
 
-      // 5. Run aggregate-join transpose (cost based)
+      // 5. Materialized view based rewriting
+      // We disable it for CTAS and MV creation queries (trying to avoid any problem
+      // due to data freshness)
+      if (conf.getBoolVar(ConfVars.HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING) &&
+              !getQB().isMaterializedView() && !getQB().isCTAS()) {
+        perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
+        // Use Calcite cost model for view rewriting
+        RelMetadataProvider calciteMdProvider = DefaultRelMetadataProvider.INSTANCE;
+        RelMetadataQuery.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(calciteMdProvider));
+        planner.registerMetadataProviders(Lists.newArrayList(calciteMdProvider));
+        // Add views to planner
+        List<RelOptMaterialization> materializations = new ArrayList<>();
+        try {
+          materializations = Hive.get().getRewritingMaterializedViews();
+          // We need to use the current cluster for the scan operator on views,
+          // otherwise the planner will throw an Exception (different planners)
+          materializations = Lists.transform(materializations,
+              new Function<RelOptMaterialization, RelOptMaterialization>() {
+                @Override
+                public RelOptMaterialization apply(RelOptMaterialization materialization) {
+                  final RelNode viewScan = materialization.tableRel;
+                  final RelNode newViewScan;
+                  if (viewScan instanceof DruidQuery) {
+                    final DruidQuery dq = (DruidQuery) viewScan;
+                    newViewScan = DruidQuery.create(optCluster, optCluster.traitSetOf(HiveRelNode.CONVENTION),
+                        (RelOptHiveTable) viewScan.getTable(), dq.getDruidTable(),
+                        ImmutableList.<RelNode>of(dq.getTableScan()));
+                  } else {
+                    newViewScan = new HiveTableScan(optCluster, optCluster.traitSetOf(HiveRelNode.CONVENTION),
+                        (RelOptHiveTable) viewScan.getTable(), viewScan.getTable().getQualifiedName().get(0),
+                        null, false, false);
+                  }
+                  return new RelOptMaterialization(newViewScan, materialization.queryRel, null);
+                }
+              }
+          );
+        } catch (HiveException e) {
+          LOG.warn("Exception loading materialized views", e);
+        }
+        if (!materializations.isEmpty()) {
+          for (RelOptMaterialization materialization : materializations) {
+            planner.addMaterialization(materialization);
+          }
+          // Add view-based rewriting rules to planner
+          planner.addRule(HiveMaterializedViewFilterScanRule.INSTANCE);
+          // Optimize plan
+          planner.setRoot(calciteOptimizedPlan);
+          calciteOptimizedPlan = planner.findBestExp();
+          // Remove view-based rewriting rules from planner
+          planner.clear();
+        }
+        // Restore default cost model
+        RelMetadataQuery.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(mdProvider.getMetadataProvider()));
+        perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: View-based rewriting");
+      }
+
+      // 6. Run aggregate-join transpose (cost based)
       //    If it failed because of missing stats, we continue with
       //    the rest of optimizations
       if (conf.getBoolVar(ConfVars.AGGR_JOIN_TRANSPOSE)) {
         perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
         try {
-          HepProgramBuilder hepPgmBldr = new HepProgramBuilder().addMatchOrder(HepMatchOrder.BOTTOM_UP);
-          hepPgmBldr.addRuleInstance(HiveAggregateJoinTransposeRule.INSTANCE);
-
-          HepProgram hepPgm = hepPgmBldr.build();
-          HepPlanner hepPlanner = new HepPlanner(hepPgm);
-
-          List<RelMetadataProvider> list = Lists.newArrayList();
-          list.add(mdProvider.getMetadataProvider());
-          hepPlanner.registerMetadataProviders(list);
-          RelMetadataProvider chainedProvider = ChainedRelMetadataProvider.of(list);
-          cluster.setMetadataProvider(new CachingRelMetadataProvider(chainedProvider, hepPlanner));
-
-          hepPlanner.setRoot(calciteOptimizedPlan);
-
-          calciteOptimizedPlan = hepPlanner.findBestExp();
+          calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
+                  HepMatchOrder.BOTTOM_UP, HiveAggregateJoinTransposeRule.INSTANCE);
         } catch (Exception e) {
           boolean isMissingStats = noColsMissingStats.get() > 0;
           if (isMissingStats) {
@@ -1118,7 +1197,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Aggregate join transpose");
       }
 
-      // 6.convert Join + GBy to semijoin
+      // 7.convert Join + GBy to semijoin
       // run this rule at later stages, since many calcite rules cant deal with semijoin
       if (conf.getBoolVar(ConfVars.SEMIJOIN_CONVERSION)) {
         perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
@@ -1127,7 +1206,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       }
 
 
-      // 7. Run rule to fix windowing issue when it is done over
+      // 8. Run rule to fix windowing issue when it is done over
       // aggregation columns (HIVE-10627)
       if (profilesCBO.contains(ExtendedCBOProfile.WINDOWING_POSTPROCESSING)) {
         perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
@@ -1136,7 +1215,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Window fixing rule");
       }
 
-      // 8. Apply Druid transformation rules
+      // 9. Apply Druid transformation rules
       perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
       calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
               HepMatchOrder.BOTTOM_UP, DruidRules.FILTER, DruidRules.PROJECT_AGGREGATE,
@@ -1144,10 +1223,10 @@ public class CalcitePlanner extends SemanticAnalyzer {
               DruidRules.SORT, DruidRules.SORT_PROJECT);
       perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Druid transformation rules");
 
-      // 9. Run rules to aid in translation from Calcite tree to Hive tree
+      // 10. Run rules to aid in translation from Calcite tree to Hive tree
       if (HiveConf.getBoolVar(conf, ConfVars.HIVE_CBO_RETPATH_HIVEOP)) {
         perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
-        // 9.1. Merge join into multijoin operators (if possible)
+        // 10.1. Merge join into multijoin operators (if possible)
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, true, mdProvider.getMetadataProvider(), null,
                 HepMatchOrder.BOTTOM_UP, HiveJoinProjectTransposeRule.BOTH_PROJECT_INCLUDE_OUTER,
                 HiveJoinProjectTransposeRule.LEFT_PROJECT_INCLUDE_OUTER,
@@ -1156,7 +1235,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         // The previous rules can pull up projections through join operators,
         // thus we run the field trimmer again to push them back down
         fieldTrimmer = new HiveRelFieldTrimmer(null,
-            HiveRelFactories.HIVE_BUILDER.create(cluster, null));
+            HiveRelFactories.HIVE_BUILDER.create(optCluster, null));
         calciteOptimizedPlan = fieldTrimmer.trim(calciteOptimizedPlan);
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
                 HepMatchOrder.BOTTOM_UP, ProjectRemoveRule.INSTANCE,
@@ -1165,7 +1244,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
                 HiveFilterProjectTSTransposeRule.INSTANCE, HiveFilterProjectTSTransposeRule.INSTANCE_DRUID,
                 HiveProjectFilterPullUpConstantsRule.INSTANCE);
 
-        // 9.2.  Introduce exchange operators below join/multijoin operators
+        // 10.2.  Introduce exchange operators below join/multijoin operators
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
                 HepMatchOrder.BOTTOM_UP, HiveInsertExchange4JoinRule.EXCHANGE_BELOW_JOIN,
                 HiveInsertExchange4JoinRule.EXCHANGE_BELOW_MULTIJOIN);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 55915a6..8015eba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -256,6 +256,8 @@ TOK_ALTERVIEW_DROPPARTS;
 TOK_ALTERVIEW_RENAME;
 TOK_CREATE_MATERIALIZED_VIEW;
 TOK_DROP_MATERIALIZED_VIEW;
+TOK_REWRITE_ENABLED;
+TOK_REWRITE_DISABLED;
 TOK_VIEWPARTCOLS;
 TOK_EXPLAIN;
 TOK_EXPLAIN_SQ_REWRITE;
@@ -882,6 +884,20 @@ ifNotExists
     -> ^(TOK_IFNOTEXISTS)
     ;
 
+rewriteEnabled
+@init { pushMsg("rewrite enabled clause", state); }
+@after { popMsg(state); }
+    : KW_ENABLE KW_REWRITE
+    -> ^(TOK_REWRITE_ENABLED)
+    ;
+
+rewriteDisabled
+@init { pushMsg("rewrite disabled clause", state); }
+@after { popMsg(state); }
+    : KW_DISABLE KW_REWRITE
+    -> ^(TOK_REWRITE_DISABLED)
+    ;
+
 storedAsDirs
 @init { pushMsg("stored as directories", state); }
 @after { popMsg(state); }
@@ -1832,10 +1848,11 @@ createMaterializedViewStatement
 }
 @after { popMsg(state); }
     : KW_CREATE KW_MATERIALIZED KW_VIEW (ifNotExists)? name=tableName
-        tableComment? tableRowFormat? tableFileFormat? tableLocation?
+        rewriteEnabled? tableComment? tableRowFormat? tableFileFormat? tableLocation?
         tablePropertiesPrefixed? KW_AS selectStatementWithCTE
     -> ^(TOK_CREATE_MATERIALIZED_VIEW $name 
          ifNotExists?
+         rewriteEnabled?
          tableComment?
          tableRowFormat?
          tableFileFormat?

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 7e6dcf3..1f6dfdd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -11976,6 +11976,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     String dbDotTable = getDotName(qualTabName);
     List<FieldSchema> cols = null;
     boolean ifNotExists = false;
+    boolean rewriteEnabled = false;
     boolean orReplace = false;
     boolean isAlterViewAs = false;
     String comment = null;
@@ -11999,6 +12000,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       case HiveParser.TOK_IFNOTEXISTS:
         ifNotExists = true;
         break;
+      case HiveParser.TOK_REWRITE_ENABLED:
+        rewriteEnabled = true;
+        break;
       case HiveParser.TOK_ORREPLACE:
         orReplace = true;
         break;
@@ -12058,20 +12062,21 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     if (isMaterialized) {
       createVwDesc = new CreateViewDesc(
-              dbDotTable, cols, comment, tblProps, partColNames,
-              ifNotExists, orReplace, isAlterViewAs, storageFormat.getInputFormat(),
-              storageFormat.getOutputFormat(), location, storageFormat.getSerde(),
-              storageFormat.getStorageHandler(), storageFormat.getSerdeProps());
+          dbDotTable, cols, comment, tblProps, partColNames,
+          ifNotExists, orReplace, rewriteEnabled, isAlterViewAs,
+          storageFormat.getInputFormat(), storageFormat.getOutputFormat(),
+          location, storageFormat.getSerde(), storageFormat.getStorageHandler(),
+          storageFormat.getSerdeProps());
       addDbAndTabToOutputs(qualTabName, TableType.MATERIALIZED_VIEW);
       queryState.setCommandType(HiveOperation.CREATE_MATERIALIZED_VIEW);
       qb.setViewDesc(createVwDesc);
     } else {
       createVwDesc = new CreateViewDesc(
-              dbDotTable, cols, comment, tblProps, partColNames,
-              ifNotExists, orReplace, isAlterViewAs, storageFormat.getInputFormat(),
-              storageFormat.getOutputFormat(), storageFormat.getSerde());
+          dbDotTable, cols, comment, tblProps, partColNames,
+          ifNotExists, orReplace, isAlterViewAs, storageFormat.getInputFormat(),
+          storageFormat.getOutputFormat(), storageFormat.getSerde());
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-              createVwDesc), conf));
+          createVwDesc), conf));
       addDbAndTabToOutputs(qualTabName, TableType.VIRTUAL_VIEW);
       queryState.setCommandType(HiveOperation.CREATEVIEW);
     }
@@ -12083,8 +12088,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return this.createVwDesc;
   }
 
-  // validate the create view statement
-  // the statement could be CREATE VIEW, REPLACE VIEW, or ALTER VIEW AS SELECT
+  // validate the (materialized) view statement
   // check semantic conditions
   private void validateCreateView()
     throws SemanticException {

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java
index d3b955c..64218b2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/StorageFormat.java
@@ -111,8 +111,8 @@ public class StorageFormat {
       String defaultManagedFormat;
       if (isMaterializedView) {
         defaultFormat = defaultManagedFormat =
-            HiveConf.getVar(conf, HiveConf.ConfVars.HIVEMATERIALIZEDVIEWFILEFORMAT);
-        serde = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEMATERIALIZEDVIEWSERDE);
+            HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_FILE_FORMAT);
+        serde = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_SERDE);
       } else {
         defaultFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT);
         defaultManagedFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEDEFAULTMANAGEDFILEFORMAT);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
index 6830bda..a69f8e6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
@@ -38,6 +38,7 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
   private String viewName;
   private String originalText;
   private String expandedText;
+  private boolean rewriteEnabled;
   private List<FieldSchema> schema;
   private Map<String, String> tblProps;
   private List<String> partColNames;
@@ -79,7 +80,7 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
    */
   public CreateViewDesc(String viewName, List<FieldSchema> schema, String comment,
           Map<String, String> tblProps, List<String> partColNames,
-          boolean ifNotExists, boolean orReplace, boolean isAlterViewAs,
+          boolean ifNotExists, boolean orReplace, boolean rewriteEnabled, boolean isAlterViewAs,
           String inputFormat, String outputFormat, String location,
           String serde, String storageHandler, Map<String, String> serdeProps) {
     this.viewName = viewName;
@@ -89,8 +90,9 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
     this.comment = comment;
     this.ifNotExists = ifNotExists;
     this.orReplace = orReplace;
-    this.isAlterViewAs = isAlterViewAs;
     this.isMaterialized = true;
+    this.rewriteEnabled = rewriteEnabled;
+    this.isAlterViewAs = isAlterViewAs;
     this.inputFormat = inputFormat;
     this.outputFormat = outputFormat;
     this.location = location;
@@ -126,6 +128,7 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
     this.orReplace = orReplace;
     this.isAlterViewAs = isAlterViewAs;
     this.isMaterialized = false;
+    this.rewriteEnabled = false;
     this.inputFormat = inputFormat;
     this.outputFormat = outputFormat;
     this.serde = serde;
@@ -158,6 +161,15 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
     this.expandedText = expandedText;
   }
 
+  @Explain(displayName = "rewrite enabled")
+  public boolean isRewriteEnabled() {
+    return rewriteEnabled;
+  }
+
+  public void setRewriteEnabled(boolean rewriteEnabled) {
+    this.rewriteEnabled = rewriteEnabled;
+  }
+
   @Explain(displayName = "columns")
   public List<String> getSchemaString() {
     return Utilities.getFieldSchemaString(schema);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 71aea3a..453e0a5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -42,11 +42,8 @@ import java.util.UUID;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.locks.ReentrantLock;
 
-import com.google.common.collect.Maps;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.ArrayUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -89,9 +86,12 @@ import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
 
 /**
  * SessionState encapsulates common data associated with a session.

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
index 42c715e..ccc0f9c 100755
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
@@ -169,6 +169,8 @@ public class TestHive extends TestCase {
       tbl.setSerializationLib(LazySimpleSerDe.class.getName());
       tbl.setStoredAsSubDirectories(false);
 
+      tbl.setRewriteEnabled(false);
+
       // create table
       setNullCreateTableGrants();
       try {
@@ -228,6 +230,8 @@ public class TestHive extends TestCase {
           .getName());
       tbl.setStoredAsSubDirectories(false);
 
+      tbl.setRewriteEnabled(false);
+
       setNullCreateTableGrants();
       try {
         hm.createTable(tbl);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q
new file mode 100644
index 0000000..e95a868
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q
@@ -0,0 +1,59 @@
+set hive.strict.checks.cartesian.product=false;
+set hive.materializedview.rewriting=true;
+set hive.stats.column.autogather=true;
+
+create table cmv_basetable (a int, b varchar(256), c decimal(10,2), d int);
+
+insert into cmv_basetable values
+ (1, 'alfred', 10.30, 2),
+ (2, 'bob', 3.14, 3),
+ (2, 'bonnie', 172342.2, 3),
+ (3, 'calvin', 978.76, 3),
+ (3, 'charlie', 9.8, 1);
+
+create materialized view cmv_mat_view enable rewrite
+as select a, b, c from cmv_basetable where a = 2;
+
+select * from cmv_mat_view;
+
+show tblproperties cmv_mat_view;
+
+create materialized view if not exists cmv_mat_view2 enable rewrite
+as select a, c from cmv_basetable where a = 3;
+
+select * from cmv_mat_view2;
+
+show tblproperties cmv_mat_view2;
+
+explain
+select a, c from cmv_basetable where a = 3;
+
+select a, c from cmv_basetable where a = 3;
+
+explain
+select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a);
+
+select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a);
+
+drop materialized view cmv_mat_view2;
+
+explain
+select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a);
+
+select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q
new file mode 100644
index 0000000..e4cdc22
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q
@@ -0,0 +1,40 @@
+set hive.strict.checks.cartesian.product=false;
+set hive.materializedview.rewriting=true;
+set hive.stats.column.autogather=true;
+
+create database db1;
+use db1;
+
+create table cmv_basetable (a int, b varchar(256), c decimal(10,2), d int);
+
+insert into cmv_basetable values
+ (1, 'alfred', 10.30, 2),
+ (2, 'bob', 3.14, 3),
+ (2, 'bonnie', 172342.2, 3),
+ (3, 'calvin', 978.76, 3),
+ (3, 'charlie', 9.8, 1);
+
+create database db2;
+use db2;
+
+create materialized view cmv_mat_view enable rewrite
+as select a, b, c from db1.cmv_basetable where a = 2;
+
+select * from cmv_mat_view;
+
+show tblproperties cmv_mat_view;
+
+create materialized view if not exists cmv_mat_view2 enable rewrite
+as select a, c from db1.cmv_basetable where a = 3;
+
+select * from cmv_mat_view2;
+
+show tblproperties cmv_mat_view2;
+
+create database db3;
+use db3;
+
+explain
+select a, c from db1.cmv_basetable where a = 3;
+
+select a, c from db1.cmv_basetable where a = 3;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out b/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
index 4e43819..adbc7a8 100644
--- a/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
@@ -69,5 +69,6 @@ View Original Text: 	SELECT key, value
 View Expanded Text: 	SELECT `src`.`key`, `src`.`value`	 
 	                    	FROM `default`.`src`
 	                    	WHERE `src`.`key`=86
+View Rewrite Enabled:	No                  	 
 FAILED: SemanticException [Error 10217]: Cannot replace a view with CREATE VIEW or REPLACE VIEW or ALTER VIEW AS SELECT if the view has partitions
  The following view has partition, it could not be replaced: default.testViewPart

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/alter_view_as_select.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_view_as_select.q.out b/ql/src/test/results/clientpositive/alter_view_as_select.q.out
index dc1814e..9cbaa24 100644
--- a/ql/src/test/results/clientpositive/alter_view_as_select.q.out
+++ b/ql/src/test/results/clientpositive/alter_view_as_select.q.out
@@ -47,6 +47,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	SELECT * FROM srcpart	 
 View Expanded Text: 	SELECT `srcpart`.`key`, `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` FROM `default`.`srcpart`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: ALTER VIEW tv.testView AS SELECT value FROM src WHERE key=86
 PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@src
@@ -87,6 +88,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	SELECT value FROM src WHERE key=86	 
 View Expanded Text: 	SELECT `src`.`value` FROM `default`.`src` WHERE `src`.`key`=86	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: ALTER VIEW tv.testView AS
 SELECT * FROM src
 WHERE key > 80 AND key < 100
@@ -142,6 +144,7 @@ View Expanded Text: 	SELECT `src`.`key`, `src`.`value` FROM `default`.`src`
 	                    	WHERE `src`.`key` > 80 AND `src`.`key` < 100
 	                    	ORDER BY `src`.`key`, `src`.`value`
 	                    	LIMIT 10            
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: DROP VIEW tv.testView
 PREHOOK: type: DROPVIEW
 PREHOOK: Input: tv@testview

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/create_or_replace_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_or_replace_view.q.out b/ql/src/test/results/clientpositive/create_or_replace_view.q.out
index f6f26d2..834cdf0 100644
--- a/ql/src/test/results/clientpositive/create_or_replace_view.q.out
+++ b/ql/src/test/results/clientpositive/create_or_replace_view.q.out
@@ -47,6 +47,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select * from srcpart	 
 View Expanded Text: 	select `srcpart`.`key`, `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` from `default`.`srcpart`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: -- modifying definition of unpartitioned view
 create or replace view vt.v partitioned on (ds, hr) as select * from srcpart
 PREHOOK: type: CREATEVIEW
@@ -137,6 +138,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select * from srcpart	 
 View Expanded Text: 	select `srcpart`.`key`, `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` from `default`.`srcpart`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: show partitions vt.v
 PREHOOK: type: SHOWPARTITIONS
 PREHOOK: Input: vt@v
@@ -230,6 +232,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select value, ds, hr from srcpart	 
 View Expanded Text: 	select `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` from `default`.`srcpart`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: show partitions vt.v
 PREHOOK: type: SHOWPARTITIONS
 PREHOOK: Input: vt@v
@@ -300,6 +303,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select key, value, ds, hr from srcpart	 
 View Expanded Text: 	select `srcpart`.`key`, `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` from `default`.`srcpart`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: show partitions vt.v
 PREHOOK: type: SHOWPARTITIONS
 PREHOOK: Input: vt@v
@@ -391,6 +395,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select * from srcpart	 
 View Expanded Text: 	select `srcpart`.`key`, `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` from `default`.`srcpart`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: drop view vt.v
 PREHOOK: type: DROPVIEW
 PREHOOK: Input: vt@v

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/create_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view.q.out b/ql/src/test/results/clientpositive/create_view.q.out
index 12457b4..26b15e2 100644
--- a/ql/src/test/results/clientpositive/create_view.q.out
+++ b/ql/src/test/results/clientpositive/create_view.q.out
@@ -171,6 +171,7 @@ STAGE PLANS:
           expanded text: SELECT `_c0` AS `valoo` FROM (SELECT upper(`src`.`value`) FROM `default`.`src` WHERE `src`.`key`=86) `default.view0`
           name: default.view0
           original text: SELECT upper(value) FROM src WHERE key=86
+          rewrite enabled: false
 
 PREHOOK: query: -- make sure EXPLAIN works with a query which references a view
 EXPLAIN
@@ -269,6 +270,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	SELECT value FROM src WHERE key=86	 
 View Expanded Text: 	SELECT `src`.`value` FROM `default`.`src` WHERE `src`.`key`=86	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: DESCRIBE view2
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view2
@@ -318,6 +320,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	SELECT * FROM src   	 
 View Expanded Text: 	SELECT `src`.`key`, `src`.`value` FROM `default`.`src`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: DESCRIBE view3
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view3
@@ -365,6 +368,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	SELECT upper(value) FROM src WHERE key=86	 
 View Expanded Text: 	SELECT `_c0` AS `valoo` FROM (SELECT upper(`src`.`value`) FROM `default`.`src` WHERE `src`.`key`=86) `default.view3`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: ALTER VIEW view3 SET TBLPROPERTIES ("biggest" = "loser")
 PREHOOK: type: ALTERVIEW_PROPERTIES
 PREHOOK: Input: default@view3
@@ -414,6 +418,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	SELECT upper(value) FROM src WHERE key=86	 
 View Expanded Text: 	SELECT `_c0` AS `valoo` FROM (SELECT upper(`src`.`value`) FROM `default`.`src` WHERE `src`.`key`=86) `default.view3`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: CREATE TABLE table1 (key int)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
@@ -745,7 +750,7 @@ c                   	string
 	 	 
 #### A masked pattern was here ####
 FROM table1, viewExpandedText:SELECT `_c0` AS `c` FROM (SELECT `test_translate`('abc', 'a', 'b')	 	 
-FROM `default`.`table1`) `default.view8`, tableType:VIRTUAL_VIEW)		 
+FROM `default`.`table1`) `default.view8`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: DESCRIBE FORMATTED view8
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view8
@@ -778,6 +783,7 @@ View Original Text: 	SELECT test_translate('abc', 'a', 'b')
 	                    	FROM table1         
 View Expanded Text: 	SELECT `_c0` AS `c` FROM (SELECT `test_translate`('abc', 'a', 'b')	 
 	                    	FROM `default`.`table1`) `default.view8`
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view8
 PREHOOK: type: QUERY
 PREHOOK: Input: default@table1
@@ -825,7 +831,7 @@ m                   	int
 	 	 
 #### A masked pattern was here ####
 FROM src, viewExpandedText:SELECT `_c0` AS `m` FROM (SELECT `test_max`(length(`src`.`value`))	 	 
-FROM `default`.`src`) `default.view9`, tableType:VIRTUAL_VIEW)		 
+FROM `default`.`src`) `default.view9`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: DESCRIBE FORMATTED view9
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view9
@@ -858,6 +864,7 @@ View Original Text: 	SELECT test_max(length(value))
 	                    	FROM src            
 View Expanded Text: 	SELECT `_c0` AS `m` FROM (SELECT `test_max`(length(`src`.`value`))	 
 	                    	FROM `default`.`src`) `default.view9`
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view9
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -903,7 +910,7 @@ m                   	int
 	 	 
 #### A masked pattern was here ####
 FROM src, viewExpandedText:SELECT `_c0` AS `m` FROM (SELECT `test_max`(length(`src`.`value`))	 	 
-FROM `default`.`src`) `default.view9`, tableType:VIRTUAL_VIEW)		 
+FROM `default`.`src`) `default.view9`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: DESCRIBE FORMATTED view9
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view9
@@ -936,6 +943,7 @@ View Original Text: 	SELECT test_max(length(value))
 	                    	FROM src            
 View Expanded Text: 	SELECT `_c0` AS `m` FROM (SELECT `test_max`(length(`src`.`value`))	 
 	                    	FROM `default`.`src`) `default.view9`
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view9
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -1002,6 +1010,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	SELECT slurp.* FROM (SELECT * FROM src WHERE key=86) slurp	 
 View Expanded Text: 	SELECT `slurp`.`key`, `slurp`.`value` FROM (SELECT `src`.`key`, `src`.`value` FROM `default`.`src` WHERE `src`.`key`=86) `slurp`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -1047,7 +1056,7 @@ boom                	int
 	 	 
 #### A masked pattern was here ####
 FROM table1, viewExpandedText:SELECT `test_explode`(array(1,2,3)) AS (`boom`)	 	 
-FROM `default`.`table1`, tableType:VIRTUAL_VIEW)		 
+FROM `default`.`table1`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: DESCRIBE FORMATTED view11
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view11
@@ -1080,6 +1089,7 @@ View Original Text: 	SELECT test_explode(array(1,2,3)) AS (boom)
 	                    	FROM table1         
 View Expanded Text: 	SELECT `test_explode`(array(1,2,3)) AS (`boom`)	 
 	                    	FROM `default`.`table1`
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view11
 PREHOOK: type: QUERY
 PREHOOK: Input: default@table1
@@ -1150,6 +1160,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol	 
 View Expanded Text: 	SELECT `src`.`key`, `src`.`value`, `mytable`.`mycol` FROM `default`.`src` LATERAL VIEW explode(array(1,2,3)) `myTable` AS `myCol`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view12
 ORDER BY key ASC, myCol ASC LIMIT 1
 PREHOOK: type: QUERY
@@ -1204,7 +1215,7 @@ key                 	int
 	 	 
 #### A masked pattern was here ####
 FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 ON key) s, viewExpandedText:SELECT `s`.`key`	 	 
-FROM `default`.`srcbucket` TABLESAMPLE (BUCKET 1 OUT OF 5 ON `key`) `s`, tableType:VIRTUAL_VIEW)		 
+FROM `default`.`srcbucket` TABLESAMPLE (BUCKET 1 OUT OF 5 ON `key`) `s`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: DESCRIBE FORMATTED view13
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view13
@@ -1237,6 +1248,7 @@ View Original Text: 	SELECT s.key
 	                    	FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 ON key) s
 View Expanded Text: 	SELECT `s`.`key`    	 
 	                    	FROM `default`.`srcbucket` TABLESAMPLE (BUCKET 1 OUT OF 5 ON `key`) `s`
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view13
 ORDER BY key LIMIT 12
 PREHOOK: type: QUERY
@@ -1322,7 +1334,7 @@ JOIN
      (select 'tst1' as `key`, cast(count(1) as string) as `value` from `default`.`src` `s3`	 	 
                          UNION  ALL	 	 
       select `s4`.`key` as `key`, `s4`.`value` as `value` from `default`.`src` `s4` where `s4`.`key` < 10) `unionsrc2`	 	 
-ON (`unionsrc1`.`key` = `unionsrc2`.`key`), tableType:VIRTUAL_VIEW)		 
+ON (`unionsrc1`.`key` = `unionsrc2`.`key`), tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: DESCRIBE FORMATTED view14
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view14
@@ -1374,6 +1386,7 @@ View Expanded Text: 	SELECT `unionsrc1`.`key` as `k1`, `unionsrc1`.`value` as `v
 	                    	                         UNION  ALL
 	                    	      select `s4`.`key` as `key`, `s4`.`value` as `value` from `default`.`src` `s4` where `s4`.`key` < 10) `unionsrc2`
 	                    	ON (`unionsrc1`.`key` = `unionsrc2`.`key`)
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view14
 ORDER BY k1
 PREHOOK: type: QUERY
@@ -1440,7 +1453,7 @@ value_count         	bigint
 FROM src	 	 
 GROUP BY key, viewExpandedText:SELECT `src`.`key`,COUNT(`src`.`value`) AS `value_count`	 	 
 FROM `default`.`src`	 	 
-GROUP BY `src`.`key`, tableType:VIRTUAL_VIEW)		 
+GROUP BY `src`.`key`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: DESCRIBE FORMATTED view15
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view15
@@ -1476,6 +1489,7 @@ View Original Text: 	SELECT key,COUNT(value) AS value_count
 View Expanded Text: 	SELECT `src`.`key`,COUNT(`src`.`value`) AS `value_count`	 
 	                    	FROM `default`.`src`
 	                    	GROUP BY `src`.`key`
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view15
 ORDER BY value_count DESC, key
 LIMIT 10
@@ -1526,7 +1540,7 @@ value               	string
 	 	 
 #### A masked pattern was here ####
 FROM src, viewExpandedText:SELECT DISTINCT `src`.`value`	 	 
-FROM `default`.`src`, tableType:VIRTUAL_VIEW)		 
+FROM `default`.`src`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: DESCRIBE FORMATTED view16
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@view16
@@ -1559,6 +1573,7 @@ View Original Text: 	SELECT DISTINCT value
 	                    	FROM src            
 View Expanded Text: 	SELECT DISTINCT `src`.`value`	 
 	                    	FROM `default`.`src`
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM view16
 ORDER BY value
 LIMIT 10

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/create_view_defaultformats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view_defaultformats.q.out b/ql/src/test/results/clientpositive/create_view_defaultformats.q.out
index dbc4a20..2412513 100644
--- a/ql/src/test/results/clientpositive/create_view_defaultformats.q.out
+++ b/ql/src/test/results/clientpositive/create_view_defaultformats.q.out
@@ -57,6 +57,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select * from src   	 
 View Expanded Text: 	select `src`.`key`, `src`.`value` from `default`.`src`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: describe formatted rcsrc
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@rcsrc
@@ -88,6 +89,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select * from src   	 
 View Expanded Text: 	select `src`.`key`, `src`.`value` from `default`.`src`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select * from sfsrc where key = 100 limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@sfsrc

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/create_view_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view_partitioned.q.out b/ql/src/test/results/clientpositive/create_view_partitioned.q.out
index 4373303..1be3556 100644
--- a/ql/src/test/results/clientpositive/create_view_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/create_view_partitioned.q.out
@@ -52,7 +52,7 @@ value               	string
 FROM src	 	 
 WHERE key=86, viewExpandedText:SELECT `src`.`key`, `src`.`value`	 	 
 FROM `default`.`src`	 	 
-WHERE `src`.`key`=86, tableType:VIRTUAL_VIEW)		 
+WHERE `src`.`key`=86, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: DESCRIBE FORMATTED vp1
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@vp1
@@ -92,6 +92,7 @@ View Original Text: 	SELECT key, value
 View Expanded Text: 	SELECT `src`.`key`, `src`.`value`	 
 	                    	FROM `default`.`src`
 	                    	WHERE `src`.`key`=86
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: SELECT * FROM vp1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -295,6 +296,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	SELECT * FROM srcpart WHERE key < 10	 
 View Expanded Text: 	SELECT `srcpart`.`key`, `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` FROM `default`.`srcpart` WHERE `srcpart`.`key` < 10	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: ALTER VIEW vp2 ADD PARTITION (hr='11') PARTITION (hr='12')
 PREHOOK: type: ALTERTABLE_ADDPARTS
 PREHOOK: Input: default@srcpart
@@ -412,6 +414,7 @@ View Original Text: 	SELECT key, value
 View Expanded Text: 	SELECT `key` AS `k`, `value` AS `v` FROM (SELECT `src`.`key`, `src`.`value`	 
 	                    	FROM `default`.`src`
 	                    	WHERE `src`.`key`=86) `default.vp3`
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: ALTER VIEW vp3
 ADD PARTITION (v='val_86')
 PREHOOK: type: ALTERTABLE_ADDPARTS

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/create_view_translate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view_translate.q.out b/ql/src/test/results/clientpositive/create_view_translate.q.out
index 43b9062..cb7402c 100644
--- a/ql/src/test/results/clientpositive/create_view_translate.q.out
+++ b/ql/src/test/results/clientpositive/create_view_translate.q.out
@@ -46,6 +46,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select cast(key as string) from src	 
 View Expanded Text: 	select `src`.`key` from `default`.`src`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: create view w as select key, value from (
   select key, value from src
 ) a
@@ -95,6 +96,7 @@ View Original Text: 	select key, value from (
 View Expanded Text: 	select `a`.`key`, `a`.`value` from (	 
 	                    	  select `src`.`key`, `src`.`value` from `default`.`src`
 	                    	) `a`               
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: drop view v
 PREHOOK: type: DROPVIEW
 PREHOOK: Input: default@v
@@ -141,6 +143,7 @@ STAGE PLANS:
           expanded text: SELECT `items`.`id`, `items`.`info`['price'] FROM `default`.`items`
           name: default.priceview
           original text: SELECT items.id, items.info['price'] FROM items
+          rewrite enabled: false
 
 PREHOOK: query: CREATE VIEW priceview AS SELECT items.id, items.info['price'] FROM items
 PREHOOK: type: CREATEVIEW

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/cteViews.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cteViews.q.out b/ql/src/test/results/clientpositive/cteViews.q.out
index eb3cfc0..69f0a91 100644
--- a/ql/src/test/results/clientpositive/cteViews.q.out
+++ b/ql/src/test/results/clientpositive/cteViews.q.out
@@ -45,7 +45,7 @@ key                 	string
 	 	 
 #### A masked pattern was here ####
 select key from cte, viewExpandedText:with cte as (select `src`.`key`, `src`.`value` from `default`.`src` order by key limit 5)	 	 
-select `cte`.`key` from cte, tableType:VIRTUAL_VIEW)		 
+select `cte`.`key` from cte, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: create database bug
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:bug
@@ -118,7 +118,7 @@ value               	string
 	 	 
 #### A masked pattern was here ####
 select * from cte, viewExpandedText:with cte as (select `src`.`key`, `src`.`value` from `default`.`src`  order by `src`.`key` limit 5)	 	 
-select `cte`.`key`, `cte`.`value` from cte, tableType:VIRTUAL_VIEW)		 
+select `cte`.`key`, `cte`.`value` from cte, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: create database bug
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:bug
@@ -190,7 +190,7 @@ key                 	string
 	 	 
 #### A masked pattern was here ####
 select * from src1, viewExpandedText:with src1 as (select `src`.`key` from `default`.`src` order by key limit 5)	 	 
-select `src1`.`key` from src1, tableType:VIRTUAL_VIEW)		 
+select `src1`.`key` from src1, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: create database bug
 PREHOOK: type: CREATEDATABASE
 PREHOOK: Output: database:bug
@@ -254,7 +254,7 @@ key                 	string
 	 	 
 #### A masked pattern was here ####
 select * from src1 a where a.key is not null, viewExpandedText:with src1 as (select `src`.`key` from `default`.`src` order by key limit 5)	 	 
-select `a`.`key` from src1 a where `a`.`key` is not null, tableType:VIRTUAL_VIEW)		 
+select `a`.`key` from src1 a where `a`.`key` is not null, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/escape_comments.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/escape_comments.q.out b/ql/src/test/results/clientpositive/escape_comments.q.out
index 0b8c5c5..ff5a1ed 100644
--- a/ql/src/test/results/clientpositive/escape_comments.q.out
+++ b/ql/src/test/results/clientpositive/escape_comments.q.out
@@ -188,6 +188,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select col1 from escape_comments_tbl1	 
 View Expanded Text: 	SELECT `col1` AS `col1` FROM (select `escape_comments_tbl1`.`col1` from `escape_comments_db`.`escape_comments_tbl1`) `escape_comments_db.escape_comments_view1`	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: show formatted index on escape_comments_tbl1
 PREHOOK: type: SHOWINDEXES
 POSTHOOK: query: show formatted index on escape_comments_tbl1

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/explain_ddl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_ddl.q.out b/ql/src/test/results/clientpositive/explain_ddl.q.out
index e8438a1..2b89f28 100644
--- a/ql/src/test/results/clientpositive/explain_ddl.q.out
+++ b/ql/src/test/results/clientpositive/explain_ddl.q.out
@@ -432,6 +432,7 @@ STAGE PLANS:
           expanded text: select `m1`.`key`, `m1`.`value` from `default`.`M1`
           name: default.V1
           original text: select * from M1
+          rewrite enabled: false
 
 PREHOOK: query: EXPLAIN CREATE TABLE M1 LIKE src
 PREHOOK: type: CREATETABLE

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out b/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out
index 304d74f..10609d9 100644
--- a/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/cbo_rp_unionDistinct_2.q.out
@@ -365,7 +365,7 @@ union
 select `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
 union all	 	 
 select `u3`.`key` as `key`, `u3`.`value` from `default`.`u3`	 	 
-) `tab`, tableType:VIRTUAL_VIEW)		 
+) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -435,7 +435,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union  	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW)		 
+) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -500,7 +500,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union all 	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW)		 
+) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/llap/selectDistinctStar.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/selectDistinctStar.q.out b/ql/src/test/results/clientpositive/llap/selectDistinctStar.q.out
index 985086d..2eff533 100644
--- a/ql/src/test/results/clientpositive/llap/selectDistinctStar.q.out
+++ b/ql/src/test/results/clientpositive/llap/selectDistinctStar.q.out
@@ -1362,6 +1362,7 @@ STAGE PLANS:
           expanded text: select distinct `src`.`key`, `src`.`value` from `default`.`src` order by `src`.`key` limit 2
           name: default.sdi
           original text: select distinct * from src order by key limit 2
+          rewrite enabled: false
 
 PREHOOK: query: create view sdi as select distinct * from src order by key limit 2
 PREHOOK: type: CREATEVIEW
@@ -1414,6 +1415,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select distinct * from src order by key limit 2	 
 View Expanded Text: 	select distinct `src`.`key`, `src`.`value` from `default`.`src` order by `src`.`key` limit 2	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select * from sdi
 PREHOOK: type: QUERY
 PREHOOK: Input: default@sdi
@@ -3822,6 +3824,7 @@ STAGE PLANS:
           expanded text: select distinct `src`.`key`, `src`.`value` from `default`.`src` order by `src`.`key` limit 2
           name: default.sdi
           original text: select distinct * from src order by key limit 2
+          rewrite enabled: false
 
 PREHOOK: query: create view sdi as select distinct * from src order by key limit 2
 PREHOOK: type: CREATEVIEW
@@ -3874,6 +3877,7 @@ Sort Columns:       	[]
 # View Information	 	 
 View Original Text: 	select distinct * from src order by key limit 2	 
 View Expanded Text: 	select distinct `src`.`key`, `src`.`value` from `default`.`src` order by `src`.`key` limit 2	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select * from sdi
 PREHOOK: type: QUERY
 PREHOOK: Input: default@sdi

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/llap/subquery_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_views.q.out b/ql/src/test/results/clientpositive/llap/subquery_views.q.out
index 35e80ae..4cd0485 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_views.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_views.q.out
@@ -45,7 +45,7 @@ from `default`.`src` `b`
 where exists	 	 
   (select `a`.`key` 	 	 
   from `default`.`src` `a` 	 	 
-  where `b`.`value` = `a`.`value`  and `a`.`key` = `b`.`key` and `a`.`value` > 'val_9'), tableType:VIRTUAL_VIEW)		 
+  where `b`.`value` = `a`.`value`  and `a`.`key` = `b`.`key` and `a`.`value` > 'val_9'), tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * 
 from cv1 where cv1.key in (select key from cv1 c where c.key > '95')
 PREHOOK: type: QUERY
@@ -110,7 +110,7 @@ where `b`.`key` not in
   (select `a`.`key` 	 	 
   from `default`.`src` `a` 	 	 
   where `b`.`value` = `a`.`value`  and `a`.`key` = `b`.`key` and `a`.`value` > 'val_11'	 	 
-  ), tableType:VIRTUAL_VIEW)		 
+  ), tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 Warning: Shuffle Join MERGEJOIN[67][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 Warning: Shuffle Join MERGEJOIN[69][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 9' is a cross product
 PREHOOK: query: explain
@@ -465,7 +465,7 @@ having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.k
 from `default`.`src` `b`	 	 
 where `b`.`key` in (select `src`.`key` from `default`.`src` where `src`.`key` > '8')	 	 
 group by `b`.`key`, `b`.`value`	 	 
-having count(*) in (select count(*) from `default`.`src` `s1` where `s1`.`key` > '9' group by `s1`.`key` ), tableType:VIRTUAL_VIEW)		 
+having count(*) in (select count(*) from `default`.`src` `s1` where `s1`.`key` > '9' group by `s1`.`key` ), tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from cv3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@cv3

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out b/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out
index 304d74f..10609d9 100644
--- a/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/unionDistinct_2.q.out
@@ -365,7 +365,7 @@ union
 select `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
 union all	 	 
 select `u3`.`key` as `key`, `u3`.`value` from `default`.`u3`	 	 
-) `tab`, tableType:VIRTUAL_VIEW)		 
+) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -435,7 +435,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union  	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW)		 
+) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1
@@ -500,7 +500,7 @@ select distinct * from u2
 select distinct `u1`.`key`, `u1`.`value` from `default`.`u1`	 	 
 union all 	 	 
 select distinct `u2`.`key`, `u2`.`value` from `default`.`u2`	 	 
-) `tab`, tableType:VIRTUAL_VIEW)		 
+) `tab`, tableType:VIRTUAL_VIEW, rewriteEnabled:false)		 
 PREHOOK: query: select * from v
 PREHOOK: type: QUERY
 PREHOOK: Input: default@u1

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/llap/union_top_level.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/union_top_level.q.out b/ql/src/test/results/clientpositive/llap/union_top_level.q.out
index b4e4d93..6ea1f7e 100644
--- a/ql/src/test/results/clientpositive/llap/union_top_level.q.out
+++ b/ql/src/test/results/clientpositive/llap/union_top_level.q.out
@@ -1135,6 +1135,7 @@ union all
 select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b
 union all
 select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c
+          rewrite enabled: false
 
 PREHOOK: query: create view union_top_view as
 select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/materialized_view_create_rewrite.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/materialized_view_create_rewrite.q.out b/ql/src/test/results/clientpositive/materialized_view_create_rewrite.q.out
new file mode 100644
index 0000000..041621f
--- /dev/null
+++ b/ql/src/test/results/clientpositive/materialized_view_create_rewrite.q.out
@@ -0,0 +1,322 @@
+PREHOOK: query: create table cmv_basetable (a int, b varchar(256), c decimal(10,2), d int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cmv_basetable
+POSTHOOK: query: create table cmv_basetable (a int, b varchar(256), c decimal(10,2), d int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cmv_basetable
+PREHOOK: query: insert into cmv_basetable values
+ (1, 'alfred', 10.30, 2),
+ (2, 'bob', 3.14, 3),
+ (2, 'bonnie', 172342.2, 3),
+ (3, 'calvin', 978.76, 3),
+ (3, 'charlie', 9.8, 1)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@cmv_basetable
+POSTHOOK: query: insert into cmv_basetable values
+ (1, 'alfred', 10.30, 2),
+ (2, 'bob', 3.14, 3),
+ (2, 'bonnie', 172342.2, 3),
+ (3, 'calvin', 978.76, 3),
+ (3, 'charlie', 9.8, 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@cmv_basetable
+POSTHOOK: Lineage: cmv_basetable.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.d EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+PREHOOK: query: create materialized view cmv_mat_view enable rewrite
+as select a, b, c from cmv_basetable where a = 2
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
+PREHOOK: Input: default@cmv_basetable
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cmv_mat_view
+POSTHOOK: query: create materialized view cmv_mat_view enable rewrite
+as select a, b, c from cmv_basetable where a = 2
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
+POSTHOOK: Input: default@cmv_basetable
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cmv_mat_view
+PREHOOK: query: select * from cmv_mat_view
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cmv_mat_view
+#### A masked pattern was here ####
+POSTHOOK: query: select * from cmv_mat_view
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cmv_mat_view
+#### A masked pattern was here ####
+2	bob	3.14
+2	bonnie	172342.20
+PREHOOK: query: show tblproperties cmv_mat_view
+PREHOOK: type: SHOW_TBLPROPERTIES
+POSTHOOK: query: show tblproperties cmv_mat_view
+POSTHOOK: type: SHOW_TBLPROPERTIES
+numFiles	1
+totalSize	453
+#### A masked pattern was here ####
+PREHOOK: query: create materialized view if not exists cmv_mat_view2 enable rewrite
+as select a, c from cmv_basetable where a = 3
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
+PREHOOK: Input: default@cmv_basetable
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cmv_mat_view2
+POSTHOOK: query: create materialized view if not exists cmv_mat_view2 enable rewrite
+as select a, c from cmv_basetable where a = 3
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
+POSTHOOK: Input: default@cmv_basetable
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cmv_mat_view2
+PREHOOK: query: select * from cmv_mat_view2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cmv_mat_view2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from cmv_mat_view2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cmv_mat_view2
+#### A masked pattern was here ####
+3	978.76
+3	9.80
+PREHOOK: query: show tblproperties cmv_mat_view2
+PREHOOK: type: SHOW_TBLPROPERTIES
+POSTHOOK: query: show tblproperties cmv_mat_view2
+POSTHOOK: type: SHOW_TBLPROPERTIES
+numFiles	1
+totalSize	322
+#### A masked pattern was here ####
+PREHOOK: query: explain
+select a, c from cmv_basetable where a = 3
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select a, c from cmv_basetable where a = 3
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: default.cmv_mat_view2
+          Statistics: Num rows: 2 Data size: 322 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: a (type: int), c (type: decimal(10,2))
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 2 Data size: 322 Basic stats: COMPLETE Column stats: NONE
+            ListSink
+
+PREHOOK: query: select a, c from cmv_basetable where a = 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cmv_basetable
+PREHOOK: Input: default@cmv_mat_view2
+#### A masked pattern was here ####
+POSTHOOK: query: select a, c from cmv_basetable where a = 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cmv_basetable
+POSTHOOK: Input: default@cmv_mat_view2
+#### A masked pattern was here ####
+3	978.76
+3	9.80
+Warning: Shuffle Join JOIN[7][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: explain
+select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: default.cmv_mat_view2
+            Statistics: Num rows: 2 Data size: 322 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: c (type: decimal(10,2))
+              outputColumnNames: _col0
+              Statistics: Num rows: 2 Data size: 322 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 2 Data size: 322 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: decimal(10,2))
+          TableScan
+            alias: cmv_basetable
+            Statistics: Num rows: 5 Data size: 81 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((d = 3) and (3 = a)) (type: boolean)
+              Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: c (type: decimal(10,2))
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: decimal(10,2))
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: 3 (type: int), _col0 (type: decimal(10,2)), 3 (type: int), _col1 (type: decimal(10,2))
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[7][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cmv_basetable
+PREHOOK: Input: default@cmv_mat_view2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cmv_basetable
+POSTHOOK: Input: default@cmv_mat_view2
+#### A masked pattern was here ####
+3	9.80	3	978.76
+3	978.76	3	978.76
+PREHOOK: query: drop materialized view cmv_mat_view2
+PREHOOK: type: DROP_MATERIALIZED_VIEW
+PREHOOK: Input: default@cmv_mat_view2
+PREHOOK: Output: default@cmv_mat_view2
+POSTHOOK: query: drop materialized view cmv_mat_view2
+POSTHOOK: type: DROP_MATERIALIZED_VIEW
+POSTHOOK: Input: default@cmv_mat_view2
+POSTHOOK: Output: default@cmv_mat_view2
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: explain
+select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: cmv_basetable
+            Statistics: Num rows: 5 Data size: 81 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (a = 3) (type: boolean)
+              Statistics: Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: c (type: decimal(10,2))
+                outputColumnNames: _col0
+                Statistics: Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 2 Data size: 32 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: decimal(10,2))
+          TableScan
+            alias: cmv_basetable
+            Statistics: Num rows: 5 Data size: 81 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((d = 3) and (3 = a)) (type: boolean)
+              Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: c (type: decimal(10,2))
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: decimal(10,2))
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 2 Data size: 66 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: 3 (type: int), _col0 (type: decimal(10,2)), 3 (type: int), _col1 (type: decimal(10,2))
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 2 Data size: 66 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 2 Data size: 66 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cmv_basetable
+#### A masked pattern was here ####
+POSTHOOK: query: select * from (
+  (select a, c from cmv_basetable where a = 3) table1
+  join
+  (select a, c from cmv_basetable where d = 3) table2
+  on table1.a = table2.a)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cmv_basetable
+#### A masked pattern was here ####
+3	9.80	3	978.76
+3	978.76	3	978.76

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/materialized_view_create_rewrite_multi_db.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/materialized_view_create_rewrite_multi_db.q.out b/ql/src/test/results/clientpositive/materialized_view_create_rewrite_multi_db.q.out
new file mode 100644
index 0000000..f715803
--- /dev/null
+++ b/ql/src/test/results/clientpositive/materialized_view_create_rewrite_multi_db.q.out
@@ -0,0 +1,157 @@
+PREHOOK: query: create database db1
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:db1
+POSTHOOK: query: create database db1
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:db1
+PREHOOK: query: use db1
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:db1
+POSTHOOK: query: use db1
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:db1
+PREHOOK: query: create table cmv_basetable (a int, b varchar(256), c decimal(10,2), d int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:db1
+PREHOOK: Output: db1@cmv_basetable
+POSTHOOK: query: create table cmv_basetable (a int, b varchar(256), c decimal(10,2), d int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:db1
+POSTHOOK: Output: db1@cmv_basetable
+PREHOOK: query: insert into cmv_basetable values
+ (1, 'alfred', 10.30, 2),
+ (2, 'bob', 3.14, 3),
+ (2, 'bonnie', 172342.2, 3),
+ (3, 'calvin', 978.76, 3),
+ (3, 'charlie', 9.8, 1)
+PREHOOK: type: QUERY
+PREHOOK: Output: db1@cmv_basetable
+POSTHOOK: query: insert into cmv_basetable values
+ (1, 'alfred', 10.30, 2),
+ (2, 'bob', 3.14, 3),
+ (2, 'bonnie', 172342.2, 3),
+ (3, 'calvin', 978.76, 3),
+ (3, 'charlie', 9.8, 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: db1@cmv_basetable
+POSTHOOK: Lineage: cmv_basetable.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.b EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: cmv_basetable.d EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col4, type:string, comment:), ]
+PREHOOK: query: create database db2
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:db2
+POSTHOOK: query: create database db2
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:db2
+PREHOOK: query: use db2
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:db2
+POSTHOOK: query: use db2
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:db2
+PREHOOK: query: create materialized view cmv_mat_view enable rewrite
+as select a, b, c from db1.cmv_basetable where a = 2
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
+PREHOOK: Input: db1@cmv_basetable
+PREHOOK: Output: database:db2
+PREHOOK: Output: db2@cmv_mat_view
+POSTHOOK: query: create materialized view cmv_mat_view enable rewrite
+as select a, b, c from db1.cmv_basetable where a = 2
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
+POSTHOOK: Input: db1@cmv_basetable
+POSTHOOK: Output: database:db2
+POSTHOOK: Output: db2@cmv_mat_view
+PREHOOK: query: select * from cmv_mat_view
+PREHOOK: type: QUERY
+PREHOOK: Input: db2@cmv_mat_view
+#### A masked pattern was here ####
+POSTHOOK: query: select * from cmv_mat_view
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db2@cmv_mat_view
+#### A masked pattern was here ####
+2	bob	3.14
+2	bonnie	172342.20
+PREHOOK: query: show tblproperties cmv_mat_view
+PREHOOK: type: SHOW_TBLPROPERTIES
+POSTHOOK: query: show tblproperties cmv_mat_view
+POSTHOOK: type: SHOW_TBLPROPERTIES
+numFiles	1
+totalSize	453
+#### A masked pattern was here ####
+PREHOOK: query: create materialized view if not exists cmv_mat_view2 enable rewrite
+as select a, c from db1.cmv_basetable where a = 3
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
+PREHOOK: Input: db1@cmv_basetable
+PREHOOK: Output: database:db2
+PREHOOK: Output: db2@cmv_mat_view2
+POSTHOOK: query: create materialized view if not exists cmv_mat_view2 enable rewrite
+as select a, c from db1.cmv_basetable where a = 3
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
+POSTHOOK: Input: db1@cmv_basetable
+POSTHOOK: Output: database:db2
+POSTHOOK: Output: db2@cmv_mat_view2
+PREHOOK: query: select * from cmv_mat_view2
+PREHOOK: type: QUERY
+PREHOOK: Input: db2@cmv_mat_view2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from cmv_mat_view2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db2@cmv_mat_view2
+#### A masked pattern was here ####
+3	978.76
+3	9.80
+PREHOOK: query: show tblproperties cmv_mat_view2
+PREHOOK: type: SHOW_TBLPROPERTIES
+POSTHOOK: query: show tblproperties cmv_mat_view2
+POSTHOOK: type: SHOW_TBLPROPERTIES
+numFiles	1
+totalSize	322
+#### A masked pattern was here ####
+PREHOOK: query: create database db3
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:db3
+POSTHOOK: query: create database db3
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:db3
+PREHOOK: query: use db3
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:db3
+POSTHOOK: query: use db3
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:db3
+PREHOOK: query: explain
+select a, c from db1.cmv_basetable where a = 3
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select a, c from db1.cmv_basetable where a = 3
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: db2.cmv_mat_view2
+          Statistics: Num rows: 2 Data size: 322 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: a (type: int), c (type: decimal(10,2))
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 2 Data size: 322 Basic stats: COMPLETE Column stats: NONE
+            ListSink
+
+PREHOOK: query: select a, c from db1.cmv_basetable where a = 3
+PREHOOK: type: QUERY
+PREHOOK: Input: db1@cmv_basetable
+PREHOOK: Input: db2@cmv_mat_view2
+#### A masked pattern was here ####
+POSTHOOK: query: select a, c from db1.cmv_basetable where a = 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: db1@cmv_basetable
+POSTHOOK: Input: db2@cmv_mat_view2
+#### A masked pattern was here ####
+3	978.76
+3	9.80

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/materialized_view_describe.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/materialized_view_describe.q.out b/ql/src/test/results/clientpositive/materialized_view_describe.q.out
index 65d94d3..92de293 100644
--- a/ql/src/test/results/clientpositive/materialized_view_describe.q.out
+++ b/ql/src/test/results/clientpositive/materialized_view_describe.q.out
@@ -79,6 +79,11 @@ Compressed:         	No
 Num Buckets:        	-1                  	 
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
+	 	 
+# View Information	 	 
+View Original Text: 	select a, c from cmv_basetable	 
+View Expanded Text: 	null                	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: show tblproperties cmv_mat_view
 PREHOOK: type: SHOW_TBLPROPERTIES
 POSTHOOK: query: show tblproperties cmv_mat_view
@@ -157,6 +162,11 @@ Compressed:         	No
 Num Buckets:        	-1                  	 
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
+	 	 
+# View Information	 	 
+View Original Text: 	select a from cmv_basetable	 
+View Expanded Text: 	null                	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select a from cmv_mat_view2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@cmv_mat_view2
@@ -228,6 +238,11 @@ Compressed:         	No
 Num Buckets:        	-1                  	 
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
+	 	 
+# View Information	 	 
+View Original Text: 	select * from cmv_basetable	 
+View Expanded Text: 	null                	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select a, b, c from cmv_mat_view3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@cmv_mat_view3
@@ -310,6 +325,11 @@ Compressed:         	No
 Num Buckets:        	-1                  	 
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
+	 	 
+# View Information	 	 
+View Original Text: 	select a from cmv_basetable	 
+View Expanded Text: 	null                	 
+View Rewrite Enabled:	No                  	 
 PREHOOK: query: select a from cmv_mat_view4
 PREHOOK: type: QUERY
 PREHOOK: Input: default@cmv_mat_view4

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/test/results/clientpositive/spark/union_top_level.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_top_level.q.out b/ql/src/test/results/clientpositive/spark/union_top_level.q.out
index e1c7fc7..9be5361 100644
--- a/ql/src/test/results/clientpositive/spark/union_top_level.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_top_level.q.out
@@ -1002,6 +1002,7 @@ union all
 select * from (select key, 1 as value from src where key % 3 == 1 limit 3)b
 union all
 select * from (select key, 2 as value from src where key % 3 == 2 limit 3)c
+          rewrite enabled: false
 
 PREHOOK: query: create view union_top_view as
 select * from (select key, 0 as value from src where key % 3 == 0 limit 3)a


[3/6] hive git commit: HIVE-14496: Enable Calcite rewriting with materialized views (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/SubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/SubstitutionVisitor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/SubstitutionVisitor.java
new file mode 100644
index 0000000..93dcc0e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/SubstitutionVisitor.java
@@ -0,0 +1,2458 @@
+/**
+ * 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.optimizer.calcite.rules.views;
+
+import static org.apache.calcite.rex.RexUtil.andNot;
+import static org.apache.calcite.rex.RexUtil.removeAll;
+import static org.apache.calcite.rex.RexUtil.simplify;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.calcite.avatica.util.Spaces;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RexImplicationChecker;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexExecutorImpl;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mapping;
+import org.apache.calcite.util.mapping.Mappings;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Equivalence;
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+
+/**
+ * Substitutes part of a tree of relational expressions with another tree.
+ *
+ * <p>The call {@code new SubstitutionVisitor(target, query).go(replacement))}
+ * will return {@code query} with every occurrence of {@code target} replaced
+ * by {@code replacement}.</p>
+ *
+ * <p>The following example shows how {@code SubstitutionVisitor} can be used
+ * for materialized view recognition.</p>
+ *
+ * <ul>
+ * <li>query = SELECT a, c FROM t WHERE x = 5 AND b = 4</li>
+ * <li>target = SELECT a, b, c FROM t WHERE x = 5</li>
+ * <li>replacement = SELECT * FROM mv</li>
+ * <li>result = SELECT a, c FROM mv WHERE b = 4</li>
+ * </ul>
+ *
+ * <p>Note that {@code result} uses the materialized view table {@code mv} and a
+ * simplified condition {@code b = 4}.</p>
+ *
+ * <p>Uses a bottom-up matching algorithm. Nodes do not need to be identical.
+ * At each level, returns the residue.</p>
+ *
+ * <p>The inputs must only include the core relational operators:
+ * {@link org.apache.calcite.rel.logical.LogicalTableScan},
+ * {@link org.apache.calcite.rel.logical.LogicalFilter},
+ * {@link org.apache.calcite.rel.logical.LogicalProject},
+ * {@link org.apache.calcite.rel.logical.LogicalJoin},
+ * {@link org.apache.calcite.rel.logical.LogicalUnion},
+ * {@link org.apache.calcite.rel.logical.LogicalAggregate}.</p>
+ *
+ * TODO: Remove when we upgrade to Calcite version using builders.
+ */
+public class SubstitutionVisitor {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SubstitutionVisitor.class);
+
+  /** Equivalence that compares objects by their {@link Object#toString()}
+   * method. */
+  private static final Equivalence<Object> STRING_EQUIVALENCE =
+      new Equivalence<Object>() {
+        @Override protected boolean doEquivalent(Object o, Object o2) {
+          return o.toString().equals(o2.toString());
+        }
+
+        @Override protected int doHash(Object o) {
+          return o.toString().hashCode();
+        }
+      };
+
+  /** Equivalence that compares {@link Lists}s by the
+   * {@link Object#toString()} of their elements. */
+  @SuppressWarnings("unchecked")
+  private static final Equivalence<List<?>> PAIRWISE_STRING_EQUIVALENCE =
+      (Equivalence) STRING_EQUIVALENCE.pairwise();
+
+  protected static final ImmutableList<UnifyRule> DEFAULT_RULES =
+      ImmutableList.<UnifyRule>of(
+          TrivialRule.INSTANCE,
+          ScanToProjectUnifyRule.INSTANCE,
+          ProjectToProjectUnifyRule.INSTANCE,
+          FilterToProjectUnifyRule.INSTANCE,
+//          ProjectToFilterUnifyRule.INSTANCE,
+//          FilterToFilterUnifyRule.INSTANCE,
+          AggregateToAggregateUnifyRule.INSTANCE,
+          AggregateOnProjectToAggregateUnifyRule.INSTANCE);
+
+  /**
+   * Factory for a builder for relational expressions.
+   * <p>The actual builder is available via {@link RelOptRuleCall#builder()}.
+   */
+  protected final RelBuilder relBuilder;
+
+  private final ImmutableList<UnifyRule> rules;
+  private final Map<Pair<Class, Class>, List<UnifyRule>> ruleMap =
+      new HashMap<>();
+  private final RelOptCluster cluster;
+  private final Holder query;
+  private final MutableRel target;
+
+  /**
+   * Nodes in {@link #target} that have no children.
+   */
+  final List<MutableRel> targetLeaves;
+
+  /**
+   * Nodes in {@link #query} that have no children.
+   */
+  final List<MutableRel> queryLeaves;
+
+  final Map<MutableRel, MutableRel> replacementMap = new HashMap<>();
+
+  final Multimap<MutableRel, MutableRel> equivalents =
+      LinkedHashMultimap.create();
+
+  /** Workspace while rule is being matched.
+   * Careful, re-entrant!
+   * Assumes no rule needs more than 2 slots. */
+  protected final MutableRel[] slots = new MutableRel[2];
+
+  /** Creates a SubstitutionVisitor with the default rule set. */
+  public SubstitutionVisitor(RelNode target_, RelNode query_) {
+    this(target_, query_, DEFAULT_RULES);
+  }
+
+  public SubstitutionVisitor(RelNode target_, RelNode query_,
+          ImmutableList<UnifyRule> rules) {
+    this(target_, query_, rules, RelFactories.LOGICAL_BUILDER);
+  }
+
+  /** Creates a SubstitutionVisitor. */
+  public SubstitutionVisitor(RelNode target_, RelNode query_,
+      ImmutableList<UnifyRule> rules, RelBuilderFactory relBuilderFactory) {
+    this.cluster = target_.getCluster();
+    this.rules = rules;
+    this.query = Holder.of(toMutable(query_));
+    this.target = toMutable(target_);
+    this.relBuilder = relBuilderFactory.create(cluster, null);
+    final Set<MutableRel> parents = Sets.newIdentityHashSet();
+    final List<MutableRel> allNodes = new ArrayList<>();
+    final MutableRelVisitor visitor =
+        new MutableRelVisitor() {
+          public void visit(MutableRel node) {
+            parents.add(node.parent);
+            allNodes.add(node);
+            super.visit(node);
+          }
+        };
+    visitor.go(target);
+
+    // Populate the list of leaves in the tree under "target".
+    // Leaves are all nodes that are not parents.
+    // For determinism, it is important that the list is in scan order.
+    allNodes.removeAll(parents);
+    targetLeaves = ImmutableList.copyOf(allNodes);
+
+    allNodes.clear();
+    parents.clear();
+    visitor.go(query);
+    allNodes.removeAll(parents);
+    queryLeaves = ImmutableList.copyOf(allNodes);
+  }
+
+  private static MutableRel toMutable(RelNode rel) {
+    if (rel instanceof TableScan) {
+      return MutableScan.of((TableScan) rel);
+    }
+    if (rel instanceof Values) {
+      return MutableValues.of((Values) rel);
+    }
+    if (rel instanceof Project) {
+      final Project project = (Project) rel;
+      final MutableRel input = toMutable(project.getInput());
+      return MutableProject.of(input, project.getProjects(),
+          project.getRowType().getFieldNames());
+    }
+    if (rel instanceof Filter) {
+      final Filter filter = (Filter) rel;
+      final MutableRel input = toMutable(filter.getInput());
+      return MutableFilter.of(input, filter.getCondition());
+    }
+    if (rel instanceof Aggregate) {
+      final Aggregate aggregate = (Aggregate) rel;
+      final MutableRel input = toMutable(aggregate.getInput());
+      return MutableAggregate.of(input, aggregate.indicator,
+          aggregate.getGroupSet(), aggregate.getGroupSets(),
+          aggregate.getAggCallList());
+    }
+    if (rel instanceof Join) {
+      final Join join = (Join) rel;
+      final MutableRel left = toMutable(join.getLeft());
+      final MutableRel right = toMutable(join.getRight());
+      return MutableJoin.of(join.getCluster(), left, right,
+          join.getCondition(), join.getJoinType(), join.getVariablesSet());
+    }
+    if (rel instanceof Sort) {
+      final Sort sort = (Sort) rel;
+      final MutableRel input = toMutable(sort.getInput());
+      return MutableSort.of(input, sort.getCollation(), sort.offset, sort.fetch);
+    }
+    throw new RuntimeException("cannot translate " + rel + " to MutableRel");
+  }
+
+  void register(MutableRel result, MutableRel query) {
+  }
+
+  /**
+   * Maps a condition onto a target.
+   *
+   * <p>If condition is stronger than target, returns the residue.
+   * If it is equal to target, returns the expression that evaluates to
+   * the constant {@code true}. If it is weaker than target, returns
+   * {@code null}.</p>
+   *
+   * <p>The terms satisfy the relation</p>
+   *
+   * <pre>
+   *     {@code condition = target AND residue}
+   * </pre>
+   *
+   * <p>and {@code residue} must be as weak as possible.</p>
+   *
+   * <p>Example #1: condition stronger than target</p>
+   * <ul>
+   * <li>condition: x = 1 AND y = 2</li>
+   * <li>target: x = 1</li>
+   * <li>residue: y = 2</li>
+   * </ul>
+   *
+   * <p>Note that residue {@code x &gt; 0 AND y = 2} would also satisfy the
+   * relation {@code condition = target AND residue} but is stronger than
+   * necessary, so we prefer {@code y = 2}.</p>
+   *
+   * <p>Example #2: target weaker than condition (valid, but not currently
+   * implemented)</p>
+   * <ul>
+   * <li>condition: x = 1</li>
+   * <li>target: x = 1 OR z = 3</li>
+   * <li>residue: NOT (z = 3)</li>
+   * </ul>
+   *
+   * <p>Example #3: condition and target are equivalent</p>
+   * <ul>
+   * <li>condition: x = 1 AND y = 2</li>
+   * <li>target: y = 2 AND x = 1</li>
+   * <li>residue: TRUE</li>
+   * </ul>
+   *
+   * <p>Example #4: condition weaker than target</p>
+   * <ul>
+   * <li>condition: x = 1</li>
+   * <li>target: x = 1 AND y = 2</li>
+   * <li>residue: null (i.e. no match)</li>
+   * </ul>
+   *
+   * <p>There are many other possible examples. It amounts to solving
+   * whether {@code condition AND NOT target} can ever evaluate to
+   * true, and therefore is a form of the NP-complete
+   * <a href="http://en.wikipedia.org/wiki/Satisfiability">Satisfiability</a>
+   * problem.</p>
+   */
+  @VisibleForTesting
+  public static RexNode splitFilter(
+      final RexBuilder rexBuilder, RexNode condition, RexNode target) {
+    // First, try splitting into ORs.
+    // Given target    c1 OR c2 OR c3 OR c4
+    // and condition   c2 OR c4
+    // residue is      NOT c1 AND NOT c3
+    // Also deals with case target [x] condition [x] yields residue [true].
+    RexNode z = splitOr(rexBuilder, condition, target);
+    if (z != null) {
+      return z;
+    }
+
+    RexNode x = andNot(rexBuilder, target, condition);
+    if (mayBeSatisfiable(x)) {
+      RexNode x2 = andNot(rexBuilder, condition, target);
+      return simplify(rexBuilder, x2);
+    }
+    return null;
+  }
+
+  private static RexNode splitOr(
+      final RexBuilder rexBuilder, RexNode condition, RexNode target) {
+    List<RexNode> targets = RelOptUtil.disjunctions(target);
+    for (RexNode e : RelOptUtil.disjunctions(condition)) {
+      boolean found = removeAll(targets, e);
+      if (!found) {
+        return null;
+      }
+    }
+    return RexUtil.composeConjunction(rexBuilder,
+        Lists.transform(targets, RexUtil.notFn(rexBuilder)), false);
+  }
+
+  /**
+   * Returns whether a boolean expression ever returns true.
+   *
+   * <p>This method may give false positives. For instance, it will say
+   * that {@code x = 5 AND x > 10} is satisfiable, because at present it
+   * cannot prove that it is not.</p>
+   */
+  public static boolean mayBeSatisfiable(RexNode e) {
+    // Example:
+    //  e: x = 1 AND y = 2 AND z = 3 AND NOT (x = 1 AND y = 2)
+    //  disjunctions: {x = 1, y = 2, z = 3}
+    //  notDisjunctions: {x = 1 AND y = 2}
+    final List<RexNode> disjunctions = new ArrayList<>();
+    final List<RexNode> notDisjunctions = new ArrayList<>();
+    RelOptUtil.decomposeConjunction(e, disjunctions, notDisjunctions);
+
+    // If there is a single FALSE or NOT TRUE, the whole expression is
+    // always false.
+    for (RexNode disjunction : disjunctions) {
+      switch (disjunction.getKind()) {
+      case LITERAL:
+        if (!RexLiteral.booleanValue(disjunction)) {
+          return false;
+        }
+      }
+    }
+    for (RexNode disjunction : notDisjunctions) {
+      switch (disjunction.getKind()) {
+      case LITERAL:
+        if (RexLiteral.booleanValue(disjunction)) {
+          return false;
+        }
+      }
+    }
+    // If one of the not-disjunctions is a disjunction that is wholly
+    // contained in the disjunctions list, the expression is not
+    // satisfiable.
+    //
+    // Example #1. x AND y AND z AND NOT (x AND y)  - not satisfiable
+    // Example #2. x AND y AND NOT (x AND y)        - not satisfiable
+    // Example #3. x AND y AND NOT (x AND y AND z)  - may be satisfiable
+    for (RexNode notDisjunction : notDisjunctions) {
+      final List<RexNode> disjunctions2 =
+          RelOptUtil.conjunctions(notDisjunction);
+      if (disjunctions.containsAll(disjunctions2)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public RelNode go0(RelNode replacement_) {
+    assert false; // not called
+    MutableRel replacement = toMutable(replacement_);
+    assert MutableRels.equalType(
+        "target", target, "replacement", replacement, Litmus.THROW);
+    replacementMap.put(target, replacement);
+    final UnifyResult unifyResult = matchRecurse(target);
+    if (unifyResult == null) {
+      return null;
+    }
+    final MutableRel node0 = unifyResult.result;
+    MutableRel node = node0; // replaceAncestors(node0);
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug("Convert: query:\n"
+          + query.deep()
+          + "\nunify.query:\n"
+          + unifyResult.call.query.deep()
+          + "\nunify.result:\n"
+          + unifyResult.result.deep()
+          + "\nunify.target:\n"
+          + unifyResult.call.target.deep()
+          + "\nnode0:\n"
+          + node0.deep()
+          + "\nnode:\n"
+          + node.deep());
+    }
+    return fromMutable(node);
+  }
+
+  /**
+   * Returns a list of all possible rels that result from substituting the
+   * matched RelNode with the replacement RelNode within the query.
+   *
+   * <p>For example, the substitution result of A join B, while A and B
+   * are both a qualified match for replacement R, is R join B, R join R,
+   * A join R.
+   */
+  public List<RelNode> go(RelNode replacement_) {
+    List<List<Replacement>> matches = go(toMutable(replacement_));
+    if (matches.isEmpty()) {
+      return ImmutableList.of();
+    }
+    List<RelNode> sub = Lists.newArrayList();
+    sub.add(fromMutable(query.input));
+    reverseSubstitute(query, matches, sub, 0, matches.size());
+    return sub;
+  }
+
+  /**
+   * Substitutes the query with replacement whenever possible but meanwhile
+   * keeps track of all the substitutions and their original rel before
+   * replacement, so that in later processing stage, the replacement can be
+   * recovered individually to produce a list of all possible rels with
+   * substitution in different places.
+   */
+  private List<List<Replacement>> go(MutableRel replacement) {
+    assert MutableRels.equalType(
+        "target", target, "replacement", replacement, Litmus.THROW);
+    final List<MutableRel> queryDescendants = MutableRels.descendants(query);
+    final List<MutableRel> targetDescendants = MutableRels.descendants(target);
+
+    // Populate "equivalents" with (q, t) for each query descendant q and
+    // target descendant t that are equal.
+    final Map<MutableRel, MutableRel> map = Maps.newHashMap();
+    for (MutableRel queryDescendant : queryDescendants) {
+      map.put(queryDescendant, queryDescendant);
+    }
+    for (MutableRel targetDescendant : targetDescendants) {
+      MutableRel queryDescendant = map.get(targetDescendant);
+      if (queryDescendant != null) {
+        assert queryDescendant.rowType.equals(targetDescendant.rowType);
+        equivalents.put(queryDescendant, targetDescendant);
+      }
+    }
+    map.clear();
+
+    final List<Replacement> attempted = Lists.newArrayList();
+    List<List<Replacement>> substitutions = Lists.newArrayList();
+
+    for (;;) {
+      int count = 0;
+      MutableRel queryDescendant = query;
+    outer:
+      while (queryDescendant != null) {
+        for (Replacement r : attempted) {
+          if (queryDescendant == r.after) {
+            // This node has been replaced by previous iterations in the
+            // hope to match its ancestors, so the node itself should not
+            // be matched again.
+            queryDescendant = MutableRels.preOrderTraverseNext(queryDescendant);
+            continue outer;
+          }
+        }
+        final MutableRel next = MutableRels.preOrderTraverseNext(queryDescendant);
+        final MutableRel childOrNext =
+            queryDescendant.getInputs().isEmpty()
+                ? next : queryDescendant.getInputs().get(0);
+        for (MutableRel targetDescendant : targetDescendants) {
+          for (UnifyRule rule
+              : applicableRules(queryDescendant, targetDescendant)) {
+            UnifyRuleCall call =
+                rule.match(this, queryDescendant, targetDescendant);
+            if (call != null) {
+              final UnifyResult result = rule.apply(call);
+              if (result != null) {
+                ++count;
+                attempted.add(new Replacement(result.call.query, result.result));
+                MutableRel parent = result.call.query.replaceInParent(result.result);
+
+                // Replace previous equivalents with new equivalents, higher up
+                // the tree.
+                for (int i = 0; i < rule.slotCount; i++) {
+                  Collection<MutableRel> equi = equivalents.get(slots[i]);
+                  if (!equi.isEmpty()) {
+                    equivalents.remove(slots[i], equi.iterator().next());
+                  }
+                }
+                assert result.result.rowType.equals(result.call.query.rowType)
+                    : Pair.of(result.result, result.call.query);
+                equivalents.put(result.result, result.call.query);
+                if (targetDescendant == target) {
+                  // A real substitution happens. We purge the attempted
+                  // replacement list and add them into substitution list.
+                  // Meanwhile we stop matching the descendants and jump
+                  // to the next subtree in pre-order traversal.
+                  if (!target.equals(replacement)) {
+                    Replacement r = MutableRels.replace(
+                        query.input, target, copyMutable(replacement));
+                    assert r != null
+                        : rule + "should have returned a result containing the target.";
+                    attempted.add(r);
+                  }
+                  substitutions.add(ImmutableList.copyOf(attempted));
+                  attempted.clear();
+                  queryDescendant = next;
+                  continue outer;
+                }
+                // We will try walking the query tree all over again to see
+                // if there can be any substitutions after the replacement
+                // attempt.
+                break outer;
+              }
+            }
+          }
+        }
+        queryDescendant = childOrNext;
+      }
+      // Quit the entire loop if:
+      // 1) we have walked the entire query tree with one or more successful
+      //    substitutions, thus count != 0 && attempted.isEmpty();
+      // 2) we have walked the entire query tree but have made no replacement
+      //    attempt, thus count == 0 && attempted.isEmpty();
+      // 3) we had done some replacement attempt in a previous walk, but in
+      //    this one we have not found any potential matches or substitutions,
+      //    thus count == 0 && !attempted.isEmpty().
+      if (count == 0 || attempted.isEmpty()) {
+        break;
+      }
+    }
+    if (!attempted.isEmpty()) {
+      // We had done some replacement attempt in the previous walk, but that
+      // did not lead to any substitutions in this walk, so we need to recover
+      // the replacement.
+      undoReplacement(attempted);
+    }
+    return substitutions;
+  }
+
+  /**
+   * Represents a replacement action: before &rarr; after.
+   */
+  private static class Replacement {
+    final MutableRel before;
+    final MutableRel after;
+
+    Replacement(MutableRel before, MutableRel after) {
+      this.before = before;
+      this.after = after;
+    }
+  }
+
+  private static void undoReplacement(List<Replacement> replacement) {
+    for (int i = replacement.size() - 1; i >= 0; i--) {
+      Replacement r = replacement.get(i);
+      r.after.replaceInParent(r.before);
+    }
+  }
+
+  private static void redoReplacement(List<Replacement> replacement) {
+    for (Replacement r : replacement) {
+      r.before.replaceInParent(r.after);
+    }
+  }
+
+  private void reverseSubstitute(Holder query,
+      List<List<Replacement>> matches, List<RelNode> sub,
+      int replaceCount, int maxCount) {
+    if (matches.isEmpty()) {
+      return;
+    }
+    final List<List<Replacement>> rem = matches.subList(1, matches.size());
+    reverseSubstitute(query, rem, sub, replaceCount, maxCount);
+    undoReplacement(matches.get(0));
+    if (++replaceCount < maxCount) {
+      sub.add(fromMutable(query.input));
+    }
+    reverseSubstitute(query, rem, sub, replaceCount, maxCount);
+    redoReplacement(matches.get(0));
+  }
+
+  private List<RelNode> fromMutables(List<MutableRel> nodes) {
+    return Lists.transform(nodes,
+        new Function<MutableRel, RelNode>() {
+          public RelNode apply(MutableRel mutableRel) {
+            return fromMutable(mutableRel);
+          }
+        });
+  }
+
+  private RelNode fromMutable(MutableRel node) {
+    switch (node.type) {
+    case SCAN:
+    case VALUES:
+      return ((MutableLeafRel) node).rel;
+    case PROJECT:
+      final MutableProject project = (MutableProject) node;
+      relBuilder.push(fromMutable(project.input));
+      relBuilder.project(project.projects);
+      return relBuilder.build();
+    case FILTER:
+      final MutableFilter filter = (MutableFilter) node;
+      relBuilder.push(fromMutable(filter.input));
+      relBuilder.filter(filter.condition);
+      return relBuilder.build();
+    case AGGREGATE:
+      final MutableAggregate aggregate = (MutableAggregate) node;
+      return LogicalAggregate.create(fromMutable(aggregate.input),
+          aggregate.indicator, aggregate.groupSet, aggregate.groupSets,
+          aggregate.aggCalls);
+    case SORT:
+      final MutableSort sort = (MutableSort) node;
+      return LogicalSort.create(fromMutable(sort.input), sort.collation,
+          sort.offset, sort.fetch);
+    case UNION:
+      final MutableUnion union = (MutableUnion) node;
+      return LogicalUnion.create(fromMutables(union.inputs), union.all);
+    case JOIN:
+      final MutableJoin join = (MutableJoin) node;
+      return LogicalJoin.create(fromMutable(join.getLeft()), fromMutable(join.getRight()),
+          join.getCondition(), join.getVariablesSet(), join.getJoinType());
+    default:
+      throw new AssertionError(node.deep());
+    }
+  }
+
+  private static List<MutableRel> copyMutables(List<MutableRel> nodes) {
+    return Lists.transform(nodes,
+        new Function<MutableRel, MutableRel>() {
+          public MutableRel apply(MutableRel mutableRel) {
+            return copyMutable(mutableRel);
+          }
+        });
+  }
+
+  private static MutableRel copyMutable(MutableRel node) {
+    switch (node.type) {
+    case SCAN:
+      return MutableScan.of((TableScan) ((MutableScan) node).rel);
+    case VALUES:
+      return MutableValues.of((Values) ((MutableValues) node).rel);
+    case PROJECT:
+      final MutableProject project = (MutableProject) node;
+      return MutableProject.of(project.rowType,
+          copyMutable(project.input), project.projects);
+    case FILTER:
+      final MutableFilter filter = (MutableFilter) node;
+      return MutableFilter.of(copyMutable(filter.input), filter.condition);
+    case AGGREGATE:
+      final MutableAggregate aggregate = (MutableAggregate) node;
+      return MutableAggregate.of(copyMutable(aggregate.input),
+          aggregate.indicator, aggregate.groupSet, aggregate.groupSets,
+          aggregate.aggCalls);
+    case SORT:
+      final MutableSort sort = (MutableSort) node;
+      return MutableSort.of(copyMutable(sort.input), sort.collation,
+          sort.offset, sort.fetch);
+    case UNION:
+      final MutableUnion union = (MutableUnion) node;
+      return MutableUnion.of(copyMutables(union.inputs), union.all);
+    case JOIN:
+      final MutableJoin join = (MutableJoin) node;
+      return MutableJoin.of(join.cluster, copyMutable(join.getLeft()),
+          copyMutable(join.getRight()), join.getCondition(), join.getJoinType(),
+          join.getVariablesSet());
+    default:
+      throw new AssertionError(node.deep());
+    }
+  }
+
+  private UnifyResult matchRecurse(MutableRel target) {
+    assert false; // not called
+    final List<MutableRel> targetInputs = target.getInputs();
+    MutableRel queryParent = null;
+
+    for (MutableRel targetInput : targetInputs) {
+      UnifyResult unifyResult = matchRecurse(targetInput);
+      if (unifyResult == null) {
+        return null;
+      }
+      queryParent = unifyResult.call.query.replaceInParent(unifyResult.result);
+    }
+
+    if (targetInputs.isEmpty()) {
+      for (MutableRel queryLeaf : queryLeaves) {
+        for (UnifyRule rule : applicableRules(queryLeaf, target)) {
+          final UnifyResult x = apply(rule, queryLeaf, target);
+          if (x != null) {
+            if (LOGGER.isDebugEnabled()) {
+              LOGGER.debug("Rule: " + rule
+                  + "\nQuery:\n"
+                  + queryParent
+                  + (x.call.query != queryParent
+                     ? "\nQuery (original):\n"
+                     + queryParent
+                     : "")
+                  + "\nTarget:\n"
+                  + target.deep()
+                  + "\nResult:\n"
+                  + x.result.deep()
+                  + "\n");
+            }
+            return x;
+          }
+        }
+      }
+    } else {
+      assert queryParent != null;
+      for (UnifyRule rule : applicableRules(queryParent, target)) {
+        final UnifyResult x = apply(rule, queryParent, target);
+        if (x != null) {
+          if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug(
+                "Rule: " + rule
+                + "\nQuery:\n"
+                + queryParent.deep()
+                + (x.call.query != queryParent
+                   ? "\nQuery (original):\n"
+                   + queryParent.toString()
+                   : "")
+                + "\nTarget:\n"
+                + target.deep()
+                + "\nResult:\n"
+                + x.result.deep()
+                + "\n");
+          }
+          return x;
+        }
+      }
+    }
+    if (LOGGER.isDebugEnabled()) {
+      LOGGER.debug(
+          "Unify failed:"
+          + "\nQuery:\n"
+          + queryParent.toString()
+          + "\nTarget:\n"
+          + target.toString()
+          + "\n");
+    }
+    return null;
+  }
+
+  private UnifyResult apply(UnifyRule rule, MutableRel query,
+      MutableRel target) {
+    final UnifyRuleCall call = new UnifyRuleCall(rule, query, target, null);
+    return rule.apply(call);
+  }
+
+  private List<UnifyRule> applicableRules(MutableRel query,
+      MutableRel target) {
+    final Class queryClass = query.getClass();
+    final Class targetClass = target.getClass();
+    final Pair<Class, Class> key = Pair.of(queryClass, targetClass);
+    List<UnifyRule> list = ruleMap.get(key);
+    if (list == null) {
+      final ImmutableList.Builder<UnifyRule> builder =
+          ImmutableList.builder();
+      for (UnifyRule rule : rules) {
+        //noinspection unchecked
+        if (mightMatch(rule, queryClass, targetClass)) {
+          builder.add(rule);
+        }
+      }
+      list = builder.build();
+      ruleMap.put(key, list);
+    }
+    return list;
+  }
+
+  private static boolean mightMatch(UnifyRule rule,
+      Class queryClass, Class targetClass) {
+    return rule.queryOperand.clazz.isAssignableFrom(queryClass)
+        && rule.targetOperand.clazz.isAssignableFrom(targetClass);
+  }
+
+  /** Exception thrown to exit a matcher. Not really an error. */
+  protected static class MatchFailed extends ControlFlowException {
+    @SuppressWarnings("ThrowableInstanceNeverThrown")
+    public static final MatchFailed INSTANCE = new MatchFailed();
+  }
+
+  /** Rule that attempts to match a query relational expression
+   * against a target relational expression.
+   *
+   * <p>The rule declares the query and target types; this allows the
+   * engine to fire only a few rules in a given context.</p>
+   */
+  protected abstract static class UnifyRule {
+    protected final int slotCount;
+    protected final Operand queryOperand;
+    protected final Operand targetOperand;
+
+    protected UnifyRule(int slotCount, Operand queryOperand,
+        Operand targetOperand) {
+      this.slotCount = slotCount;
+      this.queryOperand = queryOperand;
+      this.targetOperand = targetOperand;
+    }
+
+    /**
+     * <p>Applies this rule to a particular node in a query. The goal is
+     * to convert {@code query} into {@code target}. Before the rule is
+     * invoked, Calcite has made sure that query's children are equivalent
+     * to target's children.
+     *
+     * <p>There are 3 possible outcomes:</p>
+     *
+     * <ul>
+     *
+     * <li>{@code query} already exactly matches {@code target}; returns
+     * {@code target}</li>
+     *
+     * <li>{@code query} is sufficiently close to a match for
+     * {@code target}; returns {@code target}</li>
+     *
+     * <li>{@code query} cannot be made to match {@code target}; returns
+     * null</li>
+     *
+     * </ul>
+     *
+     * <p>REVIEW: Is possible that we match query PLUS one or more of its
+     * ancestors?</p>
+     *
+     * @param call Input parameters
+     */
+    protected abstract UnifyResult apply(UnifyRuleCall call);
+
+    protected UnifyRuleCall match(SubstitutionVisitor visitor, MutableRel query,
+        MutableRel target) {
+      if (queryOperand.matches(visitor, query)) {
+        if (targetOperand.matches(visitor, target)) {
+          return visitor.new UnifyRuleCall(this, query, target,
+              copy(visitor.slots, slotCount));
+        }
+      }
+      return null;
+    }
+
+    protected <E> ImmutableList<E> copy(E[] slots, int slotCount) {
+      // Optimize if there are 0 or 1 slots.
+      switch (slotCount) {
+      case 0:
+        return ImmutableList.of();
+      case 1:
+        return ImmutableList.of(slots[0]);
+      default:
+        return ImmutableList.copyOf(slots).subList(0, slotCount);
+      }
+    }
+  }
+
+  /**
+   * Arguments to an application of a {@link UnifyRule}.
+   */
+  protected class UnifyRuleCall {
+    protected final UnifyRule rule;
+    public final MutableRel query;
+    public final MutableRel target;
+    protected final ImmutableList<MutableRel> slots;
+
+    public UnifyRuleCall(UnifyRule rule, MutableRel query, MutableRel target,
+        ImmutableList<MutableRel> slots) {
+      this.rule = Preconditions.checkNotNull(rule);
+      this.query = Preconditions.checkNotNull(query);
+      this.target = Preconditions.checkNotNull(target);
+      this.slots = Preconditions.checkNotNull(slots);
+    }
+
+    public UnifyResult result(MutableRel result) {
+      assert MutableRels.contains(result, target);
+      assert MutableRels.equalType("result", result, "query", query,
+          Litmus.THROW);
+      MutableRel replace = replacementMap.get(target);
+      if (replace != null) {
+        assert false; // replacementMap is always empty
+        // result =
+        MutableRels.replace(result, target, replace);
+      }
+      register(result, query);
+      return new UnifyResult(this, result);
+    }
+
+    /**
+     * Creates a {@link UnifyRuleCall} based on the parent of {@code query}.
+     */
+    public UnifyRuleCall create(MutableRel query) {
+      return new UnifyRuleCall(rule, query, target, slots);
+    }
+
+    public RelOptCluster getCluster() {
+      return cluster;
+    }
+  }
+
+  /**
+   * Result of an application of a {@link UnifyRule} indicating that the
+   * rule successfully matched {@code query} against {@code target} and
+   * generated a {@code result} that is equivalent to {@code query} and
+   * contains {@code target}.
+   */
+  protected static class UnifyResult {
+    private final UnifyRuleCall call;
+    // equivalent to "query", contains "result"
+    private final MutableRel result;
+
+    UnifyResult(UnifyRuleCall call, MutableRel result) {
+      this.call = call;
+      assert MutableRels.equalType("query", call.query, "result", result,
+          Litmus.THROW);
+      this.result = result;
+    }
+  }
+
+  /** Abstract base class for implementing {@link UnifyRule}. */
+  protected abstract static class AbstractUnifyRule extends UnifyRule {
+    public AbstractUnifyRule(Operand queryOperand, Operand targetOperand,
+        int slotCount) {
+      super(slotCount, queryOperand, targetOperand);
+      //noinspection AssertWithSideEffects
+      assert isValid();
+    }
+
+    protected boolean isValid() {
+      final SlotCounter slotCounter = new SlotCounter();
+      slotCounter.visit(queryOperand);
+      assert slotCounter.queryCount == slotCount;
+      assert slotCounter.targetCount == 0;
+      slotCounter.queryCount = 0;
+      slotCounter.visit(targetOperand);
+      assert slotCounter.queryCount == 0;
+      assert slotCounter.targetCount == slotCount;
+      return true;
+    }
+
+    /** Creates an operand with given inputs. */
+    protected static Operand operand(Class<? extends MutableRel> clazz,
+        Operand... inputOperands) {
+      return new InternalOperand(clazz, ImmutableList.copyOf(inputOperands));
+    }
+
+    /** Creates an operand that doesn't check inputs. */
+    protected static Operand any(Class<? extends MutableRel> clazz) {
+      return new AnyOperand(clazz);
+    }
+
+    /** Creates an operand that matches a relational expression in the query. */
+    protected static Operand query(int ordinal) {
+      return new QueryOperand(ordinal);
+    }
+
+    /** Creates an operand that matches a relational expression in the
+     * target. */
+    protected static Operand target(int ordinal) {
+      return new TargetOperand(ordinal);
+    }
+  }
+
+  /** Implementation of {@link UnifyRule} that matches if the query is already
+   * equal to the target.
+   *
+   * <p>Matches scans to the same table, because these will be
+   * {@link MutableScan}s with the same
+   * {@link org.apache.calcite.rel.logical.LogicalTableScan} instance.</p>
+   */
+  private static class TrivialRule extends AbstractUnifyRule {
+    private static final TrivialRule INSTANCE = new TrivialRule();
+
+    private TrivialRule() {
+      super(any(MutableRel.class), any(MutableRel.class), 0);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      if (call.query.equals(call.target)) {
+        return call.result(call.query);
+      }
+      return null;
+    }
+  }
+
+  /** Implementation of {@link UnifyRule} that matches
+   * {@link org.apache.calcite.rel.logical.LogicalTableScan}. */
+  private static class ScanToProjectUnifyRule extends AbstractUnifyRule {
+    public static final ScanToProjectUnifyRule INSTANCE =
+        new ScanToProjectUnifyRule();
+
+    private ScanToProjectUnifyRule() {
+      super(any(MutableScan.class),
+          any(MutableProject.class), 0);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      final MutableProject target = (MutableProject) call.target;
+      final MutableScan query = (MutableScan) call.query;
+      // We do not need to check query's parent type to avoid duplication
+      // of ProjectToProjectUnifyRule or FilterToProjectUnifyRule, since
+      // SubstitutionVisitor performs a top-down match.
+      if (!query.equals(target.getInput())) {
+        return null;
+      }
+      final RexShuttle shuttle = getRexShuttle(target);
+      final RexBuilder rexBuilder = target.cluster.getRexBuilder();
+      final List<RexNode> newProjects;
+      try {
+        newProjects = (List<RexNode>)
+            shuttle.apply(rexBuilder.identityProjects(query.getRowType()));
+      } catch (MatchFailed e) {
+        return null;
+      }
+      final MutableProject newProject =
+          MutableProject.of(
+              query.getRowType(), target, newProjects);
+      final MutableRel newProject2 = MutableRels.strip(newProject);
+      return call.result(newProject2);
+    }
+  }
+
+  /** Implementation of {@link UnifyRule} that matches
+   * {@link org.apache.calcite.rel.logical.LogicalProject}. */
+  private static class ProjectToProjectUnifyRule extends AbstractUnifyRule {
+    public static final ProjectToProjectUnifyRule INSTANCE =
+        new ProjectToProjectUnifyRule();
+
+    private ProjectToProjectUnifyRule() {
+      super(operand(MutableProject.class, query(0)),
+          operand(MutableProject.class, target(0)), 1);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      final MutableProject target = (MutableProject) call.target;
+      final MutableProject query = (MutableProject) call.query;
+      final RexShuttle shuttle = getRexShuttle(target);
+      final List<RexNode> newProjects;
+      try {
+        newProjects = shuttle.apply(query.getProjects());
+      } catch (MatchFailed e) {
+        return null;
+      }
+      final MutableProject newProject =
+          MutableProject.of(
+              query.getRowType(), target, newProjects);
+      final MutableRel newProject2 = MutableRels.strip(newProject);
+      return call.result(newProject2);
+    }
+  }
+
+
+  /** Implementation of {@link UnifyRule} that matches a {@link MutableFilter}
+   * to a {@link MutableProject}. */
+  private static class FilterToProjectUnifyRule extends AbstractUnifyRule {
+    public static final FilterToProjectUnifyRule INSTANCE =
+        new FilterToProjectUnifyRule();
+
+    private FilterToProjectUnifyRule() {
+      super(operand(MutableFilter.class, query(0)),
+          operand(MutableProject.class, target(0)), 1);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      // Child of projectTarget is equivalent to child of filterQuery.
+      try {
+        // TODO: make sure that constants are ok
+        final MutableProject target = (MutableProject) call.target;
+        final RexShuttle shuttle = getRexShuttle(target);
+        final RexNode newCondition;
+        final MutableFilter query = (MutableFilter) call.query;
+        try {
+          newCondition = query.getCondition().accept(shuttle);
+        } catch (MatchFailed e) {
+          return null;
+        }
+        final MutableFilter newFilter = MutableFilter.of(target, newCondition);
+        if (query.parent instanceof MutableProject) {
+          final MutableRel inverse =
+              invert(((MutableProject) query.parent).getNamedProjects(),
+                  newFilter, shuttle);
+          return call.create(query.parent).result(inverse);
+        } else {
+          final MutableRel inverse = invert(query, newFilter, target);
+          return call.result(inverse);
+        }
+      } catch (MatchFailed e) {
+        return null;
+      }
+    }
+
+    protected MutableRel invert(List<Pair<RexNode, String>> namedProjects,
+        MutableRel input,
+        RexShuttle shuttle) {
+      LOGGER.trace("SubstitutionVisitor: invert:\nprojects: {}\ninput: {}\nproject: {}\n",
+          namedProjects, input, shuttle);
+      final List<RexNode> exprList = new ArrayList<>();
+      final RexBuilder rexBuilder = input.cluster.getRexBuilder();
+      final List<RexNode> projects = Pair.left(namedProjects);
+      for (RexNode expr : projects) {
+        exprList.add(rexBuilder.makeZeroLiteral(expr.getType()));
+      }
+      for (Ord<RexNode> expr : Ord.zip(projects)) {
+        final RexNode node = expr.e.accept(shuttle);
+        if (node == null) {
+          throw MatchFailed.INSTANCE;
+        }
+        exprList.set(expr.i, node);
+      }
+      return MutableProject.of(input, exprList, Pair.right(namedProjects));
+    }
+
+    protected MutableRel invert(MutableRel model, MutableRel input,
+        MutableProject project) {
+      LOGGER.trace("SubstitutionVisitor: invert:\nmodel: {}\ninput: {}\nproject: {}\n",
+          model, input, project);
+      if (project.getProjects().size() < model.getRowType().getFieldCount()) {
+        throw MatchFailed.INSTANCE;
+      }
+      final List<RexNode> exprList = new ArrayList<>();
+      final RexBuilder rexBuilder = model.cluster.getRexBuilder();
+      for (RelDataTypeField field : model.getRowType().getFieldList()) {
+        exprList.add(rexBuilder.makeZeroLiteral(field.getType()));
+      }
+      for (Ord<RexNode> expr : Ord.zip(project.getProjects())) {
+        if (expr.e instanceof RexInputRef) {
+          final int target = ((RexInputRef) expr.e).getIndex();
+          exprList.set(target,
+              rexBuilder.ensureType(expr.e.getType(),
+                  RexInputRef.of(expr.i, input.rowType),
+                  false));
+        } else {
+          throw MatchFailed.INSTANCE;
+        }
+      }
+      return MutableProject.of(model.rowType, input, exprList);
+    }
+  }
+
+  /** Implementation of {@link UnifyRule} that matches a
+   * {@link MutableFilter}. */
+  private static class FilterToFilterUnifyRule extends AbstractUnifyRule {
+    public static final FilterToFilterUnifyRule INSTANCE =
+        new FilterToFilterUnifyRule();
+
+    private FilterToFilterUnifyRule() {
+      super(operand(MutableFilter.class, query(0)),
+          operand(MutableFilter.class, target(0)), 1);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      // in.query can be rewritten in terms of in.target if its condition
+      // is weaker. For example:
+      //   query: SELECT * FROM t WHERE x = 1 AND y = 2
+      //   target: SELECT * FROM t WHERE x = 1
+      // transforms to
+      //   result: SELECT * FROM (target) WHERE y = 2
+      final MutableFilter query = (MutableFilter) call.query;
+      final MutableFilter target = (MutableFilter) call.target;
+      final MutableFilter newFilter =
+          createFilter(query, target);
+      if (newFilter == null) {
+        return null;
+      }
+      return call.result(newFilter);
+    }
+
+    MutableFilter createFilter(MutableFilter query, MutableFilter target) {
+      final RexNode newCondition =
+          splitFilter(query.cluster.getRexBuilder(), query.getCondition(),
+              target.getCondition());
+      if (newCondition == null) {
+        // Could not map query onto target.
+        return null;
+      }
+      if (newCondition.isAlwaysTrue()) {
+        return target;
+      }
+      return MutableFilter.of(target, newCondition);
+    }
+  }
+
+  /** Implementation of {@link UnifyRule} that matches a {@link MutableProject}
+   * to a {@link MutableFilter}. */
+  private static class ProjectToFilterUnifyRule extends AbstractUnifyRule {
+    public static final ProjectToFilterUnifyRule INSTANCE =
+        new ProjectToFilterUnifyRule();
+
+    private ProjectToFilterUnifyRule() {
+      super(operand(MutableProject.class, query(0)),
+          operand(MutableFilter.class, target(0)), 1);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      if (call.query.parent instanceof MutableFilter) {
+        final UnifyRuleCall in2 = call.create(call.query.parent);
+        final MutableFilter query = (MutableFilter) in2.query;
+        final MutableFilter target = (MutableFilter) in2.target;
+        final MutableFilter newFilter =
+            FilterToFilterUnifyRule.INSTANCE.createFilter(
+                query, target);
+        if (newFilter == null) {
+          return null;
+        }
+        return in2.result(query.replaceInParent(newFilter));
+      }
+      return null;
+    }
+  }
+
+  /** Implementation of {@link UnifyRule} that matches a
+   * {@link org.apache.calcite.rel.logical.LogicalAggregate} to a
+   * {@link org.apache.calcite.rel.logical.LogicalAggregate}, provided
+   * that they have the same child. */
+  private static class AggregateToAggregateUnifyRule extends AbstractUnifyRule {
+    public static final AggregateToAggregateUnifyRule INSTANCE =
+        new AggregateToAggregateUnifyRule();
+
+    private AggregateToAggregateUnifyRule() {
+      super(operand(MutableAggregate.class, query(0)),
+          operand(MutableAggregate.class, target(0)), 1);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      final MutableAggregate query = (MutableAggregate) call.query;
+      final MutableAggregate target = (MutableAggregate) call.target;
+      assert query != target;
+      // in.query can be rewritten in terms of in.target if its groupSet is
+      // a subset, and its aggCalls are a superset. For example:
+      //   query: SELECT x, COUNT(b) FROM t GROUP BY x
+      //   target: SELECT x, y, SUM(a) AS s, COUNT(b) AS cb FROM t GROUP BY x, y
+      // transforms to
+      //   result: SELECT x, SUM(cb) FROM (target) GROUP BY x
+      if (!target.getGroupSet().contains(query.getGroupSet())) {
+        return null;
+      }
+      MutableRel result = unifyAggregates(query, target);
+      if (result == null) {
+        return null;
+      }
+      return call.result(result);
+    }
+  }
+
+  public static MutableAggregate permute(MutableAggregate aggregate,
+      MutableRel input, Mapping mapping) {
+    ImmutableBitSet groupSet = Mappings.apply(mapping, aggregate.getGroupSet());
+    ImmutableList<ImmutableBitSet> groupSets =
+        Mappings.apply2(mapping, aggregate.getGroupSets());
+    List<AggregateCall> aggregateCalls =
+        apply(mapping, aggregate.getAggCallList());
+    return MutableAggregate.of(input, aggregate.indicator, groupSet, groupSets,
+        aggregateCalls);
+  }
+
+  private static List<AggregateCall> apply(final Mapping mapping,
+      List<AggregateCall> aggCallList) {
+    return Lists.transform(aggCallList,
+        new Function<AggregateCall, AggregateCall>() {
+          public AggregateCall apply(AggregateCall call) {
+            return call.copy(Mappings.apply2(mapping, call.getArgList()),
+                Mappings.apply(mapping, call.filterArg));
+          }
+        });
+  }
+
+  public static MutableRel unifyAggregates(MutableAggregate query,
+      MutableAggregate target) {
+    if (query.getGroupType() != Aggregate.Group.SIMPLE
+        || target.getGroupType() != Aggregate.Group.SIMPLE) {
+      throw new AssertionError(Bug.CALCITE_461_FIXED);
+    }
+    MutableRel result;
+    if (query.getGroupSet().equals(target.getGroupSet())) {
+      // Same level of aggregation. Generate a project.
+      final List<Integer> projects = Lists.newArrayList();
+      final int groupCount = query.getGroupSet().cardinality();
+      for (int i = 0; i < groupCount; i++) {
+        projects.add(i);
+      }
+      for (AggregateCall aggregateCall : query.getAggCallList()) {
+        int i = target.getAggCallList().indexOf(aggregateCall);
+        if (i < 0) {
+          return null;
+        }
+        projects.add(groupCount + i);
+      }
+      result = MutableRels.createProject(target, projects);
+    } else {
+      // Target is coarser level of aggregation. Generate an aggregate.
+      final ImmutableBitSet.Builder groupSet = ImmutableBitSet.builder();
+      final List<Integer> targetGroupList = target.getGroupSet().asList();
+      for (int c : query.getGroupSet()) {
+        int c2 = targetGroupList.indexOf(c);
+        if (c2 < 0) {
+          return null;
+        }
+        groupSet.set(c2);
+      }
+      final List<AggregateCall> aggregateCalls = Lists.newArrayList();
+      for (AggregateCall aggregateCall : query.getAggCallList()) {
+        if (aggregateCall.isDistinct()) {
+          return null;
+        }
+        int i = target.getAggCallList().indexOf(aggregateCall);
+        if (i < 0) {
+          return null;
+        }
+        aggregateCalls.add(
+            AggregateCall.create(getRollup(aggregateCall.getAggregation()),
+                aggregateCall.isDistinct(),
+                ImmutableList.of(target.groupSet.cardinality() + i), -1,
+                aggregateCall.type, aggregateCall.name));
+      }
+      result = MutableAggregate.of(target, false, groupSet.build(), null,
+          aggregateCalls);
+    }
+    return MutableRels.createCastRel(result, query.getRowType(), true);
+  }
+
+  /** Implementation of {@link UnifyRule} that matches a
+   * {@link MutableAggregate} on
+   * a {@link MutableProject} query to an {@link MutableAggregate} target.
+   *
+   * <p>The rule is necessary when we unify query=Aggregate(x) with
+   * target=Aggregate(x, y). Query will tend to have an extra Project(x) on its
+   * input, which this rule knows is safe to ignore.</p> */
+  private static class AggregateOnProjectToAggregateUnifyRule
+      extends AbstractUnifyRule {
+    public static final AggregateOnProjectToAggregateUnifyRule INSTANCE =
+        new AggregateOnProjectToAggregateUnifyRule();
+
+    private AggregateOnProjectToAggregateUnifyRule() {
+      super(
+          operand(MutableAggregate.class,
+              operand(MutableProject.class, query(0))),
+          operand(MutableAggregate.class, target(0)), 1);
+    }
+
+    public UnifyResult apply(UnifyRuleCall call) {
+      final MutableAggregate query = (MutableAggregate) call.query;
+      final MutableAggregate target = (MutableAggregate) call.target;
+      if (!(query.getInput() instanceof MutableProject)) {
+        return null;
+      }
+      final MutableProject project = (MutableProject) query.getInput();
+      if (project.getInput() != target.getInput()) {
+        return null;
+      }
+      final Mappings.TargetMapping mapping = project.getMapping();
+      if (mapping == null) {
+        return null;
+      }
+      final MutableAggregate aggregate2 =
+          permute(query, project.getInput(), mapping.inverse());
+      final MutableRel result = unifyAggregates(aggregate2, target);
+      return result == null ? null : call.result(result);
+    }
+  }
+
+  public static SqlAggFunction getRollup(SqlAggFunction aggregation) {
+    if (aggregation == SqlStdOperatorTable.SUM
+        || aggregation == SqlStdOperatorTable.MIN
+        || aggregation == SqlStdOperatorTable.MAX
+        || aggregation == SqlStdOperatorTable.SUM0) {
+      return aggregation;
+    } else if (aggregation == SqlStdOperatorTable.COUNT) {
+      return SqlStdOperatorTable.SUM0;
+    } else {
+      return null;
+    }
+  }
+
+  /** Builds a shuttle that stores a list of expressions, and can map incoming
+   * expressions to references to them. */
+  protected static RexShuttle getRexShuttle(MutableProject target) {
+    final Map<String, Integer> map = new HashMap<>();
+    for (RexNode e : target.getProjects()) {
+      map.put(e.toString(), map.size());
+    }
+    return new RexShuttle() {
+      @Override public RexNode visitInputRef(RexInputRef ref) {
+        final Integer integer = map.get(ref.getName());
+        if (integer != null) {
+          return new RexInputRef(integer, ref.getType());
+        }
+        throw MatchFailed.INSTANCE;
+      }
+
+      @Override public RexNode visitCall(RexCall call) {
+        final Integer integer = map.get(call.toString());
+        if (integer != null) {
+          return new RexInputRef(integer, call.getType());
+        }
+        return super.visitCall(call);
+      }
+    };
+  }
+
+  /** Type of {@code MutableRel}. */
+  private enum MutableRelType {
+    SCAN,
+    PROJECT,
+    FILTER,
+    AGGREGATE,
+    SORT,
+    UNION,
+    JOIN,
+    HOLDER,
+    VALUES
+  }
+
+  /** Visitor over {@link MutableRel}. */
+  private static class MutableRelVisitor {
+    private MutableRel root;
+
+    public void visit(MutableRel node) {
+      node.childrenAccept(this);
+    }
+
+    public MutableRel go(MutableRel p) {
+      this.root = p;
+      visit(p);
+      return root;
+    }
+  }
+
+  /** Mutable equivalent of {@link RelNode}.
+   *
+   * <p>Each node has mutable state, and keeps track of its parent and position
+   * within parent.
+   * It doesn't make sense to canonize {@code MutableRels},
+   * otherwise one node could end up with multiple parents.
+   * It follows that {@code #hashCode} and {@code #equals} are less efficient
+   * than their {@code RelNode} counterparts.
+   * But, you don't need to copy a {@code MutableRel} in order to change it.
+   * For this reason, you should use {@code MutableRel} for short-lived
+   * operations, and transcribe back to {@code RelNode} when you are done.</p>
+   */
+  protected abstract static class MutableRel {
+    MutableRel parent;
+    int ordinalInParent;
+    public final RelOptCluster cluster;
+    final RelDataType rowType;
+    final MutableRelType type;
+
+    private MutableRel(RelOptCluster cluster, RelDataType rowType,
+        MutableRelType type) {
+      this.cluster = cluster;
+      this.rowType = rowType;
+      this.type = type;
+    }
+
+    public RelDataType getRowType() {
+      return rowType;
+    }
+
+    public abstract void setInput(int ordinalInParent, MutableRel input);
+
+    public abstract List<MutableRel> getInputs();
+
+    public abstract void childrenAccept(MutableRelVisitor visitor);
+
+    /** Replaces this {@code MutableRel} in its parent with another node at the
+     * same position.
+     *
+     * <p>Before the method, {@code child} must be an orphan (have null parent)
+     * and after this method, this {@code MutableRel} is an orphan.
+     *
+     * @return The parent
+     */
+    public MutableRel replaceInParent(MutableRel child) {
+      final MutableRel parent = this.parent;
+      if (this != child) {
+/*
+        if (child.parent != null) {
+          child.parent.setInput(child.ordinalInParent, null);
+          child.parent = null;
+        }
+*/
+        if (parent != null) {
+          parent.setInput(ordinalInParent, child);
+          this.parent = null;
+          this.ordinalInParent = 0;
+        }
+      }
+      return parent;
+    }
+
+    public abstract StringBuilder digest(StringBuilder buf);
+
+    public final String deep() {
+      return new MutableRelDumper().apply(this);
+    }
+
+    @Override public final String toString() {
+      return deep();
+    }
+
+    public MutableRel getParent() { return parent; }
+  }
+
+  /** Implementation of {@link MutableRel} whose only purpose is to have a
+   * child. Used as the root of a tree. */
+  private static class Holder extends MutableSingleRel {
+    private Holder(MutableRelType type, RelDataType rowType, MutableRel input) {
+      super(type, rowType, input);
+    }
+
+    static Holder of(MutableRel input) {
+      return new Holder(MutableRelType.HOLDER, input.rowType, input);
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      return buf.append("Holder");
+    }
+  }
+
+   /** Abstract base class for implementations of {@link MutableRel} that have
+   * no inputs. */
+  protected abstract static class MutableLeafRel extends MutableRel {
+    protected final RelNode rel;
+
+    MutableLeafRel(MutableRelType type, RelNode rel) {
+      super(rel.getCluster(), rel.getRowType(), type);
+      this.rel = rel;
+    }
+
+    public void setInput(int ordinalInParent, MutableRel input) {
+      throw new IllegalArgumentException();
+    }
+
+    public List<MutableRel> getInputs() {
+      return ImmutableList.of();
+    }
+
+    public void childrenAccept(MutableRelVisitor visitor) {
+      // no children - nothing to do
+    }
+  }
+
+  /** Mutable equivalent of {@link SingleRel}. */
+  protected abstract static class MutableSingleRel extends MutableRel {
+    protected MutableRel input;
+
+    MutableSingleRel(MutableRelType type, RelDataType rowType,
+        MutableRel input) {
+      super(input.cluster, rowType, type);
+      this.input = input;
+      input.parent = this;
+      input.ordinalInParent = 0;
+    }
+
+    public void setInput(int ordinalInParent, MutableRel input) {
+      if (ordinalInParent >= 1) {
+        throw new IllegalArgumentException();
+      }
+      this.input = input;
+      if (input != null) {
+        input.parent = this;
+        input.ordinalInParent = 0;
+      }
+    }
+
+    public List<MutableRel> getInputs() {
+      return ImmutableList.of(input);
+    }
+
+    public void childrenAccept(MutableRelVisitor visitor) {
+      visitor.visit(input);
+    }
+
+    public MutableRel getInput() {
+      return input;
+    }
+  }
+
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalTableScan}. */
+  protected static class MutableScan extends MutableLeafRel {
+    private MutableScan(TableScan rel) {
+      super(MutableRelType.SCAN, rel);
+    }
+
+    static MutableScan of(TableScan rel) {
+      return new MutableScan(rel);
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof MutableScan
+          && rel.getTable().equals(((MutableScan) obj).rel.getTable());
+    }
+
+    @Override public int hashCode() {
+      return rel.getTable().hashCode();
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      return buf.append("Scan(table: ")
+          .append(rel.getTable().getQualifiedName()).append(")");
+    }
+  }
+
+  /** Mutable equivalent of {@link org.apache.calcite.rel.core.Values}. */
+  protected static class MutableValues extends MutableLeafRel {
+    private MutableValues(Values rel) {
+      super(MutableRelType.VALUES, rel);
+    }
+
+    static MutableValues of(Values rel) {
+      return new MutableValues(rel);
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof MutableValues
+          && rel == ((MutableValues) obj).rel;
+    }
+
+    @Override public int hashCode() {
+      return rel.hashCode();
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      return buf.append("Values(tuples: ")
+          .append(((Values) rel).getTuples()).append(")");
+    }
+  }
+
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalProject}. */
+  protected static class MutableProject extends MutableSingleRel {
+    private final List<RexNode> projects;
+
+    private MutableProject(RelDataType rowType, MutableRel input,
+        List<RexNode> projects) {
+      super(MutableRelType.PROJECT, rowType, input);
+      this.projects = projects;
+      assert RexUtil.compatibleTypes(projects, rowType, Litmus.THROW);
+    }
+
+    public static MutableProject of(RelDataType rowType, MutableRel input,
+        List<RexNode> projects) {
+      return new MutableProject(rowType, input, projects);
+    }
+
+    /** Equivalent to
+     * {@link RelOptUtil#createProject(org.apache.calcite.rel.RelNode, java.util.List, java.util.List)}
+     * for {@link MutableRel}. */
+    public static MutableRel of(MutableRel child, List<RexNode> exprList,
+        List<String> fieldNameList) {
+      final RelDataType rowType =
+          RexUtil.createStructType(child.cluster.getTypeFactory(), exprList,
+              fieldNameList, SqlValidatorUtil.F_SUGGESTER);
+      return of(rowType, child, exprList);
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof MutableProject
+          && PAIRWISE_STRING_EQUIVALENCE.equivalent(
+              projects, ((MutableProject) obj).projects)
+          && input.equals(((MutableProject) obj).input);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(input,
+          PAIRWISE_STRING_EQUIVALENCE.hash(projects));
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      return buf.append("Project(projects: ").append(projects).append(")");
+    }
+
+    public List<RexNode> getProjects() {
+      return projects;
+    }
+
+    /** Returns a list of (expression, name) pairs. */
+    public final List<Pair<RexNode, String>> getNamedProjects() {
+      return Pair.zip(getProjects(), getRowType().getFieldNames());
+    }
+
+    public Mappings.TargetMapping getMapping() {
+      return Project.getMapping(
+          input.getRowType().getFieldCount(), projects);
+    }
+  }
+
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalFilter}. */
+  protected static class MutableFilter extends MutableSingleRel {
+    private final RexNode condition;
+
+    private MutableFilter(MutableRel input, RexNode condition) {
+      super(MutableRelType.FILTER, input.rowType, input);
+      this.condition = condition;
+    }
+
+    public static MutableFilter of(MutableRel input, RexNode condition) {
+      return new MutableFilter(input, condition);
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof MutableFilter
+          && condition.toString().equals(
+              ((MutableFilter) obj).condition.toString())
+          && input.equals(((MutableFilter) obj).input);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(input, condition.toString());
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      return buf.append("Filter(condition: ").append(condition).append(")");
+    }
+
+    public RexNode getCondition() {
+      return condition;
+    }
+  }
+
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalAggregate}. */
+  protected static class MutableAggregate extends MutableSingleRel {
+    public final boolean indicator;
+    private final ImmutableBitSet groupSet;
+    private final ImmutableList<ImmutableBitSet> groupSets;
+    private final List<AggregateCall> aggCalls;
+
+    private MutableAggregate(MutableRel input, RelDataType rowType,
+        boolean indicator, ImmutableBitSet groupSet,
+        List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
+      super(MutableRelType.AGGREGATE, rowType, input);
+      this.indicator = indicator;
+      this.groupSet = groupSet;
+      this.groupSets = groupSets == null
+          ? ImmutableList.of(groupSet)
+          : ImmutableList.copyOf(groupSets);
+      this.aggCalls = aggCalls;
+    }
+
+    static MutableAggregate of(MutableRel input, boolean indicator,
+        ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
+        List<AggregateCall> aggCalls) {
+      RelDataType rowType =
+          Aggregate.deriveRowType(input.cluster.getTypeFactory(),
+              input.getRowType(), indicator, groupSet, groupSets, aggCalls);
+      return new MutableAggregate(input, rowType, indicator, groupSet,
+          groupSets, aggCalls);
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof MutableAggregate
+          && groupSet.equals(((MutableAggregate) obj).groupSet)
+          && aggCalls.equals(((MutableAggregate) obj).aggCalls)
+          && input.equals(((MutableAggregate) obj).input);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(input, groupSet, aggCalls);
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      return buf.append("Aggregate(groupSet: ").append(groupSet)
+          .append(", groupSets: ").append(groupSets)
+          .append(", calls: ").append(aggCalls).append(")");
+    }
+
+    public ImmutableBitSet getGroupSet() {
+      return groupSet;
+    }
+
+    public ImmutableList<ImmutableBitSet> getGroupSets() {
+      return groupSets;
+    }
+
+    public List<AggregateCall> getAggCallList() {
+      return aggCalls;
+    }
+
+    public Aggregate.Group getGroupType() {
+      return Aggregate.Group.induce(groupSet, groupSets);
+    }
+  }
+
+  /** Mutable equivalent of {@link org.apache.calcite.rel.core.Sort}. */
+  protected static class MutableSort extends MutableSingleRel {
+    private final RelCollation collation;
+    private final RexNode offset;
+    private final RexNode fetch;
+
+    private MutableSort(MutableRel input, RelCollation collation,
+        RexNode offset, RexNode fetch) {
+      super(MutableRelType.SORT, input.rowType, input);
+      this.collation = collation;
+      this.offset = offset;
+      this.fetch = fetch;
+    }
+
+    static MutableSort of(MutableRel input, RelCollation collation,
+        RexNode offset, RexNode fetch) {
+      return new MutableSort(input, collation, offset, fetch);
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof MutableSort
+          && collation.equals(((MutableSort) obj).collation)
+          && Objects.equals(offset, ((MutableSort) obj).offset)
+          && Objects.equals(fetch, ((MutableSort) obj).fetch)
+          && input.equals(((MutableSort) obj).input);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(input, collation, offset, fetch);
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      buf.append("Sort(collation: ").append(collation);
+      if (offset != null) {
+        buf.append(", offset: ").append(offset);
+      }
+      if (fetch != null) {
+        buf.append(", fetch: ").append(fetch);
+      }
+      return buf.append(")");
+    }
+  }
+
+  /** Base class for set-operations. */
+  protected abstract static class MutableSetOp extends MutableRel {
+    protected final List<MutableRel> inputs;
+
+    private MutableSetOp(RelOptCluster cluster, RelDataType rowType,
+        MutableRelType type, List<MutableRel> inputs) {
+      super(cluster, rowType, type);
+      this.inputs = inputs;
+    }
+
+    @Override public void setInput(int ordinalInParent, MutableRel input) {
+      inputs.set(ordinalInParent, input);
+      if (input != null) {
+        input.parent = this;
+        input.ordinalInParent = ordinalInParent;
+      }
+    }
+
+    @Override public List<MutableRel> getInputs() {
+      return inputs;
+    }
+
+    @Override public void childrenAccept(MutableRelVisitor visitor) {
+      for (MutableRel input : inputs) {
+        visitor.visit(input);
+      }
+    }
+  }
+
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalUnion}. */
+  protected static class MutableUnion extends MutableSetOp {
+    public boolean all;
+
+    private MutableUnion(RelOptCluster cluster, RelDataType rowType,
+        List<MutableRel> inputs, boolean all) {
+      super(cluster, rowType, MutableRelType.UNION, inputs);
+      this.all = all;
+    }
+
+    static MutableUnion of(List<MutableRel> inputs, boolean all) {
+      assert inputs.size() >= 2;
+      final MutableRel input0 = inputs.get(0);
+      return new MutableUnion(input0.cluster, input0.rowType, inputs, all);
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof MutableUnion
+          && inputs.equals(((MutableUnion) obj).getInputs());
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(type, inputs);
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      return buf.append("Union");
+    }
+  }
+
+  /** Base Class for relations with two inputs */
+  private abstract static class MutableBiRel extends MutableRel {
+    protected MutableRel left;
+    protected MutableRel right;
+
+    MutableBiRel(MutableRelType type, RelOptCluster cluster, RelDataType rowType,
+                        MutableRel left, MutableRel right) {
+      super(cluster, rowType, type);
+      this.left = left;
+      left.parent = this;
+      left.ordinalInParent = 0;
+
+      this.right = right;
+      right.parent = this;
+      right.ordinalInParent = 1;
+    }
+
+    public void setInput(int ordinalInParent, MutableRel input) {
+      if (ordinalInParent > 1) {
+        throw new IllegalArgumentException();
+      }
+      if (ordinalInParent == 0) {
+        this.left = input;
+      } else {
+        this.right = input;
+      }
+      if (input != null) {
+        input.parent = this;
+        input.ordinalInParent = ordinalInParent;
+      }
+    }
+
+    public List<MutableRel> getInputs() {
+      return ImmutableList.of(left, right);
+    }
+
+    public MutableRel getLeft() {
+      return left;
+    }
+
+    public MutableRel getRight() {
+      return right;
+    }
+
+    public void childrenAccept(MutableRelVisitor visitor) {
+
+      visitor.visit(left);
+      visitor.visit(right);
+    }
+  }
+
+  /** Mutable equivalent of
+   * {@link org.apache.calcite.rel.logical.LogicalJoin}. */
+  private static class MutableJoin extends MutableBiRel {
+    //~ Instance fields --------------------------------------------------------
+
+    protected final RexNode condition;
+    protected final ImmutableSet<CorrelationId> variablesSet;
+
+    /**
+     * Values must be of enumeration {@link JoinRelType}, except that
+     * {@link JoinRelType#RIGHT} is disallowed.
+     */
+    protected JoinRelType joinType;
+
+    private MutableJoin(
+        RelDataType rowType,
+        MutableRel left,
+        MutableRel right,
+        RexNode condition,
+        JoinRelType joinType,
+        Set<CorrelationId> variablesSet) {
+      super(MutableRelType.JOIN, left.cluster, rowType, left, right);
+      this.condition = Preconditions.checkNotNull(condition);
+      this.variablesSet = ImmutableSet.copyOf(variablesSet);
+      this.joinType = Preconditions.checkNotNull(joinType);
+    }
+
+    public RexNode getCondition() {
+      return condition;
+    }
+
+    public JoinRelType getJoinType() {
+      return joinType;
+    }
+
+    public ImmutableSet<CorrelationId> getVariablesSet() {
+      return variablesSet;
+    }
+
+    static MutableJoin of(RelOptCluster cluster, MutableRel left,
+        MutableRel right, RexNode condition, JoinRelType joinType,
+        Set<CorrelationId> variablesStopped) {
+      List<RelDataTypeField> fieldList = Collections.emptyList();
+      RelDataType rowType =
+          SqlValidatorUtil.deriveJoinRowType(left.getRowType(),
+              right.getRowType(), joinType, cluster.getTypeFactory(), null,
+              fieldList);
+      return new MutableJoin(rowType, left, right, condition, joinType,
+          variablesStopped);
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof MutableJoin
+          && joinType == ((MutableJoin) obj).joinType
+          && condition.toString().equals(
+              ((MutableJoin) obj).condition.toString())
+          && left.equals(((MutableJoin) obj).left)
+          && right.equals(((MutableJoin) obj).right);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(left, right, condition.toString(), joinType);
+    }
+
+    @Override public StringBuilder digest(StringBuilder buf) {
+      return buf.append("Join(left: ").append(left)
+          .append(", right:").append(right)
+          .append(")");
+    }
+  }
+
+  /** Utilities for dealing with {@link MutableRel}s. */
+  protected static class MutableRels {
+    public static boolean contains(MutableRel ancestor,
+        final MutableRel target) {
+      if (ancestor.equals(target)) {
+        // Short-cut common case.
+        return true;
+      }
+      try {
+        new MutableRelVisitor() {
+          @Override public void visit(MutableRel node) {
+            if (node.equals(target)) {
+              throw Util.FoundOne.NULL;
+            }
+            super.visit(node);
+          }
+          // CHECKSTYLE: IGNORE 1
+        }.go(ancestor);
+        return false;
+      } catch (Util.FoundOne e) {
+        return true;
+      }
+    }
+
+    public static MutableRel preOrderTraverseNext(MutableRel node) {
+      MutableRel parent = node.getParent();
+      int ordinal = node.ordinalInParent + 1;
+      while (parent != null) {
+        if (parent.getInputs().size() > ordinal) {
+          return parent.getInputs().get(ordinal);
+        }
+        node = parent;
+        parent = node.getParent();
+        ordinal = node.ordinalInParent + 1;
+      }
+      return null;
+    }
+
+    private static List<MutableRel> descendants(MutableRel query) {
+      final List<MutableRel> list = new ArrayList<>();
+      descendantsRecurse(list, query);
+      return list;
+    }
+
+    private static void descendantsRecurse(List<MutableRel> list,
+        MutableRel rel) {
+      list.add(rel);
+      for (MutableRel input : rel.getInputs()) {
+        descendantsRecurse(list, input);
+      }
+    }
+
+    /** Returns whether two relational expressions have the same row-type. */
+    public static boolean equalType(String desc0, MutableRel rel0, String desc1,
+        MutableRel rel1, Litmus litmus) {
+      return RelOptUtil.equal(desc0, rel0.getRowType(),
+          desc1, rel1.getRowType(), litmus);
+    }
+
+    /** Within a relational expression {@code query}, replaces occurrences of
+     * {@code find} with {@code replace}.
+     *
+     * <p>Assumes relational expressions (and their descendants) are not null.
+     * Does not handle cycles. */
+    public static Replacement replace(MutableRel query, MutableRel find,
+        MutableRel replace) {
+      if (find.equals(replace)) {
+        // Short-cut common case.
+        return null;
+      }
+      assert equalType("find", find, "replace", replace, Litmus.THROW);
+      return replaceRecurse(query, find, replace);
+    }
+
+    /** Helper for {@link #replace}. */
+    private static Replacement replaceRecurse(MutableRel query,
+        MutableRel find, MutableRel replace) {
+      if (find.equals(query)) {
+        query.replaceInParent(replace);
+        return new Replacement(query, replace);
+      }
+      for (MutableRel input : query.getInputs()) {
+        Replacement r = replaceRecurse(input, find, replace);
+        if (r != null) {
+          return r;
+        }
+      }
+      return null;
+    }
+
+    /** Based on
+     * {@link org.apache.calcite.rel.rules.ProjectRemoveRule#strip}. */
+    public static MutableRel strip(MutableProject project) {
+      return isTrivial(project) ? project.getInput() : project;
+    }
+
+    /** Based on
+     * {@link org.apache.calcite.rel.rules.ProjectRemoveRule#isTrivial(org.apache.calcite.rel.core.Project)}. */
+    public static boolean isTrivial(MutableProject project) {
+      MutableRel child = project.getInput();
+      final RelDataType childRowType = child.getRowType();
+      return RexUtil.isIdentity(project.getProjects(), childRowType);
+    }
+
+    /** Equivalent to
+     * {@link RelOptUtil#createProject(org.apache.calcite.rel.RelNode, java.util.List)}
+     * for {@link MutableRel}. */
+    public static MutableRel createProject(final MutableRel child,
+        final List<Integer> posList) {
+      final RelDataType rowType = child.getRowType();
+      if (Mappings.isIdentity(posList, rowType.getFieldCount())) {
+        return child;
+      }
+      return MutableProject.of(
+          RelOptUtil.permute(child.cluster.getTypeFactory(), rowType,
+              Mappings.bijection(posList)),
+          child,
+          new AbstractList<RexNode>() {
+            public int size() {
+              return posList.size();
+            }
+
+            public RexNode get(int index) {
+              final int pos = posList.get(index);
+              return RexInputRef.of(pos, rowType);
+            }
+          });
+    }
+
+    /** Equivalence to {@link org.apache.calcite.plan.RelOptUtil#createCastRel}
+     * for {@link MutableRel}. */
+    public static MutableRel createCastRel(MutableRel rel,
+        RelDataType castRowType, boolean rename) {
+      RelDataType rowType = rel.getRowType();
+      if (RelOptUtil.areRowTypesEqual(rowType, castRowType, rename)) {
+        // nothing to do
+        return rel;
+      }
+      List<RexNode> castExps =
+          RexUtil.generateCastExpressions(rel.cluster.getRexBuilder(),
+              castRowType, rowType);
+      final List<String> fieldNames =
+          rename ? castRowType.getFieldNames() : rowType.getFieldNames();
+      return MutableProject.of(rel, castExps, fieldNames);
+    }
+  }
+
+  /** Visitor that prints an indented tree of {@link MutableRel}s. */
+  protected static class MutableRelDumper extends MutableRelVisitor {
+    private final StringBuilder buf = new StringBuilder();
+    private int level;
+
+    @Override public void visit(MutableRel node) {
+      Spaces.append(buf, level * 2);
+      if (node == null) {
+        buf.append("null");
+      } else {
+        node.digest(buf);
+        buf.append("\n");
+        ++level;
+        super.visit(node);
+        --level;
+      }
+    }
+
+    public String apply(MutableRel rel) {
+      go(rel);
+      return buf.toString();
+    }
+  }
+
+  /** Returns if one rel is weaker than another. */
+  protected boolean isWeaker(MutableRel rel0, MutableRel rel) {
+    if (rel0 == rel || equivalents.get(rel0).contains(rel)) {
+      return false;
+    }
+
+    if (!(rel0 instanceof MutableFilter)
+        || !(rel instanceof MutableFilter)) {
+      return false;
+    }
+
+    if (!rel.getRowType().equals(rel0.getRowType())) {
+      return false;
+    }
+
+    final MutableRel rel0input = ((MutableFilter) rel0).getInput();
+    final MutableRel relinput = ((MutableFilter) rel).getInput();
+    if (rel0input != relinput
+        && !equivalents.get(rel0input).contains(relinput)) {
+      return false;
+    }
+
+    RexExecutorImpl rexImpl =
+        (RexExecutorImpl) (rel.cluster.getPlanner().getExecutor());
+    RexImplicationChecker rexImplicationChecker = new RexImplicationChecker(
+        rel.cluster.getRexBuilder(),
+        rexImpl, rel.getRowType());
+
+    return rexImplicationChecker.implies(((MutableFilter) rel0).getCondition(),
+        ((MutableFilter) rel).getCondition());
+  }
+
+  /** Operand to a {@link UnifyRule}. */
+  protected abstract static class Operand {
+    protected final Class<? extends MutableRel> clazz;
+
+    protected Operand(Class<? extends MutableRel> clazz) {
+      this.clazz = clazz;
+    }
+
+    public abstract boolean matches(SubstitutionVisitor visitor, MutableRel rel);
+
+    public boolean isWeaker(SubstitutionVisitor visitor, MutableRel rel) {
+      return false;
+    }
+  }
+
+  /** Operand to a {@link UnifyRule} that matches a relational expression of a
+   * given type. It has zero or more child operands. */
+  private static class InternalOperand extends Operand {
+    private final List<Operand> inputs;
+
+    InternalOperand(Class<? extends MutableRel> clazz, List<Operand> inputs) {
+      super(clazz);
+      this.inputs = inputs;
+    }
+
+    @Override public boolean matches(SubstitutionVisitor visitor, MutableRel rel) {
+      return clazz.isInstance(rel)
+          && allMatch(visitor, inputs, rel.getInputs());
+    }
+
+    @Override public boolean isWeaker(SubstitutionVisitor visitor, MutableRel rel) {
+      return clazz.isInstance(rel)
+          && allWeaker(visitor, inputs, rel.getInputs());
+    }
+    private static boolean allMatch(SubstitutionVisitor visitor,
+        List<Operand> operands, List<MutableRel> rels) {
+      if (operands.size() != rels.size()) {
+        return false;
+      }
+      for (Pair<Operand, MutableRel> pair : Pair.zip(operands, rels)) {
+        if (!pair.left.matches(visitor, pair.right)) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    private static boolean allWeaker(
+        SubstitutionVisitor visitor,
+        List<Operand> operands, List<MutableRel> rels) {
+      if (operands.size() != rels.size()) {
+        return false;
+      }
+      for (Pair<Operand, MutableRel> pair : Pair.zip(operands, rels)) {
+        if (!pair.left.isWeaker(visitor, pair.right)) {
+          return false;
+        }
+      }
+      return true;
+    }
+  }
+
+  /** Operand to a {@link UnifyRule} that matches a relational expression of a
+   * given type. */
+  private static class AnyOperand extends Operand {
+    AnyOperand(Class<? extends MutableRel> clazz) {
+      super(clazz);
+    }
+
+    @Override public boolean matches(SubstitutionVisitor visitor, MutableRel rel) {
+      return clazz.isInstance(rel);
+    }
+  }
+
+  /** Operand that assigns a particular relational expression to a variable.
+   *
+   * <p>It is applied to a descendant of the query, writes the operand into the
+   * slots array, and always matches.
+   * There is a corresponding operand of type {@link TargetOperand} that checks
+   * whether its relational expression, a descendant of the target, is
+   * equivalent to this {@code QueryOperand}'s relational expression.
+   */
+  private static class QueryOperand extends Operand {
+    private final int ordinal;
+
+    protected QueryOperand(int ordinal) {
+      super(MutableRel.class);
+      this.ordinal = ordinal;
+    }
+
+    @Override public boolean matches(SubstitutionVisitor visitor, MutableRel rel) {
+      visitor.slots[ordinal] = rel;
+      return true;
+    }
+  }
+
+  /** Operand that checks that a relational expression matches the corresponding
+   * relational expression that was passed to a {@link QueryOperand}. */
+  private static class TargetOperand extends Operand {
+    private final int ordinal;
+
+    protected TargetOperand(int ordinal) {
+      super(MutableRel.class);
+      this.ordinal = ordinal;
+    }
+
+    @Override public boolean matches(SubstitutionVisitor visitor, MutableRel rel) {
+      final MutableRel rel0 = visitor.slots[ordinal];
+      assert rel0 != null : "QueryOperand should have been called first";
+      return rel0 == rel || visitor.equivalents.get(rel0).contains(rel);
+    }
+
+    @Override public boolean isWeaker(SubstitutionVisitor visitor, MutableRel rel) {
+      final MutableRel rel0 = visitor.slots[ordinal];
+      assert rel0 != null : "QueryOperand should have been called first";
+      return visitor.isWeaker(rel0, rel);
+    }
+  }
+
+  /** Visitor that counts how many {@link QueryOperand} and
+   * {@link TargetOperand} in an operand tree. */
+  private static class SlotCounter {
+    int queryCount;
+    int targetCount;
+
+    void visit(Operand operand) {
+      if (operand instanceof QueryOperand) {
+        ++queryCount;
+      } else if (operand instanceof TargetOperand) {
+        ++targetCount;
+      } else if (operand instanceof AnyOperand) {
+        // nothing
+      } else {
+        for (Operand input : ((InternalOperand) operand).inputs) {
+          visit(input);
+        }
+      }
+    }
+  }
+
+  /**
+   * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalFilter}
+   * on top of a {@link org.apache.calcite.rel.logical.LogicalProject} into a
+   * trivial filter (on a boolean column).
+   */
+  public static class FilterOnProjectRule extends RelOptRule {
+    private static final Predicate<Filter> PREDICATE =
+        new Predicate<Filter>() {
+          public boolean apply(Filter input) {
+            return input.getCondition() instanceof RexInputRef;
+          }
+        };
+
+    public static final FilterOnProjectRule INSTANCE =
+        new FilterOnProjectRule();
+
+    private FilterOnProjectRule() {
+      super(
+          operand(Filter.class, null, PREDICATE,
+              some(operand(Project.class, any()))));
+    }
+
+    public void onMatch(RelOptRuleCall call) {
+      final Filter filter = call.rel(0);
+      final Project project = call.rel(1);
+
+      final List<RexNode> newProjects = new ArrayList<>(project.getProjects());
+      newProjects.add(filter.getCondition());
+
+      final RelOptCluster cluster = filter.getCluster();
+      RelDataType newRowType =
+          cluster.getTypeFactory().builder()
+              .addAll(project.getRowType().getFieldList())
+              .add("condition", Util.last(newProjects).getType())
+              .build();
+      final RelNode newProject =
+          project.copy(project.getTraitSet(),
+              project.getInput(),
+              newProjects,
+              newRowType);
+
+      final RexInputRef newCondition =
+          cluster.getRexBuilder().makeInputRef(newProject,
+              newProjects.size() - 1);
+
+      call.transformTo(filter.copy(filter.getTraitSet(), newProject, newCondition));
+    }
+  }
+}
+
+// End SubstitutionVisitor.java


[5/6] hive git commit: HIVE-14496: Enable Calcite rewriting with materialized views (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index f7b2ed7..d0a66b0 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -18,25 +18,44 @@
 
 package org.apache.hadoop.hive.metastore;
 
-import org.apache.hadoop.hive.metastore.api.ClientCapabilities;
-import org.apache.hadoop.hive.metastore.api.ClientCapability;
-import org.apache.hadoop.hive.metastore.api.CompactionResponse;
-import org.apache.hadoop.hive.metastore.api.GetTableRequest;
-import org.apache.hadoop.hive.metastore.api.GetTableResult;
-import org.apache.hadoop.hive.metastore.api.GetTablesRequest;
-import org.apache.hadoop.hive.metastore.api.GetTablesResult;
-import org.apache.hadoop.hive.metastore.api.MetaException;
+import static org.apache.commons.lang.StringUtils.join;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.validateName;
 
-import com.facebook.fb303.FacebookBase;
-import com.facebook.fb303.fb_status;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Splitter;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableListMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimaps;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Formatter;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.Timer;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Pattern;
+
+import javax.jdo.JDOException;
 
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.hadoop.conf.Configuration;
@@ -45,9 +64,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.JvmPauseMonitor;
 import org.apache.hadoop.hive.common.LogUtils;
+import org.apache.hadoop.hive.common.LogUtils.LogInitializationException;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.auth.HiveAuthUtils;
-import org.apache.hadoop.hive.common.LogUtils.LogInitializationException;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
 import org.apache.hadoop.hive.common.cli.CommonCliOptions;
@@ -57,119 +76,7 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.common.metrics.common.MetricsVariable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
-import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
-import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
-import org.apache.hadoop.hive.metastore.api.AddForeignKeyRequest;
-import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest;
-import org.apache.hadoop.hive.metastore.api.AddPartitionsResult;
-import org.apache.hadoop.hive.metastore.api.AddPrimaryKeyRequest;
-import org.apache.hadoop.hive.metastore.api.AggrStats;
-import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.CacheFileMetadataRequest;
-import org.apache.hadoop.hive.metastore.api.CacheFileMetadataResult;
-import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
-import org.apache.hadoop.hive.metastore.api.ClearFileMetadataRequest;
-import org.apache.hadoop.hive.metastore.api.ClearFileMetadataResult;
-import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
-import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
-import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
-import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
-import org.apache.hadoop.hive.metastore.api.CompactionRequest;
-import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
-import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.DropConstraintRequest;
-import org.apache.hadoop.hive.metastore.api.DropPartitionsExpr;
-import org.apache.hadoop.hive.metastore.api.DropPartitionsRequest;
-import org.apache.hadoop.hive.metastore.api.DropPartitionsResult;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
-import org.apache.hadoop.hive.metastore.api.FireEventRequest;
-import org.apache.hadoop.hive.metastore.api.FireEventResponse;
-import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
-import org.apache.hadoop.hive.metastore.api.ForeignKeysResponse;
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
-import org.apache.hadoop.hive.metastore.api.GetFileMetadataByExprRequest;
-import org.apache.hadoop.hive.metastore.api.GetFileMetadataByExprResult;
-import org.apache.hadoop.hive.metastore.api.GetFileMetadataRequest;
-import org.apache.hadoop.hive.metastore.api.GetFileMetadataResult;
-import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
-import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse;
-import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
-import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse;
-import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
-import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
-import org.apache.hadoop.hive.metastore.api.GrantRevokePrivilegeRequest;
-import org.apache.hadoop.hive.metastore.api.GrantRevokePrivilegeResponse;
-import org.apache.hadoop.hive.metastore.api.GrantRevokeRoleRequest;
-import org.apache.hadoop.hive.metastore.api.GrantRevokeRoleResponse;
-import org.apache.hadoop.hive.metastore.api.HeartbeatRequest;
-import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeRequest;
-import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
-import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
-import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
-import org.apache.hadoop.hive.metastore.api.HiveObjectType;
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.api.InvalidInputException;
-import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
-import org.apache.hadoop.hive.metastore.api.LockRequest;
-import org.apache.hadoop.hive.metastore.api.LockResponse;
-import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
-import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
-import org.apache.hadoop.hive.metastore.api.NotificationEventRequest;
-import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
-import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
-import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.PartitionEventType;
-import org.apache.hadoop.hive.metastore.api.PartitionListComposingSpec;
-import org.apache.hadoop.hive.metastore.api.PartitionSpec;
-import org.apache.hadoop.hive.metastore.api.PartitionSpecWithSharedSD;
-import org.apache.hadoop.hive.metastore.api.PartitionWithoutSD;
-import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
-import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
-import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest;
-import org.apache.hadoop.hive.metastore.api.PartitionsStatsResult;
-import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
-import org.apache.hadoop.hive.metastore.api.PrimaryKeysResponse;
-import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
-import org.apache.hadoop.hive.metastore.api.PrincipalType;
-import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
-import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
-import org.apache.hadoop.hive.metastore.api.PutFileMetadataRequest;
-import org.apache.hadoop.hive.metastore.api.PutFileMetadataResult;
-import org.apache.hadoop.hive.metastore.api.RequestPartsSpec;
-import org.apache.hadoop.hive.metastore.api.Role;
-import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
-import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
-import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
-import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
-import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
-import org.apache.hadoop.hive.metastore.api.SkewedInfo;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.api.TableMeta;
-import org.apache.hadoop.hive.metastore.api.TableStatsRequest;
-import org.apache.hadoop.hive.metastore.api.TableStatsResult;
-import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
-import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
-import org.apache.hadoop.hive.metastore.api.TxnOpenException;
-import org.apache.hadoop.hive.metastore.api.Type;
-import org.apache.hadoop.hive.metastore.api.UnknownDBException;
-import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
-import org.apache.hadoop.hive.metastore.api.UnknownTableException;
-import org.apache.hadoop.hive.metastore.api.UnlockRequest;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
@@ -238,43 +145,16 @@ import org.apache.thrift.transport.TTransportFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.jdo.JDOException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedExceptionAction;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Formatter;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.Timer;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.regex.Pattern;
-
-import static org.apache.commons.lang.StringUtils.join;
-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT;
-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.validateName;
+import com.facebook.fb303.FacebookBase;
+import com.facebook.fb303.fb_status;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimaps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * TODO:pc remove application logic to a separate interface.

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 4774899..9eec56a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -18,9 +18,36 @@
 
 package org.apache.hadoop.hive.metastore;
 
-import com.google.common.collect.Lists;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.isIndexTable;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.security.auth.login.LoginException;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.auth.HiveAuthUtils;
@@ -30,7 +57,6 @@ import org.apache.hadoop.hive.common.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.conf.HiveConfUtil;
-import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
@@ -45,42 +71,14 @@ import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TCompactProtocol;
 import org.apache.thrift.protocol.TProtocol;
 import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TSSLTransportFactory;
 import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.security.auth.login.LoginException;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.net.InetAddress;
-import java.net.URI;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.isIndexTable;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
 
 /**
  * Hive Metastore Client.

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 5ea000a..8ba7352 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -19,6 +19,12 @@
 package org.apache.hadoop.hive.metastore;
 
 
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
@@ -90,12 +96,6 @@ import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.thrift.TException;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
 /**
  * Wrapper around hive metastore thrift api
  */

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index d4024d2..1aa3e1a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -56,7 +56,6 @@ import javax.jdo.Transaction;
 import javax.jdo.datastore.DataStoreCache;
 import javax.jdo.identity.IntIdentity;
 
-import com.google.common.collect.Maps;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
@@ -165,6 +164,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 /**
  * This class is the interface between the application logic and the database
@@ -1489,11 +1489,13 @@ public class ObjectStore implements RawStore, Configurable {
         tableType = TableType.MANAGED_TABLE.toString();
       }
     }
-    return new Table(mtbl.getTableName(), mtbl.getDatabase().getName(), mtbl
-        .getOwner(), mtbl.getCreateTime(), mtbl.getLastAccessTime(), mtbl
-        .getRetention(), convertToStorageDescriptor(mtbl.getSd()),
-        convertToFieldSchemas(mtbl.getPartitionKeys()), convertMap(mtbl.getParameters()),
-        mtbl.getViewOriginalText(), mtbl.getViewExpandedText(), tableType);
+    final Table table = new Table(mtbl.getTableName(), mtbl.getDatabase().getName(), mtbl
+            .getOwner(), mtbl.getCreateTime(), mtbl.getLastAccessTime(), mtbl
+            .getRetention(), convertToStorageDescriptor(mtbl.getSd()),
+            convertToFieldSchemas(mtbl.getPartitionKeys()), convertMap(mtbl.getParameters()),
+            mtbl.getViewOriginalText(), mtbl.getViewExpandedText(), tableType);
+    table.setRewriteEnabled(mtbl.isRewriteEnabled());
+    return table;
   }
 
   private MTable convertToMTable(Table tbl) throws InvalidObjectException,
@@ -1530,7 +1532,7 @@ public class ObjectStore implements RawStore, Configurable {
         convertToMStorageDescriptor(tbl.getSd()), tbl.getOwner(), tbl
         .getCreateTime(), tbl.getLastAccessTime(), tbl.getRetention(),
         convertToMFieldSchemas(tbl.getPartitionKeys()), tbl.getParameters(),
-        tbl.getViewOriginalText(), tbl.getViewExpandedText(),
+        tbl.getViewOriginalText(), tbl.getViewExpandedText(), tbl.isRewriteEnabled(),
         tableType);
   }
 
@@ -3297,6 +3299,7 @@ public class ObjectStore implements RawStore, Configurable {
       oldt.setLastAccessTime(newt.getLastAccessTime());
       oldt.setViewOriginalText(newt.getViewOriginalText());
       oldt.setViewExpandedText(newt.getViewExpandedText());
+      oldt.setRewriteEnabled(newt.isRewriteEnabled());
 
       // commit the changes
       success = commitTransaction();

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
index 4546d43..94087b1 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
@@ -1062,6 +1062,7 @@ public class HBaseUtils {
     if (table.getViewExpandedText() != null) {
       builder.setViewExpandedText(table.getViewExpandedText());
     }
+    builder.setIsRewriteEnabled(table.isRewriteEnabled());
     if (table.getTableType() != null) builder.setTableType(table.getTableType());
     if (table.getPrivileges() != null) {
       builder.setPrivileges(buildPrincipalPrivilegeSet(table.getPrivileges()));
@@ -1115,6 +1116,7 @@ public class HBaseUtils {
     table.setParameters(buildParameters(proto.getParameters()));
     if (proto.hasViewOriginalText()) table.setViewOriginalText(proto.getViewOriginalText());
     if (proto.hasViewExpandedText()) table.setViewExpandedText(proto.getViewExpandedText());
+    table.setRewriteEnabled(proto.getIsRewriteEnabled());
     table.setTableType(proto.getTableType());
     if (proto.hasPrivileges()) {
       table.setPrivileges(buildPrincipalPrivilegeSet(proto.getPrivileges()));

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java
----------------------------------------------------------------------
diff --git a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java
index 2a78ce9..6cc7157 100644
--- a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java
+++ b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MTable.java
@@ -34,6 +34,7 @@ public class MTable {
   private Map<String, String> parameters;
   private String viewOriginalText;
   private String viewExpandedText;
+  private boolean rewriteEnabled;
   private String tableType;
 
   public MTable() {}
@@ -54,8 +55,8 @@ public class MTable {
    */
   public MTable(String tableName, MDatabase database, MStorageDescriptor sd, String owner,
       int createTime, int lastAccessTime, int retention, List<MFieldSchema> partitionKeys,
-      Map<String, String> parameters,
-      String viewOriginalText, String viewExpandedText, String tableType) {
+      Map<String, String> parameters, String viewOriginalText, String viewExpandedText,
+      boolean rewriteEnabled, String tableType) {
     this.tableName = tableName;
     this.database = database;
     this.sd = sd;
@@ -67,6 +68,7 @@ public class MTable {
     this.parameters = parameters;
     this.viewOriginalText = viewOriginalText;
     this.viewExpandedText = viewExpandedText;
+    this.rewriteEnabled = rewriteEnabled;
     this.tableType = tableType;
   }
 
@@ -155,6 +157,20 @@ public class MTable {
   }
 
   /**
+   * @return whether the view can be used for rewriting queries
+   */
+  public boolean isRewriteEnabled() {
+    return rewriteEnabled;
+  }
+
+  /**
+   * @param rewriteEnabled whether the view can be used for rewriting queries
+   */
+  public void setRewriteEnabled(boolean rewriteEnabled) {
+    this.rewriteEnabled = rewriteEnabled;
+  }
+
+  /**
    * @return the owner
    */
   public String getOwner() {

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/model/package.jdo
----------------------------------------------------------------------
diff --git a/metastore/src/model/package.jdo b/metastore/src/model/package.jdo
index bfd6ddd..daee72c 100644
--- a/metastore/src/model/package.jdo
+++ b/metastore/src/model/package.jdo
@@ -179,6 +179,9 @@
       <field name="viewExpandedText" default-fetch-group="false">
         <column name="VIEW_EXPANDED_TEXT" jdbc-type="LONGVARCHAR"/>
       </field>
+      <field name="rewriteEnabled">
+        <column name="IS_REWRITE_ENABLED"/>
+      </field>
       <field name="tableType">
         <column name="TBL_TYPE" length="128" jdbc-type="VARCHAR"/>
       </field>

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
----------------------------------------------------------------------
diff --git a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
index 3f9e4c5..6499ac6 100644
--- a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
+++ b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
@@ -255,6 +255,7 @@ message Table {
   optional string table_type = 12;
   optional PrincipalPrivilegeSet privileges = 13;
   optional bool is_temporary = 14;
+  optional bool is_rewrite_enabled = 15;
 }
 
 message Index {

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
index aef1149..1f87eeb 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -149,14 +149,14 @@ public class TestObjectStore {
     StorageDescriptor sd = new StorageDescriptor(null, "location", null, null, false, 0, new SerDeInfo("SerDeName", "serializationLib", null), null, null, null);
     HashMap<String,String> params = new HashMap<String,String>();
     params.put("EXTERNAL", "false");
-    Table tbl1 = new Table(TABLE1, DB1, "owner", 1, 2, 3, sd, null, params, "viewOriginalText", "viewExpandedText", "MANAGED_TABLE");
+    Table tbl1 = new Table(TABLE1, DB1, "owner", 1, 2, 3, sd, null, params, null, null, "MANAGED_TABLE");
     objectStore.createTable(tbl1);
 
     List<String> tables = objectStore.getAllTables(DB1);
     Assert.assertEquals(1, tables.size());
     Assert.assertEquals(TABLE1, tables.get(0));
 
-    Table newTbl1 = new Table("new" + TABLE1, DB1, "owner", 1, 2, 3, sd, null, params, "viewOriginalText", "viewExpandedText", "MANAGED_TABLE");
+    Table newTbl1 = new Table("new" + TABLE1, DB1, "owner", 1, 2, 3, sd, null, params, null, null, "MANAGED_TABLE");
     objectStore.alterTable(DB1, TABLE1, newTbl1);
     tables = objectStore.getTables(DB1, "new*");
     Assert.assertEquals(1, tables.size());
@@ -181,7 +181,7 @@ public class TestObjectStore {
     tableParams.put("EXTERNAL", "false");
     FieldSchema partitionKey1 = new FieldSchema("Country", serdeConstants.STRING_TYPE_NAME, "");
     FieldSchema partitionKey2 = new FieldSchema("State", serdeConstants.STRING_TYPE_NAME, "");
-    Table tbl1 = new Table(TABLE1, DB1, "owner", 1, 2, 3, sd, Arrays.asList(partitionKey1, partitionKey2), tableParams, "viewOriginalText", "viewExpandedText", "MANAGED_TABLE");
+    Table tbl1 = new Table(TABLE1, DB1, "owner", 1, 2, 3, sd, Arrays.asList(partitionKey1, partitionKey2), tableParams, null, null, "MANAGED_TABLE");
     objectStore.createTable(tbl1);
     HashMap<String, String> partitionParams = new HashMap<String, String>();
     partitionParams.put("PARTITION_LEVEL_PRIVILEGE", "true");

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCache.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCache.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCache.java
index 6cd3a46..c6a134c 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCache.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCache.java
@@ -19,8 +19,14 @@
 package org.apache.hadoop.hive.metastore.hbase;
 
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -41,14 +47,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.SortedMap;
-import java.util.TreeMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestHBaseAggregateStatsCache {
   private static final Logger LOG = LoggerFactory.getLogger(TestHBaseAggregateStatsCache.class.getName());

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java
index e0c4094..ecc99c3 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java
@@ -18,13 +18,18 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
-import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
@@ -40,14 +45,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.SortedMap;
-import java.util.TreeMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestHBaseAggregateStatsCacheWithBitVector {
   private static final Logger LOG = LoggerFactory

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java
index f4e55ed..99ce96c 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java
@@ -18,14 +18,19 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.StatObjectConverter;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
-import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
@@ -44,14 +49,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.SortedMap;
-import java.util.TreeMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestHBaseAggregateStatsExtrapolation {
   private static final Logger LOG = LoggerFactory

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java
index 62918be..87b1ac8 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java
@@ -18,14 +18,19 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.StatObjectConverter;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
-import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
@@ -43,14 +48,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.SortedMap;
-import java.util.TreeMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestHBaseAggregateStatsNDVUniformDist {
   private static final Logger LOG = LoggerFactory

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
index a34f8ac..0cf56e5 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
@@ -27,7 +27,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
-import java.util.SortedSet;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.Cell;
@@ -67,7 +66,6 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreCached.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreCached.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreCached.java
index cfe9cd0..4ccb7dd 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreCached.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreCached.java
@@ -18,59 +18,38 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.Decimal;
-import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData;
-import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.mockito.Mock;
-import org.mockito.Mockito;
 import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/QueryState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryState.java b/ql/src/java/org/apache/hadoop/hive/ql/QueryState.java
index 78715d8..6dfaa9f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/QueryState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryState.java
@@ -18,11 +18,9 @@
 
 package org.apache.hadoop.hive.ql;
 
-import java.sql.Timestamp;
 import java.util.Map;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 
 /**

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 8a25399..b12fa9b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -124,6 +124,7 @@ import org.apache.hadoop.hive.ql.metadata.CheckResult;
 import org.apache.hadoop.hive.ql.metadata.ForeignKeyInfo;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveMaterializedViewsRegistry;
 import org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
@@ -136,9 +137,9 @@ import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatter;
 import org.apache.hadoop.hive.ql.parse.AlterTablePartMergeFilesDesc;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.ExplainConfiguration.AnalyzeState;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.parse.ExplainConfiguration.AnalyzeState;
 import org.apache.hadoop.hive.ql.plan.AbortTxnsDesc;
 import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
 import org.apache.hadoop.hive.ql.plan.AlterDatabaseDesc;
@@ -2120,7 +2121,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       needsLocation = doesTableNeedLocation(tbl);
 
       if (tbl.isView()) {
-        String createTab_stmt = "CREATE VIEW `" + tableName + "` AS " + tbl.getViewExpandedText();
+        String createTab_stmt = "CREATE VIEW `" + tableName + "` AS " +
+            tbl.getViewExpandedText();
         outStream.write(createTab_stmt.getBytes(StandardCharsets.UTF_8));
         return 0;
       }
@@ -3958,12 +3960,13 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
     }
 
-    int partitionBatchSize = HiveConf.getIntVar(conf,
-        ConfVars.METASTORE_BATCH_RETRIEVE_OBJECTS_MAX);
-
     // drop the table
     db.dropTable(dropTbl.getTableName(), dropTbl.getIfPurge());
     if (tbl != null) {
+      // Remove from cache if it is a materialized view
+      if (tbl.isMaterializedView()) {
+        HiveMaterializedViewsRegistry.get().dropMaterializedView(tbl);
+      }
       // We have already locked the table in DDLSemanticAnalyzer, don't do it again here
       addIfAbsentByName(new WriteEntity(tbl, WriteEntity.WriteType.DDL_NO_LOCK));
     }
@@ -4347,17 +4350,16 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     } else {
       // create new view
       Table tbl = db.newTable(crtView.getViewName());
+      tbl.setViewOriginalText(crtView.getViewOriginalText());
       if (crtView.isMaterialized()) {
+        tbl.setRewriteEnabled(crtView.isRewriteEnabled());
         tbl.setTableType(TableType.MATERIALIZED_VIEW);
       } else {
+        tbl.setViewExpandedText(crtView.getViewExpandedText());
         tbl.setTableType(TableType.VIRTUAL_VIEW);
       }
       tbl.setSerializationLib(null);
       tbl.clearSerDeInfo();
-      tbl.setViewOriginalText(crtView.getViewOriginalText());
-      if (!crtView.isMaterialized()) {
-        tbl.setViewExpandedText(crtView.getViewExpandedText());
-      }
       tbl.setFields(crtView.getSchema());
       if (crtView.getComment() != null) {
         tbl.setProperty("comment", crtView.getComment());
@@ -4391,6 +4393,10 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
 
       db.createTable(tbl, crtView.getIfNotExists());
+      // Add to cache if it is a materialized view
+      if (tbl.isMaterializedView()) {
+        HiveMaterializedViewsRegistry.get().addMaterializedView(tbl);
+      }
       addIfAbsentByName(new WriteEntity(tbl, WriteEntity.WriteType.DDL_NO_LOCK));
     }
     return 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 750fdef..f62d5f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -32,6 +32,7 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -44,19 +45,16 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.ConcurrentHashMap;
-
-import com.google.common.collect.ImmutableMap;
 
 import javax.jdo.JDODataStoreException;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import org.apache.calcite.plan.RelOptMaterialization;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -81,6 +79,7 @@ import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.PartitionDropOptions;
 import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient;
+import org.apache.hadoop.hive.metastore.SynchronizedMetaStoreClient;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
@@ -104,8 +103,8 @@ import org.apache.hadoop.hive.metastore.api.HiveObjectType;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.InsertEventRequestData;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
@@ -126,13 +125,12 @@ import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.FunctionTask;
 import org.apache.hadoop.hive.ql.exec.FunctionUtils;
+import org.apache.hadoop.hive.ql.exec.InPlaceUpdates;
 import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.exec.InPlaceUpdates;
 import org.apache.hadoop.hive.ql.index.HiveIndexHandler;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
-import org.apache.hadoop.hive.metastore.SynchronizedMetaStoreClient;
 import org.apache.hadoop.hive.ql.optimizer.listbucketingpruner.ListBucketingPrunerUtils;
 import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
 import org.apache.hadoop.hive.ql.plan.DropTableDesc;
@@ -151,6 +149,9 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -307,7 +308,7 @@ public class Hive {
     Hive db = hiveDB.get();
     if (db == null || !db.isCurrentUserOwner() || needsRefresh
         || (c != null && db.metaStoreClient != null && !isCompatible(db, c, isFastCheck))) {
-      return create(c, false, db, doRegisterAllFns);
+      db = create(c, false, db, doRegisterAllFns);
     }
     if (c != null) {
       db.conf = c;
@@ -1331,6 +1332,27 @@ public class Hive {
   }
 
   /**
+   * Get all tables for the specified database.
+   * @param dbName
+   * @return List of table names
+   * @throws HiveException
+   */
+  public List<Table> getAllTableObjects(String dbName) throws HiveException {
+    try {
+      return Lists.transform(getMSC().getTableObjectsByName(dbName, getMSC().getAllTables(dbName)),
+        new com.google.common.base.Function<org.apache.hadoop.hive.metastore.api.Table, Table>() {
+          @Override
+          public Table apply(org.apache.hadoop.hive.metastore.api.Table table) {
+            return new Table(table);
+          }
+        }
+      );
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  /**
    * Returns all existing tables from default database which match the given
    * pattern. The matching occurs as per Java regular expressions
    *
@@ -1407,6 +1429,55 @@ public class Hive {
   }
 
   /**
+   * Get the materialized views that have been enabled for rewriting from the
+   * metastore. If the materialized view is in the cache, we do not need to
+   * parse it to generate a logical plan for the rewriting. Instead, we
+   * return the version present in the cache.
+   *
+   * @return the list of materialized views available for rewriting
+   * @throws HiveException
+   */
+  public List<RelOptMaterialization> getRewritingMaterializedViews() throws HiveException {
+    try {
+      // Final result
+      List<RelOptMaterialization> result = new ArrayList<>();
+      for (String dbName : getMSC().getAllDatabases()) {
+        // From metastore (for security)
+        List<String> tables = getMSC().getAllTables(dbName);
+        // Cached views (includes all)
+        Collection<RelOptMaterialization> cachedViews =
+            HiveMaterializedViewsRegistry.get().getRewritingMaterializedViews(dbName);
+        if (cachedViews.isEmpty()) {
+          // Bail out: empty list
+          continue;
+        }
+        Map<String, RelOptMaterialization> qualifiedNameToView =
+            new HashMap<String, RelOptMaterialization>();
+        for (RelOptMaterialization materialization : cachedViews) {
+          qualifiedNameToView.put(materialization.table.getQualifiedName().get(0), materialization);
+        }
+        for (String table : tables) {
+          // Compose qualified name
+          String fullyQualifiedName = dbName;
+          if (fullyQualifiedName != null && !fullyQualifiedName.isEmpty()) {
+            fullyQualifiedName = fullyQualifiedName + "." + table;
+          } else {
+            fullyQualifiedName = table;
+          }
+          RelOptMaterialization materialization = qualifiedNameToView.get(fullyQualifiedName);
+          if (materialization != null) {
+            // Add to final result set
+            result.add(materialization);
+          }
+        }
+      }
+      return result;
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  /**
    * Get all existing database names.
    *
    * @return List of database names.

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java
new file mode 100644
index 0000000..89c87cd
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java
@@ -0,0 +1,393 @@
+/**
+ * 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.metadata;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.calcite.adapter.druid.DruidQuery;
+import org.apache.calcite.adapter.druid.DruidSchema;
+import org.apache.calcite.adapter.druid.DruidTable;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptMaterialization;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
+import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveVolcanoPlanner;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveRelNode;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.CalcitePlanner;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
+import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
+import org.apache.hadoop.hive.ql.parse.RowResolver;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+
+/** 
+ * Registry for materialized views. The goal of this cache is to avoid parsing and creating
+ * logical plans for the materialized views at query runtime. When a query arrives, we will
+ * just need to consult this cache and extract the logical plans for the views (which had
+ * already been parsed) from it.
+ */
+public final class HiveMaterializedViewsRegistry {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HiveMaterializedViewsRegistry.class);
+
+  /* Singleton */
+  private static final HiveMaterializedViewsRegistry SINGLETON = new HiveMaterializedViewsRegistry();
+
+  /* Key is the database name. Value a map from a unique identifier for the view comprising
+   * the qualified name and the creation time, to the view object.
+   * Since currently we cannot alter a materialized view, that should suffice to identify
+   * whether the cached view is up to date or not.
+   * Creation time is useful to ensure correctness in case multiple HS2 instances are used. */
+  private final ConcurrentMap<String, ConcurrentMap<ViewKey, RelOptMaterialization>> materializedViews =
+      new ConcurrentHashMap<String, ConcurrentMap<ViewKey, RelOptMaterialization>>();
+  private final ExecutorService pool = Executors.newCachedThreadPool();
+
+  private HiveMaterializedViewsRegistry() {
+  }
+
+  /**
+   * Get instance of HiveMaterializedViewsRegistry.
+   *
+   * @return the singleton
+   */
+  public static HiveMaterializedViewsRegistry get() {
+    return SINGLETON;
+  }
+
+  /**
+   * Initialize the registry for the given database. It will extract the materialized views
+   * that are enabled for rewriting from the metastore for the current user, parse them,
+   * and register them in this cache.
+   *
+   * The loading process runs on the background; the method returns in the moment that the
+   * runnable task is created, thus the views will still not be loaded in the cache when
+   * it does.
+   */
+  public void init(final Hive db) {
+    try {
+      List<Table> tables = new ArrayList<Table>();
+      for (String dbName : db.getAllDatabases()) {
+        // TODO: We should enhance metastore API such that it returns only
+        // materialized views instead of all tables
+        tables.addAll(db.getAllTableObjects(dbName));
+      }
+      pool.submit(new Loader(tables));
+    } catch (HiveException e) {
+      LOG.error("Problem connecting to the metastore when initializing the view registry");
+    }
+  }
+
+  private class Loader implements Runnable {
+    private final List<Table> tables;
+
+    private Loader(List<Table> tables) {
+      this.tables = tables;
+    }
+
+    @Override
+    public void run() {
+      for (Table table : tables) {
+        if (table.isMaterializedView()) {
+          addMaterializedView(table);
+        }
+      }
+    }
+  }
+
+  /**
+   * Adds the materialized view to the cache.
+   *
+   * @param materializedViewTable the materialized view
+   */
+  public RelOptMaterialization addMaterializedView(Table materializedViewTable) {
+    // Bail out if it is not enabled for rewriting
+    if (!materializedViewTable.isRewriteEnabled()) {
+      return null;
+    }
+    ConcurrentMap<ViewKey, RelOptMaterialization> cq =
+        new ConcurrentHashMap<ViewKey, RelOptMaterialization>();
+    final ConcurrentMap<ViewKey, RelOptMaterialization> prevCq = materializedViews.putIfAbsent(
+        materializedViewTable.getDbName(), cq);
+    if (prevCq != null) {
+      cq = prevCq;
+    }
+    // Bail out if it already exists
+    final ViewKey vk = new ViewKey(
+        materializedViewTable.getTableName(), materializedViewTable.getCreateTime());
+    if (cq.containsKey(vk)) {
+      return null;
+    }
+    // Add to cache
+    final String viewQuery = materializedViewTable.getViewOriginalText();
+    final RelNode tableRel = createTableScan(materializedViewTable);
+    if (tableRel == null) {
+      LOG.warn("Materialized view " + materializedViewTable.getCompleteName() +
+              " ignored; error creating view replacement");
+      return null;
+    }
+    final RelNode queryRel = parseQuery(viewQuery);
+    if (queryRel == null) {
+      LOG.warn("Materialized view " + materializedViewTable.getCompleteName() +
+              " ignored; error parsing original query");
+      return null;
+    }
+    RelOptMaterialization materialization = new RelOptMaterialization(tableRel, queryRel, null);
+    cq.put(vk, materialization);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Cached materialized view for rewriting: " + tableRel.getTable().getQualifiedName());
+    }
+    return materialization;
+  }
+
+  /**
+   * Removes the materialized view from the cache.
+   *
+   * @param materializedViewTable the materialized view to remove
+   */
+  public void dropMaterializedView(Table materializedViewTable) {
+    // Bail out if it is not enabled for rewriting
+    if (!materializedViewTable.isRewriteEnabled()) {
+      return;
+    }
+    final ViewKey vk = new ViewKey(
+        materializedViewTable.getTableName(), materializedViewTable.getCreateTime());
+    materializedViews.get(materializedViewTable.getDbName()).remove(vk);
+  }
+
+  /**
+   * Returns the materialized views in the cache for the given database.
+   *
+   * @param dbName the database
+   * @return the collection of materialized views, or the empty collection if none
+   */
+  Collection<RelOptMaterialization> getRewritingMaterializedViews(String dbName) {
+    if (materializedViews.get(dbName) != null) {
+      return Collections.unmodifiableCollection(materializedViews.get(dbName).values());
+    }
+    return ImmutableList.of();
+  }
+
+  private static RelNode createTableScan(Table viewTable) {
+    // 0. Recreate cluster
+    final RelOptPlanner planner = HiveVolcanoPlanner.createPlanner(null);
+    final RexBuilder rexBuilder = new RexBuilder(new JavaTypeFactoryImpl());
+    final RelOptCluster cluster = RelOptCluster.create(planner, rexBuilder);
+
+    // 1. Create column schema
+    final RowResolver rr = new RowResolver();
+    // 1.1 Add Column info for non partion cols (Object Inspector fields)
+    StructObjectInspector rowObjectInspector;
+    try {
+      rowObjectInspector = (StructObjectInspector) viewTable.getDeserializer()
+          .getObjectInspector();
+    } catch (SerDeException e) {
+      // Bail out
+      return null;
+    }
+    List<? extends StructField> fields = rowObjectInspector.getAllStructFieldRefs();
+    ColumnInfo colInfo;
+    String colName;
+    ArrayList<ColumnInfo> cInfoLst = new ArrayList<ColumnInfo>();
+    for (int i = 0; i < fields.size(); i++) {
+      colName = fields.get(i).getFieldName();
+      colInfo = new ColumnInfo(
+          fields.get(i).getFieldName(),
+          TypeInfoUtils.getTypeInfoFromObjectInspector(fields.get(i).getFieldObjectInspector()),
+          null, false);
+      rr.put(null, colName, colInfo);
+      cInfoLst.add(colInfo);
+    }
+    ArrayList<ColumnInfo> nonPartitionColumns = new ArrayList<ColumnInfo>(cInfoLst);
+
+    // 1.2 Add column info corresponding to partition columns
+    ArrayList<ColumnInfo> partitionColumns = new ArrayList<ColumnInfo>();
+    for (FieldSchema part_col : viewTable.getPartCols()) {
+      colName = part_col.getName();
+      colInfo = new ColumnInfo(colName,
+          TypeInfoFactory.getPrimitiveTypeInfo(part_col.getType()), null, true);
+      rr.put(null, colName, colInfo);
+      cInfoLst.add(colInfo);
+      partitionColumns.add(colInfo);
+    }
+
+    // 1.3 Build row type from field <type, name>
+    RelDataType rowType;
+    try {
+      rowType = TypeConverter.getType(cluster, rr, null);
+    } catch (CalciteSemanticException e) {
+      // Bail out
+      return null;
+    }
+
+    // 2. Build RelOptAbstractTable
+    String fullyQualifiedTabName = viewTable.getDbName();
+    if (fullyQualifiedTabName != null && !fullyQualifiedTabName.isEmpty()) {
+      fullyQualifiedTabName = fullyQualifiedTabName + "." + viewTable.getTableName();
+    }
+    else {
+      fullyQualifiedTabName = viewTable.getTableName();
+    }
+    RelOptHiveTable optTable = new RelOptHiveTable(null, fullyQualifiedTabName,
+        rowType, viewTable, nonPartitionColumns, partitionColumns, new ArrayList<VirtualColumn>(),
+        SessionState.get().getConf(), new HashMap<String, PrunedPartitionList>(),
+        new AtomicInteger());
+    RelNode tableRel;
+
+    // 3. Build operator
+    if (obtainTableType(viewTable) == TableType.DRUID) {
+      // Build Druid query
+      String address = HiveConf.getVar(SessionState.get().getConf(),
+          HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS);
+      String dataSource = viewTable.getParameters().get(Constants.DRUID_DATA_SOURCE);
+      Set<String> metrics = new HashSet<>();
+      List<RelDataType> druidColTypes = new ArrayList<>();
+      List<String> druidColNames = new ArrayList<>();
+      for (RelDataTypeField field : rowType.getFieldList()) {
+        druidColTypes.add(field.getType());
+        druidColNames.add(field.getName());
+        if (field.getName().equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
+          // timestamp
+          continue;
+        }
+        if (field.getType().getSqlTypeName() == SqlTypeName.VARCHAR) {
+          // dimension
+          continue;
+        }
+        metrics.add(field.getName());
+      }
+      List<Interval> intervals = Arrays.asList(DruidTable.DEFAULT_INTERVAL);
+
+      DruidTable druidTable = new DruidTable(new DruidSchema(address, address, false),
+          dataSource, RelDataTypeImpl.proto(rowType), metrics, DruidTable.DEFAULT_TIMESTAMP_COLUMN, intervals);
+      final TableScan scan = new HiveTableScan(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+          optTable, viewTable.getTableName(), null, false, false);
+      tableRel = DruidQuery.create(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+          optTable, druidTable, ImmutableList.<RelNode>of(scan));
+    } else {
+      // Build Hive Table Scan Rel
+      tableRel = new HiveTableScan(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION), optTable,
+          viewTable.getTableName(), null, false, false);
+    }
+    return tableRel;
+  }
+
+  private static RelNode parseQuery(String viewQuery) {
+    try {
+      final ParseDriver pd = new ParseDriver();
+      final ASTNode node = ParseUtils.findRootNonNullToken(pd.parse(viewQuery));
+      final QueryState qs = new QueryState(SessionState.get().getConf());
+      CalcitePlanner analyzer = new CalcitePlanner(qs);
+      analyzer.initCtx(new Context(SessionState.get().getConf()));
+      analyzer.init(false);
+      return analyzer.genLogicalPlan(node);
+    } catch (Exception e) {
+      // We could not parse the view
+      return null;
+    }
+  }
+
+  private static class ViewKey {
+    private String viewName;
+    private int creationDate;
+
+    private ViewKey(String viewName, int creationTime) {
+      this.viewName = viewName;
+      this.creationDate = creationTime;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if(this == obj) {
+        return true;
+      }
+      if((obj == null) || (obj.getClass() != this.getClass())) {
+        return false;
+      }
+      ViewKey viewKey = (ViewKey) obj;
+      return creationDate == viewKey.creationDate &&
+          (viewName == viewKey.viewName || (viewName != null && viewName.equals(viewKey.viewName)));
+    }
+
+    @Override
+    public int hashCode() {
+      int hash = 7;
+      hash = 31 * hash + creationDate;
+      hash = 31 * hash + viewName.hashCode();
+      return hash;
+    }
+
+    @Override
+    public String toString() {
+      return "ViewKey{" + viewName + "," + creationDate + "}";
+    }
+  }
+
+  private static TableType obtainTableType(Table tabMetaData) {
+    if (tabMetaData.getStorageHandler() != null &&
+            tabMetaData.getStorageHandler().toString().equals(
+                    Constants.DRUID_HIVE_STORAGE_HANDLER_ID)) {
+      return TableType.DRUID;
+    }
+    return TableType.NATIVE;
+  }
+
+  private enum TableType {
+    DRUID,
+    NATIVE
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
index ea90889..c6ae6f2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
@@ -180,7 +180,6 @@ public class Table implements Serializable {
       t.setOwner(SessionState.getUserFromAuthenticator());
       // set create time
       t.setCreateTime((int) (System.currentTimeMillis() / 1000));
-
     }
     return t;
   }
@@ -809,9 +808,6 @@ public class Table implements Serializable {
     return tTable.getViewExpandedText();
   }
 
-  public void clearSerDeInfo() {
-    tTable.getSd().getSerdeInfo().getParameters().clear();
-  }
   /**
    * @param viewExpandedText
    *          the expanded view text to set
@@ -821,6 +817,25 @@ public class Table implements Serializable {
   }
 
   /**
+   * @return whether this view can be used for rewriting queries
+   */
+  public boolean isRewriteEnabled() {
+    return tTable.isRewriteEnabled();
+  }
+
+  /**
+   * @param rewriteEnabled
+   *          whether this view can be used for rewriting queries
+   */
+  public void setRewriteEnabled(boolean rewriteEnabled) {
+    tTable.setRewriteEnabled(rewriteEnabled);
+  }
+
+  public void clearSerDeInfo() {
+    tTable.getSd().getSerdeInfo().getParameters().clear();
+  }
+
+  /**
    * @return whether this table is actually a view
    */
   public boolean isView() {
@@ -863,6 +878,10 @@ public class Table implements Serializable {
     return new Table(tTable.deepCopy());
   }
 
+  public int getCreateTime() {
+    return tTable.getCreateTime();
+  }
+
   public void setCreateTime(int createTime) {
     tTable.setCreateTime(createTime);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
index c850e43..03f74dd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
@@ -365,7 +365,7 @@ public final class MetaDataFormatUtils {
     tableInfo.append(LINE_DELIM).append("# Storage Information").append(LINE_DELIM);
     getStorageDescriptorInfo(tableInfo, table.getTTable().getSd());
 
-    if (table.isView()) {
+    if (table.isView() || table.isMaterializedView()) {
       tableInfo.append(LINE_DELIM).append("# View Information").append(LINE_DELIM);
       getViewInfo(tableInfo, table);
     }
@@ -376,6 +376,7 @@ public final class MetaDataFormatUtils {
   private static void getViewInfo(StringBuilder tableInfo, Table tbl) {
     formatOutput("View Original Text:", tbl.getViewOriginalText(), tableInfo);
     formatOutput("View Expanded Text:", tbl.getViewExpandedText(), tableInfo);
+    formatOutput("View Rewrite Enabled:", tbl.isRewriteEnabled() ? "Yes" : "No", tableInfo);
   }
 
   private static void getStorageDescriptorInfo(StringBuilder tableInfo,

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexExecutorImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexExecutorImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexExecutorImpl.java
index f7958c6..c6ac056 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexExecutorImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexExecutorImpl.java
@@ -22,8 +22,8 @@ import java.util.HashSet;
 import java.util.List;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexExecutorImpl;
 import org.apache.calcite.rex.RexNode;
 import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcFactory;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ExprNodeConverter;
@@ -36,15 +36,15 @@ import org.slf4j.LoggerFactory;
 
 
 
-public class HiveRexExecutorImpl implements RelOptPlanner.Executor {
+public class HiveRexExecutorImpl extends RexExecutorImpl {
 
-  private final RelOptCluster cluster;
+  private static final Logger LOG = LoggerFactory.getLogger(HiveRexExecutorImpl.class);
 
-  protected final Logger LOG;
+  private final RelOptCluster cluster;
 
   public HiveRexExecutorImpl(RelOptCluster cluster) {
+    super(null);
     this.cluster = cluster;
-    LOG = LoggerFactory.getLogger(this.getClass().getName());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
index 4ebbb13..009d9e5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
@@ -481,4 +481,18 @@ public class RelOptHiveTable extends RelOptAbstractTable {
   public Map<Integer, ColumnInfo> getNonPartColInfoMap() {
     return hiveNonPartitionColsMap;
   }
+
+  @Override
+  public boolean equals(Object obj) {
+    return obj instanceof RelOptHiveTable
+        && this.rowType.equals(((RelOptHiveTable) obj).getRowType())
+        && this.getHiveTableMD().equals(((RelOptHiveTable) obj).getHiveTableMD());
+  }
+
+  @Override
+  public int hashCode() {
+    return (this.getHiveTableMD() == null)
+        ? super.hashCode() : this.getHiveTableMD().hashCode();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java
index 6df6026..63bbdac 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java
@@ -23,8 +23,6 @@ import java.util.Set;
 
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
@@ -63,11 +61,6 @@ public class HiveAggregate extends Aggregate implements HiveRelNode {
   public void implement(Implementor implementor) {
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return mq.getNonCumulativeCost(this);
-  }
-
   // getRows will call estimateRowCount
   @Override
   public double estimateRowCount(RelMetadataQuery mq) {

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFilter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFilter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFilter.java
index 0410c91..d5fa856 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFilter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFilter.java
@@ -18,12 +18,9 @@
 package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
 
@@ -43,9 +40,4 @@ public class HiveFilter extends Filter implements HiveRelNode {
   public void implement(Implementor implementor) {
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return mq.getNonCumulativeCost(this);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveJoin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveJoin.java
index ba9483e..dc2fa86 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveJoin.java
@@ -24,7 +24,6 @@ import java.util.Set;
 
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelCollation;
@@ -214,14 +213,6 @@ public class HiveJoin extends Join implements HiveRelNode {
     this.joinCost = joinCost;
   }
 
-  /**
-   * Model cost of join as size of Inputs.
-   */
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return mq.getNonCumulativeCost(this);
-  }
-
   @Override
   public RelWriter explainTerms(RelWriter pw) {
     return super.explainTerms(pw)

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveProject.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveProject.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveProject.java
index 3e0a9a6..447db8e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveProject.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveProject.java
@@ -22,13 +22,10 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
@@ -174,11 +171,6 @@ public class HiveProject extends Project implements HiveRelNode {
   }
 
   @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return mq.getNonCumulativeCost(this);
-  }
-
-  @Override
   public void implement(Implementor implementor) {
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSemiJoin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSemiJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSemiJoin.java
index d899667..65211cc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSemiJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSemiJoin.java
@@ -21,15 +21,12 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.SemiJoin;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.ImmutableIntList;
@@ -107,9 +104,4 @@ public class HiveSemiJoin extends SemiJoin implements HiveRelNode {
   public void implement(Implementor implementor) {
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return mq.getNonCumulativeCost(this);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
index cccbd2f..fed1664 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveTableScan.java
@@ -24,7 +24,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
@@ -124,11 +123,6 @@ public class HiveTableScan extends TableScan implements HiveRelNode {
             newRowtype, this.useQBIdInDigest, this.insideView);
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-    return mq.getNonCumulativeCost(this);
-  }
-
   @Override public RelWriter explainTerms(RelWriter pw) {
     if (this.useQBIdInDigest) {
       // TODO: Only the qualified name should be left here


[6/6] hive git commit: HIVE-14496: Enable Calcite rewriting with materialized views (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
HIVE-14496: Enable Calcite rewriting with materialized views (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3da29fe7
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3da29fe7
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3da29fe7

Branch: refs/heads/master
Commit: 3da29fe7e11b849b1cf5ee8f49a867b2b691ae8d
Parents: bf37d78
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Mon Dec 12 09:22:47 2016 +0000
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri Dec 16 13:58:06 2016 +0000

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   14 +-
 data/conf/hive-site.xml                         |    6 +
 .../hive/hcatalog/common/TestHCatUtil.java      |    5 +-
 .../listener/TestDbNotificationListener.java    |    5 +-
 .../hive/metastore/TestHiveMetaStore.java       |    1 +
 .../metastore/hbase/TestHBaseSchemaTool.java    |    8 +-
 .../org/apache/hive/beeline/TestSchemaTool.java |   11 +-
 metastore/if/hive_metastore.thrift              |    5 +-
 .../upgrade/derby/037-HIVE-14496.derby.sql      |    8 +
 .../upgrade/derby/hive-schema-2.2.0.derby.sql   |    2 +-
 .../derby/upgrade-2.1.0-to-2.2.0.derby.sql      |    1 +
 .../upgrade/mssql/022-HIVE-14496.mssql.sql      |    1 +
 .../upgrade/mssql/hive-schema-2.2.0.mssql.sql   |    3 +-
 .../mssql/upgrade-2.1.0-to-2.2.0.mssql.sql      |    2 +
 .../upgrade/mysql/037-HIVE-14496.mysql.sql      |    8 +
 .../upgrade/mysql/hive-schema-2.2.0.mysql.sql   |    1 +
 .../mysql/upgrade-2.1.0-to-2.2.0.mysql.sql      |    2 +
 .../upgrade/oracle/037-HIVE-14496.oracle.sql    |    9 +
 .../upgrade/oracle/hive-schema-2.2.0.oracle.sql |    3 +-
 .../oracle/upgrade-2.1.0-to-2.2.0.oracle.sql    |    2 +
 .../postgres/036-HIVE-14496.postgres.sql        |    8 +
 .../postgres/hive-schema-2.2.0.postgres.sql     |    3 +-
 .../upgrade-2.1.0-to-2.2.0.postgres.sql         |    2 +
 .../metastore/hbase/HbaseMetastoreProto.java    |  177 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |   22 +
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   12 +-
 .../apache/hadoop/hive/metastore/api/Table.java |  107 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |   23 +
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   15 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    4 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  218 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   64 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   12 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   17 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |    2 +
 .../hadoop/hive/metastore/model/MTable.java     |   20 +-
 metastore/src/model/package.jdo                 |    3 +
 .../metastore/hbase/hbase_metastore_proto.proto |    1 +
 .../hadoop/hive/metastore/TestObjectStore.java  |    6 +-
 .../hbase/TestHBaseAggregateStatsCache.java     |   20 +-
 ...stHBaseAggregateStatsCacheWithBitVector.java |   21 +-
 .../TestHBaseAggregateStatsExtrapolation.java   |   21 +-
 .../TestHBaseAggregateStatsNDVUniformDist.java  |   21 +-
 .../hive/metastore/hbase/TestHBaseStore.java    |    2 -
 .../metastore/hbase/TestHBaseStoreCached.java   |   43 +-
 .../org/apache/hadoop/hive/ql/QueryState.java   |    2 -
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   24 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   89 +-
 .../metadata/HiveMaterializedViewsRegistry.java |  393 +++
 .../apache/hadoop/hive/ql/metadata/Table.java   |   27 +-
 .../formatting/MetaDataFormatUtils.java         |    3 +-
 .../optimizer/calcite/HiveRexExecutorImpl.java  |   10 +-
 .../ql/optimizer/calcite/RelOptHiveTable.java   |   14 +
 .../calcite/reloperators/HiveAggregate.java     |    7 -
 .../calcite/reloperators/HiveFilter.java        |    8 -
 .../calcite/reloperators/HiveJoin.java          |    9 -
 .../calcite/reloperators/HiveProject.java       |    8 -
 .../calcite/reloperators/HiveSemiJoin.java      |    8 -
 .../calcite/reloperators/HiveTableScan.java     |    6 -
 .../HiveMaterializedViewFilterScanRule.java     |   91 +
 .../MaterializedViewSubstitutionVisitor.java    |  292 +++
 .../rules/views/SubstitutionVisitor.java        | 2458 ++++++++++++++++++
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  157 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   19 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   24 +-
 .../hadoop/hive/ql/parse/StorageFormat.java     |    4 +-
 .../hadoop/hive/ql/plan/CreateViewDesc.java     |   16 +-
 .../hadoop/hive/ql/session/SessionState.java    |    6 +-
 .../hadoop/hive/ql/metadata/TestHive.java       |    4 +
 .../materialized_view_create_rewrite.q          |   59 +
 .../materialized_view_create_rewrite_multi_db.q |   40 +
 .../alter_view_as_select_with_partition.q.out   |    1 +
 .../clientpositive/alter_view_as_select.q.out   |    3 +
 .../clientpositive/create_or_replace_view.q.out |    5 +
 .../results/clientpositive/create_view.q.out    |   31 +-
 .../create_view_defaultformats.q.out            |    2 +
 .../create_view_partitioned.q.out               |    5 +-
 .../clientpositive/create_view_translate.q.out  |    3 +
 .../test/results/clientpositive/cteViews.q.out  |    8 +-
 .../clientpositive/escape_comments.q.out        |    1 +
 .../results/clientpositive/explain_ddl.q.out    |    1 +
 .../llap/cbo_rp_unionDistinct_2.q.out           |    6 +-
 .../llap/selectDistinctStar.q.out               |    4 +
 .../clientpositive/llap/subquery_views.q.out    |    6 +-
 .../clientpositive/llap/unionDistinct_2.q.out   |    6 +-
 .../clientpositive/llap/union_top_level.q.out   |    1 +
 .../materialized_view_create_rewrite.q.out      |  322 +++
 ...erialized_view_create_rewrite_multi_db.q.out |  157 ++
 .../materialized_view_describe.q.out            |   20 +
 .../clientpositive/spark/union_top_level.q.out  |    1 +
 .../results/clientpositive/subquery_views.q.out |    6 +-
 .../clientpositive/tez/unionDistinct_2.q.out    |    6 +-
 .../clientpositive/unicode_comments.q.out       |    1 +
 .../results/clientpositive/view_alias.q.out     |    6 +
 .../apache/hive/service/server/HiveServer2.java |   10 +
 95 files changed, 4791 insertions(+), 540 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
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 dcb383d..b4e89b0 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1070,6 +1070,15 @@ public class HiveConf extends Configuration {
     HIVE_COLUMN_ALIGNMENT("hive.order.columnalignment", true, "Flag to control whether we want to try to align" +
         "columns in operators such as Aggregate or Join so that we try to reduce the number of shuffling stages"),
 
+    // materialized views
+    HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING("hive.materializedview.rewriting", false,
+        "Whether to try to rewrite queries using the materialized views enabled for rewriting"),
+    HIVE_MATERIALIZED_VIEW_FILE_FORMAT("hive.materializedview.fileformat", "ORC",
+        new StringSet("none", "TextFile", "SequenceFile", "RCfile", "ORC"),
+        "Default file format for CREATE MATERIALIZED VIEW statement"),
+    HIVE_MATERIALIZED_VIEW_SERDE("hive.materializedview.serde",
+        "org.apache.hadoop.hive.ql.io.orc.OrcSerde", "Default SerDe used for materialized views"),
+
     // hive.mapjoin.bucket.cache.size has been replaced by hive.smbjoin.cache.row,
     // need to remove by hive .13. Also, do not change default (see SMB operator)
     HIVEMAPJOINBUCKETCACHESIZE("hive.mapjoin.bucket.cache.size", 100, ""),
@@ -1149,11 +1158,6 @@ public class HiveConf extends Configuration {
         "Default file format for CREATE TABLE statement applied to managed tables only. External tables will be \n" +
         "created with format specified by hive.default.fileformat. Leaving this null will result in using hive.default.fileformat \n" +
         "for all tables."),
-    HIVEMATERIALIZEDVIEWFILEFORMAT("hive.materializedview.fileformat", "ORC",
-        new StringSet("none", "TextFile", "SequenceFile", "RCfile", "ORC"),
-        "Default file format for CREATE MATERIALIZED VIEW statement"),
-    HIVEMATERIALIZEDVIEWSERDE("hive.materializedview.serde",
-        "org.apache.hadoop.hive.ql.io.orc.OrcSerde", "Default SerDe used for materialized views"),
     HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "SequenceFile", new StringSet("TextFile", "SequenceFile", "RCfile", "Llap"),
         "Default file format for storing result of the query."),
     HIVECHECKFILEFORMAT("hive.fileformat.check", true, "Whether to check file format or not when loading data files"),

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/data/conf/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/hive-site.xml b/data/conf/hive-site.xml
index b09c159..7a69711 100644
--- a/data/conf/hive-site.xml
+++ b/data/conf/hive-site.xml
@@ -309,4 +309,10 @@
 </property>
 
 
+<property>
+  <name>hive.materializedview.rewriting</name>
+  <value>true</value>
+</property>
+
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java
index 102d6d2..61add91 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/common/TestHCatUtil.java
@@ -24,8 +24,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.metastore.TableType;
@@ -40,6 +38,9 @@ import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 public class TestHCatUtil {
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index 690616d..0b691b1 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -188,9 +188,8 @@ public class TestDbNotificationListener {
     StorageDescriptor sd =
         new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0, serde, null, null,
             emptyParameters);
-    Table table =
-        new Table("mytable", "default", "me", startTime, startTime, 0, sd, null, emptyParameters,
-            null, null, null);
+    Table table = new Table("mytable", "default", "me", startTime, startTime, 0, sd, null,
+        emptyParameters, null, null, null);
     msClient.createTable(table);
     // Get the event
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index 21d1b46..af125c3 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -738,6 +738,7 @@ public abstract class TestHiveMetaStore extends TestCase {
     view.setViewOriginalText("SELECT income, name FROM " + tblName);
     view.setViewExpandedText("SELECT `" + tblName + "`.`income`, `" + tblName +
         "`.`name` FROM `" + dbName + "`.`" + tblName + "`");
+    view.setRewriteEnabled(false);
     StorageDescriptor viewSd = new StorageDescriptor();
     view.setSd(viewSd);
     viewSd.setCols(viewCols);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
index b131163..c98911a 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/hbase/TestHBaseSchemaTool.java
@@ -455,7 +455,7 @@ public class TestHBaseSchemaTool extends HBaseIntegrationTests {
         "\"createTime\":0,\"lastAccessTime\":0,\"retention\":0," +
         "\"partitionKeys\":[{\"name\":\"pcol1\",\"type\":\"string\",\"comment\":\"\"}," +
         "{\"name\":\"pcol2\",\"type\":\"string\",\"comment\":\"\"}],\"parameters\":{}," +
-        "\"tableType\":\"\"} sdHash: qQTgZAi5VzgpozzFGmIVTQ stats:" + lsep,
+        "\"tableType\":\"\",\"rewriteEnabled\":0} sdHash: qQTgZAi5VzgpozzFGmIVTQ stats:" + lsep,
         outStr.toString());
 
     outStr = new ByteArrayOutputStream();
@@ -465,7 +465,7 @@ public class TestHBaseSchemaTool extends HBaseIntegrationTests {
         "\"createTime\":0,\"lastAccessTime\":0,\"retention\":0," +
         "\"partitionKeys\":[{\"name\":\"pcol1\",\"type\":\"string\",\"comment\":\"\"}," +
         "{\"name\":\"pcol2\",\"type\":\"string\",\"comment\":\"\"}],\"parameters\":{\"COLUMN_STATS_ACCURATE\":\"{\\\"COLUMN_STATS\\\":{\\\"col1\\\":\\\"true\\\",\\\"col2\\\":\\\"true\\\"}}\"}," +
-            "\"tableType\":\"\"} sdHash: qQTgZAi5VzgpozzFGmIVTQ stats: column " +
+            "\"tableType\":\"\",\"rewriteEnabled\":0} sdHash: qQTgZAi5VzgpozzFGmIVTQ stats: column " +
             "col1: {\"colName\":\"col1\",\"colType\":\"int\"," +
             "\"statsData\":{\"longStats\":{\"lowValue\":-95,\"highValue\":95,\"numNulls\":1," +
             "\"numDVs\":2,\"bitVectors\":\"\"}}} column col2: {\"colName\":\"col2\",\"colType\":\"varchar(32)\"," +
@@ -474,12 +474,12 @@ public class TestHBaseSchemaTool extends HBaseIntegrationTests {
         "{\"tableName\":\"tab1\",\"dbName\":\"db0\",\"owner\":\"me\",\"createTime\":0," +
         "\"lastAccessTime\":0,\"retention\":0,\"partitionKeys\":[{\"name\":\"pcol1\"," +
             "\"type\":\"string\",\"comment\":\"\"},{\"name\":\"pcol2\",\"type\":\"string\"," +
-            "\"comment\":\"\"}],\"parameters\":{},\"tableType\":\"\"} sdHash: " +
+            "\"comment\":\"\"}],\"parameters\":{},\"tableType\":\"\",\"rewriteEnabled\":0} sdHash: " +
             "qQTgZAi5VzgpozzFGmIVTQ stats:" + lsep +
         "{\"tableName\":\"tab2\",\"dbName\":\"db0\",\"owner\":\"me\",\"createTime\":0," +
         "\"lastAccessTime\":0,\"retention\":0,\"partitionKeys\":[{\"name\":\"pcol1\"," +
         "\"type\":\"string\",\"comment\":\"\"},{\"name\":\"pcol2\",\"type\":\"string\"," +
-        "\"comment\":\"\"}],\"parameters\":{},\"tableType\":\"\"} sdHash: " +
+        "\"comment\":\"\"}],\"parameters\":{},\"tableType\":\"\",\"rewriteEnabled\":0} sdHash: " +
         "qQTgZAi5VzgpozzFGmIVTQ stats:" + lsep, outStr.toString());
 
     List<List<String>> partVals = Arrays.asList(Arrays.asList("a", "b"), Arrays.asList("c", "d"));

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
index 17a4bd9..724f990 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
@@ -613,10 +613,9 @@ public class TestSchemaTool extends TestCase {
          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3,null,'org.apache.hadoop.mapred.TextInputFormat','N','N',null,-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL)",
+         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
          "insert into PARTITiONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (3 ,1435255431,2,0 ,'hive',0,3,'myView','VIRTUAL_VIEW','select a.col1,a.col2 from foo','select * from foo')"
-
+         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,3,'myView','VIRTUAL_VIEW','select a.col1,a.col2 from foo','select * from foo','n')"
        };
     File scriptFile = generateTestScript(scripts);
     schemaTool.runBeeLine(scriptFile.getPath());
@@ -632,11 +631,11 @@ public class TestSchemaTool extends TestCase {
         "insert into DBS values(2, 'my db', '/user/hive/warehouse/mydb', 'mydb', 'public', 'role')",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL)",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
         "insert into PARTITiONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (5000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2016_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (3000 ,1435255431,2,0 ,'hive',0,3000,'mytal3000','MANAGED_TABLE',NULL,NULL)",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3000 ,1435255431,2,0 ,'hive',0,3000,'mytal3000','MANAGED_TABLE',NULL,NULL,'n')",
         "insert into PARTITiONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(5000, 1441402388,0, 'd1=1/d2=5000',5000,2)"
     };
     scriptFile = generateTestScript(scripts);
@@ -686,7 +685,7 @@ public class TestSchemaTool extends TestCase {
           "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role')",
           "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
           "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL)",
+          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
           "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)"
         };
      File scriptFile = generateTestScript(scripts);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index baab31b..6f77156 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -302,9 +302,10 @@ struct Table {
   9: map<string, string> parameters,   // to store comments or any other user level parameters
   10: string viewOriginalText,         // original view text, null for non-view
   11: string viewExpandedText,         // expanded view text, null for non-view
-  12: string tableType,                 // table type enum, e.g. EXTERNAL_TABLE
+  12: string tableType,                // table type enum, e.g. EXTERNAL_TABLE
   13: optional PrincipalPrivilegeSet privileges,
-  14: optional bool temporary=false
+  14: optional bool temporary=false,
+  15: optional bool rewriteEnabled     // rewrite enabled or not
 }
 
 struct Partition {

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/derby/037-HIVE-14496.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/037-HIVE-14496.derby.sql b/metastore/scripts/upgrade/derby/037-HIVE-14496.derby.sql
new file mode 100644
index 0000000..0c294ce
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/037-HIVE-14496.derby.sql
@@ -0,0 +1,8 @@
+-- Step 1: Add the column allowing null
+ALTER TABLE "APP"."TBLS" ADD "IS_REWRITE_ENABLED" CHAR(1);
+
+ -- Step 2: Replace the null with default value (false)
+UPDATE "APP"."TBLS" SET "IS_REWRITE_ENABLED" = 'N';
+
+-- Step 3: Alter the column to disallow null values
+ALTER TABLE "APP"."TBLS" ALTER COLUMN "IS_REWRITE_ENABLED" NOT NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql b/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
index ae980e0..fe18089 100644
--- a/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
@@ -60,7 +60,7 @@ CREATE TABLE "APP"."COLUMNS" ("SD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "
 
 CREATE TABLE "APP"."ROLES" ("ROLE_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "ROLE_NAME" VARCHAR(128));
 
-CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(128), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR);
+CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(128), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "IS_REWRITE_ENABLED" CHAR(1) NOT NULL);
 
 CREATE TABLE "APP"."PARTITION_KEYS" ("TBL_ID" BIGINT NOT NULL, "PKEY_COMMENT" VARCHAR(4000), "PKEY_NAME" VARCHAR(128) NOT NULL, "PKEY_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
index 25a5e37..699a619 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
@@ -1,3 +1,4 @@
 -- Upgrade MetaStore schema from 2.1.0 to 2.2.0
+RUN '037-HIVE-14496.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mssql/022-HIVE-14496.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/022-HIVE-14496.mssql.sql b/metastore/scripts/upgrade/mssql/022-HIVE-14496.mssql.sql
new file mode 100644
index 0000000..0c59467
--- /dev/null
+++ b/metastore/scripts/upgrade/mssql/022-HIVE-14496.mssql.sql
@@ -0,0 +1 @@
+ALTER TABLE TBLS ADD IS_REWRITE_ENABLED bit NOT NULL DEFAULT 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
index fdb4004..7ff881c 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
@@ -358,7 +358,8 @@ CREATE TABLE TBLS
     TBL_NAME nvarchar(128) NULL,
     TBL_TYPE nvarchar(128) NULL,
     VIEW_EXPANDED_TEXT text NULL,
-    VIEW_ORIGINAL_TEXT text NULL
+    VIEW_ORIGINAL_TEXT text NULL,
+    IS_REWRITE_ENABLED bit NOT NULL
 );
 
 ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
index df97206..55d8e9b 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
@@ -1,4 +1,6 @@
 SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0' AS MESSAGE;
 
+:r 022-HIVE-14496.mssql.sql
+
 UPDATE VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mysql/037-HIVE-14496.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/037-HIVE-14496.mysql.sql b/metastore/scripts/upgrade/mysql/037-HIVE-14496.mysql.sql
new file mode 100644
index 0000000..6cccefe
--- /dev/null
+++ b/metastore/scripts/upgrade/mysql/037-HIVE-14496.mysql.sql
@@ -0,0 +1,8 @@
+-- Step 1: Add the column allowing null
+ALTER TABLE `TBLS` ADD `IS_REWRITE_ENABLED` bit(1);
+
+ -- Step 2: Replace the null with default value (false)
+UPDATE `TBLS` SET `IS_REWRITE_ENABLED` = false;
+
+-- Step 3: Alter the column to disallow null values
+ALTER TABLE `TBLS` MODIFY COLUMN `IS_REWRITE_ENABLED` bit(1) NOT NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
index 91e221d..2009f1f 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
@@ -587,6 +587,7 @@ CREATE TABLE IF NOT EXISTS `TBLS` (
   `TBL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `VIEW_EXPANDED_TEXT` mediumtext,
   `VIEW_ORIGINAL_TEXT` mediumtext,
+  `IS_REWRITE_ENABLED` bit(1) NOT NULL,
   PRIMARY KEY (`TBL_ID`),
   UNIQUE KEY `UNIQUETABLE` (`TBL_NAME`,`DB_ID`),
   KEY `TBLS_N50` (`SD_ID`),

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
index de38b58..07a002f 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
@@ -1,5 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0' AS ' ';
 
+SOURCE 037-HIVE-14496.mysql.sql;
+
 UPDATE VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0' AS ' ';
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/oracle/037-HIVE-14496.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/037-HIVE-14496.oracle.sql b/metastore/scripts/upgrade/oracle/037-HIVE-14496.oracle.sql
new file mode 100644
index 0000000..2b3bb77
--- /dev/null
+++ b/metastore/scripts/upgrade/oracle/037-HIVE-14496.oracle.sql
@@ -0,0 +1,9 @@
+-- Step 1: Add the column allowing null
+ALTER TABLE TBLS ADD IS_REWRITE_ENABLED NUMBER(1) NULL;
+
+ -- Step 2: Replace the null with default value (false)
+UPDATE TBLS SET IS_REWRITE_ENABLED = 0;
+
+-- Step 3: Alter the column to disallow null values
+ALTER TABLE TBLS MODIFY(IS_REWRITE_ENABLED NOT NULL);
+ALTER TABLE TBLS ADD CONSTRAINT REWRITE_CHECK CHECK (IS_REWRITE_ENABLED IN (1,0));

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
index 39ba7cb..bb5a934 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
@@ -375,7 +375,8 @@ CREATE TABLE TBLS
     TBL_NAME VARCHAR2(128) NULL,
     TBL_TYPE VARCHAR2(128) NULL,
     VIEW_EXPANDED_TEXT CLOB NULL,
-    VIEW_ORIGINAL_TEXT CLOB NULL
+    VIEW_ORIGINAL_TEXT CLOB NULL,
+    IS_REWRITE_ENABLED NUMBER(1) NOT NULL CHECK (IS_REWRITE_ENABLED IN (1,0))
 );
 
 ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
index 66784a4..b5e65b9 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
@@ -1,4 +1,6 @@
 SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0' AS Status from dual;
 
+@037-HIVE-14496.oracle.sql;
+
 UPDATE VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/postgres/036-HIVE-14496.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/036-HIVE-14496.postgres.sql b/metastore/scripts/upgrade/postgres/036-HIVE-14496.postgres.sql
new file mode 100644
index 0000000..1910cc3
--- /dev/null
+++ b/metastore/scripts/upgrade/postgres/036-HIVE-14496.postgres.sql
@@ -0,0 +1,8 @@
+-- Step 1: Add the column allowing null
+ALTER TABLE "TBLS" ADD COLUMN "IS_REWRITE_ENABLED" boolean NULL;
+
+ -- Step 2: Replace the null with default value (false)
+UPDATE "TBLS" SET "IS_REWRITE_ENABLED" = false;
+
+-- Step 3: Alter the column to disallow null values
+ALTER TABLE "TBLS" ALTER COLUMN "IS_REWRITE_ENABLED" SET NOT NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
index 63ac3be..0021df0 100644
--- a/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
@@ -372,7 +372,8 @@ CREATE TABLE "TBLS" (
     "TBL_NAME" character varying(128) DEFAULT NULL::character varying,
     "TBL_TYPE" character varying(128) DEFAULT NULL::character varying,
     "VIEW_EXPANDED_TEXT" text,
-    "VIEW_ORIGINAL_TEXT" text
+    "VIEW_ORIGINAL_TEXT" text,
+    "IS_REWRITE_ENABLED" boolean NOT NULL
 );
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
index 0b4591d..0f7139a 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
@@ -1,5 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0';
 
+\i 036-HIVE-14496.postgres.sql;
+
 UPDATE "VERSION" SET "SCHEMA_VERSION"='2.2.0', "VERSION_COMMENT"='Hive release version 2.2.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0';
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
index b15b0de..03e492e 100644
--- a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
+++ b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
@@ -28422,6 +28422,16 @@ public final class HbaseMetastoreProto {
      * <code>optional bool is_temporary = 14;</code>
      */
     boolean getIsTemporary();
+
+    // optional bool is_rewrite_enabled = 15;
+    /**
+     * <code>optional bool is_rewrite_enabled = 15;</code>
+     */
+    boolean hasIsRewriteEnabled();
+    /**
+     * <code>optional bool is_rewrite_enabled = 15;</code>
+     */
+    boolean getIsRewriteEnabled();
   }
   /**
    * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.Table}
@@ -28571,6 +28581,11 @@ public final class HbaseMetastoreProto {
               isTemporary_ = input.readBool();
               break;
             }
+            case 120: {
+              bitField0_ |= 0x00002000;
+              isRewriteEnabled_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -29023,6 +29038,22 @@ public final class HbaseMetastoreProto {
       return isTemporary_;
     }
 
+    // optional bool is_rewrite_enabled = 15;
+    public static final int IS_REWRITE_ENABLED_FIELD_NUMBER = 15;
+    private boolean isRewriteEnabled_;
+    /**
+     * <code>optional bool is_rewrite_enabled = 15;</code>
+     */
+    public boolean hasIsRewriteEnabled() {
+      return ((bitField0_ & 0x00002000) == 0x00002000);
+    }
+    /**
+     * <code>optional bool is_rewrite_enabled = 15;</code>
+     */
+    public boolean getIsRewriteEnabled() {
+      return isRewriteEnabled_;
+    }
+
     private void initFields() {
       owner_ = "";
       createTime_ = 0L;
@@ -29038,6 +29069,7 @@ public final class HbaseMetastoreProto {
       tableType_ = "";
       privileges_ = org.apache.hadoop.hive.metastore.hbase.HbaseMetastoreProto.PrincipalPrivilegeSet.getDefaultInstance();
       isTemporary_ = false;
+      isRewriteEnabled_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -29121,6 +29153,9 @@ public final class HbaseMetastoreProto {
       if (((bitField0_ & 0x00001000) == 0x00001000)) {
         output.writeBool(14, isTemporary_);
       }
+      if (((bitField0_ & 0x00002000) == 0x00002000)) {
+        output.writeBool(15, isRewriteEnabled_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -29186,6 +29221,10 @@ public final class HbaseMetastoreProto {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(14, isTemporary_);
       }
+      if (((bitField0_ & 0x00002000) == 0x00002000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(15, isRewriteEnabled_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -29350,6 +29389,8 @@ public final class HbaseMetastoreProto {
         bitField0_ = (bitField0_ & ~0x00001000);
         isTemporary_ = false;
         bitField0_ = (bitField0_ & ~0x00002000);
+        isRewriteEnabled_ = false;
+        bitField0_ = (bitField0_ & ~0x00004000);
         return this;
       }
 
@@ -29451,6 +29492,10 @@ public final class HbaseMetastoreProto {
           to_bitField0_ |= 0x00001000;
         }
         result.isTemporary_ = isTemporary_;
+        if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
+          to_bitField0_ |= 0x00002000;
+        }
+        result.isRewriteEnabled_ = isRewriteEnabled_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -29542,6 +29587,9 @@ public final class HbaseMetastoreProto {
         if (other.hasIsTemporary()) {
           setIsTemporary(other.getIsTemporary());
         }
+        if (other.hasIsRewriteEnabled()) {
+          setIsRewriteEnabled(other.getIsRewriteEnabled());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -30762,6 +30810,39 @@ public final class HbaseMetastoreProto {
         return this;
       }
 
+      // optional bool is_rewrite_enabled = 15;
+      private boolean isRewriteEnabled_ ;
+      /**
+       * <code>optional bool is_rewrite_enabled = 15;</code>
+       */
+      public boolean hasIsRewriteEnabled() {
+        return ((bitField0_ & 0x00004000) == 0x00004000);
+      }
+      /**
+       * <code>optional bool is_rewrite_enabled = 15;</code>
+       */
+      public boolean getIsRewriteEnabled() {
+        return isRewriteEnabled_;
+      }
+      /**
+       * <code>optional bool is_rewrite_enabled = 15;</code>
+       */
+      public Builder setIsRewriteEnabled(boolean value) {
+        bitField0_ |= 0x00004000;
+        isRewriteEnabled_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool is_rewrite_enabled = 15;</code>
+       */
+      public Builder clearIsRewriteEnabled() {
+        bitField0_ = (bitField0_ & ~0x00004000);
+        isRewriteEnabled_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.Table)
     }
 
@@ -41582,7 +41663,7 @@ public final class HbaseMetastoreProto {
       "Info.SkewedColValueLocationMap\032.\n\022Skewed" +
       "ColValueList\022\030\n\020skewed_col_value\030\001 \003(\t\0327" +
       "\n\031SkewedColValueLocationMap\022\013\n\003key\030\001 \003(\t" +
-      "\022\r\n\005value\030\002 \002(\t\"\220\004\n\005Table\022\r\n\005owner\030\001 \001(\t" +
+      "\022\r\n\005value\030\002 \002(\t\"\254\004\n\005Table\022\r\n\005owner\030\001 \001(\t" +
       "\022\023\n\013create_time\030\002 \001(\003\022\030\n\020last_access_tim",
       "e\030\003 \001(\003\022\021\n\tretention\030\004 \001(\003\022\020\n\010location\030\005" +
       " \001(\t\022I\n\rsd_parameters\030\006 \001(\01322.org.apache" +
@@ -41595,52 +41676,52 @@ public final class HbaseMetastoreProto {
       "w_expanded_text\030\013 \001(\t\022\022\n\ntable_type\030\014 \001(" +
       "\t\022Q\n\nprivileges\030\r \001(\0132=.org.apache.hadoo",
       "p.hive.metastore.hbase.PrincipalPrivileg" +
-      "eSet\022\024\n\014is_temporary\030\016 \001(\010\"\334\002\n\005Index\022\031\n\021" +
-      "indexHandlerClass\030\001 \001(\t\022\016\n\006dbName\030\002 \002(\t\022" +
-      "\025\n\rorigTableName\030\003 \002(\t\022\020\n\010location\030\004 \001(\t" +
-      "\022I\n\rsd_parameters\030\005 \001(\01322.org.apache.had" +
-      "oop.hive.metastore.hbase.Parameters\022\022\n\nc" +
-      "reateTime\030\006 \001(\005\022\026\n\016lastAccessTime\030\007 \001(\005\022" +
-      "\026\n\016indexTableName\030\010 \001(\t\022\017\n\007sd_hash\030\t \001(\014" +
-      "\022F\n\nparameters\030\n \001(\01322.org.apache.hadoop" +
-      ".hive.metastore.hbase.Parameters\022\027\n\017defe",
-      "rredRebuild\030\013 \001(\010\"\353\004\n\026PartitionKeyCompar" +
-      "ator\022\r\n\005names\030\001 \002(\t\022\r\n\005types\030\002 \002(\t\022S\n\002op" +
-      "\030\003 \003(\0132G.org.apache.hadoop.hive.metastor" +
-      "e.hbase.PartitionKeyComparator.Operator\022" +
-      "S\n\005range\030\004 \003(\0132D.org.apache.hadoop.hive." +
-      "metastore.hbase.PartitionKeyComparator.R" +
-      "ange\032(\n\004Mark\022\r\n\005value\030\001 \002(\t\022\021\n\tinclusive" +
-      "\030\002 \002(\010\032\272\001\n\005Range\022\013\n\003key\030\001 \002(\t\022R\n\005start\030\002" +
-      " \001(\0132C.org.apache.hadoop.hive.metastore." +
-      "hbase.PartitionKeyComparator.Mark\022P\n\003end",
-      "\030\003 \001(\0132C.org.apache.hadoop.hive.metastor" +
-      "e.hbase.PartitionKeyComparator.Mark\032\241\001\n\010" +
-      "Operator\022Z\n\004type\030\001 \002(\0162L.org.apache.hado" +
-      "op.hive.metastore.hbase.PartitionKeyComp" +
-      "arator.Operator.Type\022\013\n\003key\030\002 \002(\t\022\013\n\003val" +
-      "\030\003 \002(\t\"\037\n\004Type\022\010\n\004LIKE\020\000\022\r\n\tNOTEQUALS\020\001\"" +
-      "\373\001\n\nPrimaryKey\022\017\n\007pk_name\030\001 \002(\t\022Q\n\004cols\030" +
-      "\002 \003(\0132C.org.apache.hadoop.hive.metastore" +
-      ".hbase.PrimaryKey.PrimaryKeyColumn\022\031\n\021en" +
-      "able_constraint\030\003 \001(\010\022\033\n\023validate_constr",
-      "aint\030\004 \001(\010\022\027\n\017rely_constraint\030\005 \001(\010\0328\n\020P" +
-      "rimaryKeyColumn\022\023\n\013column_name\030\001 \002(\t\022\017\n\007" +
-      "key_seq\030\002 \002(\021\"\205\004\n\013ForeignKeys\022K\n\003fks\030\001 \003" +
-      "(\0132>.org.apache.hadoop.hive.metastore.hb" +
-      "ase.ForeignKeys.ForeignKey\032\250\003\n\nForeignKe" +
-      "y\022\017\n\007fk_name\030\001 \002(\t\022\032\n\022referenced_db_name" +
-      "\030\002 \002(\t\022\035\n\025referenced_table_name\030\003 \002(\t\022\032\n" +
-      "\022referenced_pk_name\030\004 \001(\t\022\023\n\013update_rule" +
-      "\030\005 \001(\005\022\023\n\013delete_rule\030\006 \001(\005\022]\n\004cols\030\007 \003(" +
-      "\0132O.org.apache.hadoop.hive.metastore.hba",
-      "se.ForeignKeys.ForeignKey.ForeignKeyColu" +
-      "mn\022\031\n\021enable_constraint\030\010 \001(\010\022\033\n\023validat" +
-      "e_constraint\030\t \001(\010\022\027\n\017rely_constraint\030\n " +
-      "\001(\010\032X\n\020ForeignKeyColumn\022\023\n\013column_name\030\001" +
-      " \002(\t\022\036\n\026referenced_column_name\030\002 \002(\t\022\017\n\007" +
-      "key_seq\030\003 \002(\021*#\n\rPrincipalType\022\010\n\004USER\020\000" +
-      "\022\010\n\004ROLE\020\001"
+      "eSet\022\024\n\014is_temporary\030\016 \001(\010\022\032\n\022is_rewrite" +
+      "_enabled\030\017 \001(\010\"\334\002\n\005Index\022\031\n\021indexHandler" +
+      "Class\030\001 \001(\t\022\016\n\006dbName\030\002 \002(\t\022\025\n\rorigTable" +
+      "Name\030\003 \002(\t\022\020\n\010location\030\004 \001(\t\022I\n\rsd_param" +
+      "eters\030\005 \001(\01322.org.apache.hadoop.hive.met" +
+      "astore.hbase.Parameters\022\022\n\ncreateTime\030\006 " +
+      "\001(\005\022\026\n\016lastAccessTime\030\007 \001(\005\022\026\n\016indexTabl" +
+      "eName\030\010 \001(\t\022\017\n\007sd_hash\030\t \001(\014\022F\n\nparamete" +
+      "rs\030\n \001(\01322.org.apache.hadoop.hive.metast",
+      "ore.hbase.Parameters\022\027\n\017deferredRebuild\030" +
+      "\013 \001(\010\"\353\004\n\026PartitionKeyComparator\022\r\n\005name" +
+      "s\030\001 \002(\t\022\r\n\005types\030\002 \002(\t\022S\n\002op\030\003 \003(\0132G.org" +
+      ".apache.hadoop.hive.metastore.hbase.Part" +
+      "itionKeyComparator.Operator\022S\n\005range\030\004 \003" +
+      "(\0132D.org.apache.hadoop.hive.metastore.hb" +
+      "ase.PartitionKeyComparator.Range\032(\n\004Mark" +
+      "\022\r\n\005value\030\001 \002(\t\022\021\n\tinclusive\030\002 \002(\010\032\272\001\n\005R" +
+      "ange\022\013\n\003key\030\001 \002(\t\022R\n\005start\030\002 \001(\0132C.org.a" +
+      "pache.hadoop.hive.metastore.hbase.Partit",
+      "ionKeyComparator.Mark\022P\n\003end\030\003 \001(\0132C.org" +
+      ".apache.hadoop.hive.metastore.hbase.Part" +
+      "itionKeyComparator.Mark\032\241\001\n\010Operator\022Z\n\004" +
+      "type\030\001 \002(\0162L.org.apache.hadoop.hive.meta" +
+      "store.hbase.PartitionKeyComparator.Opera" +
+      "tor.Type\022\013\n\003key\030\002 \002(\t\022\013\n\003val\030\003 \002(\t\"\037\n\004Ty" +
+      "pe\022\010\n\004LIKE\020\000\022\r\n\tNOTEQUALS\020\001\"\373\001\n\nPrimaryK" +
+      "ey\022\017\n\007pk_name\030\001 \002(\t\022Q\n\004cols\030\002 \003(\0132C.org." +
+      "apache.hadoop.hive.metastore.hbase.Prima" +
+      "ryKey.PrimaryKeyColumn\022\031\n\021enable_constra",
+      "int\030\003 \001(\010\022\033\n\023validate_constraint\030\004 \001(\010\022\027" +
+      "\n\017rely_constraint\030\005 \001(\010\0328\n\020PrimaryKeyCol" +
+      "umn\022\023\n\013column_name\030\001 \002(\t\022\017\n\007key_seq\030\002 \002(" +
+      "\021\"\205\004\n\013ForeignKeys\022K\n\003fks\030\001 \003(\0132>.org.apa" +
+      "che.hadoop.hive.metastore.hbase.ForeignK" +
+      "eys.ForeignKey\032\250\003\n\nForeignKey\022\017\n\007fk_name" +
+      "\030\001 \002(\t\022\032\n\022referenced_db_name\030\002 \002(\t\022\035\n\025re" +
+      "ferenced_table_name\030\003 \002(\t\022\032\n\022referenced_" +
+      "pk_name\030\004 \001(\t\022\023\n\013update_rule\030\005 \001(\005\022\023\n\013de" +
+      "lete_rule\030\006 \001(\005\022]\n\004cols\030\007 \003(\0132O.org.apac",
+      "he.hadoop.hive.metastore.hbase.ForeignKe" +
+      "ys.ForeignKey.ForeignKeyColumn\022\031\n\021enable" +
+      "_constraint\030\010 \001(\010\022\033\n\023validate_constraint" +
+      "\030\t \001(\010\022\027\n\017rely_constraint\030\n \001(\010\032X\n\020Forei" +
+      "gnKeyColumn\022\023\n\013column_name\030\001 \002(\t\022\036\n\026refe" +
+      "renced_column_name\030\002 \002(\t\022\017\n\007key_seq\030\003 \002(" +
+      "\021*#\n\rPrincipalType\022\010\n\004USER\020\000\022\010\n\004ROLE\020\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -41856,7 +41937,7 @@ public final class HbaseMetastoreProto {
           internal_static_org_apache_hadoop_hive_metastore_hbase_Table_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_org_apache_hadoop_hive_metastore_hbase_Table_descriptor,
-              new java.lang.String[] { "Owner", "CreateTime", "LastAccessTime", "Retention", "Location", "SdParameters", "SdHash", "PartitionKeys", "Parameters", "ViewOriginalText", "ViewExpandedText", "TableType", "Privileges", "IsTemporary", });
+              new java.lang.String[] { "Owner", "CreateTime", "LastAccessTime", "Retention", "Location", "SdParameters", "SdHash", "PartitionKeys", "Parameters", "ViewOriginalText", "ViewExpandedText", "TableType", "Privileges", "IsTemporary", "IsRewriteEnabled", });
           internal_static_org_apache_hadoop_hive_metastore_hbase_Index_descriptor =
             getDescriptor().getMessageTypes().get(21);
           internal_static_org_apache_hadoop_hive_metastore_hbase_Index_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 1fae3bc..a74e28b 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -4479,6 +4479,11 @@ void Table::__set_temporary(const bool val) {
 __isset.temporary = true;
 }
 
+void Table::__set_rewriteEnabled(const bool val) {
+  this->rewriteEnabled = val;
+__isset.rewriteEnabled = true;
+}
+
 uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -4639,6 +4644,14 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 15:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->rewriteEnabled);
+          this->__isset.rewriteEnabled = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4731,6 +4744,11 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeBool(this->temporary);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.rewriteEnabled) {
+    xfer += oprot->writeFieldBegin("rewriteEnabled", ::apache::thrift::protocol::T_BOOL, 15);
+    xfer += oprot->writeBool(this->rewriteEnabled);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -4752,6 +4770,7 @@ void swap(Table &a, Table &b) {
   swap(a.tableType, b.tableType);
   swap(a.privileges, b.privileges);
   swap(a.temporary, b.temporary);
+  swap(a.rewriteEnabled, b.rewriteEnabled);
   swap(a.__isset, b.__isset);
 }
 
@@ -4770,6 +4789,7 @@ Table::Table(const Table& other221) {
   tableType = other221.tableType;
   privileges = other221.privileges;
   temporary = other221.temporary;
+  rewriteEnabled = other221.rewriteEnabled;
   __isset = other221.__isset;
 }
 Table& Table::operator=(const Table& other222) {
@@ -4787,6 +4807,7 @@ Table& Table::operator=(const Table& other222) {
   tableType = other222.tableType;
   privileges = other222.privileges;
   temporary = other222.temporary;
+  rewriteEnabled = other222.rewriteEnabled;
   __isset = other222.__isset;
   return *this;
 }
@@ -4807,6 +4828,7 @@ void Table::printTo(std::ostream& out) const {
   out << ", " << "tableType=" << to_string(tableType);
   out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "<null>"));
   out << ", " << "temporary="; (__isset.temporary ? (out << to_string(temporary)) : (out << "<null>"));
+  out << ", " << "rewriteEnabled="; (__isset.rewriteEnabled ? (out << to_string(rewriteEnabled)) : (out << "<null>"));
   out << ")";
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 6838133..0104f6e 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -2062,7 +2062,7 @@ inline std::ostream& operator<<(std::ostream& out, const StorageDescriptor& obj)
 }
 
 typedef struct _Table__isset {
-  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true) {}
+  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false) {}
   bool tableName :1;
   bool dbName :1;
   bool owner :1;
@@ -2077,6 +2077,7 @@ typedef struct _Table__isset {
   bool tableType :1;
   bool privileges :1;
   bool temporary :1;
+  bool rewriteEnabled :1;
 } _Table__isset;
 
 class Table {
@@ -2084,7 +2085,7 @@ class Table {
 
   Table(const Table&);
   Table& operator=(const Table&);
-  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false) {
+  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0) {
   }
 
   virtual ~Table() throw();
@@ -2102,6 +2103,7 @@ class Table {
   std::string tableType;
   PrincipalPrivilegeSet privileges;
   bool temporary;
+  bool rewriteEnabled;
 
   _Table__isset __isset;
 
@@ -2133,6 +2135,8 @@ class Table {
 
   void __set_temporary(const bool val);
 
+  void __set_rewriteEnabled(const bool val);
+
   bool operator == (const Table & rhs) const
   {
     if (!(tableName == rhs.tableName))
@@ -2167,6 +2171,10 @@ class Table {
       return false;
     else if (__isset.temporary && !(temporary == rhs.temporary))
       return false;
+    if (__isset.rewriteEnabled != rhs.__isset.rewriteEnabled)
+      return false;
+    else if (__isset.rewriteEnabled && !(rewriteEnabled == rhs.rewriteEnabled))
+      return false;
     return true;
   }
   bool operator != (const Table &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
index 5d683fb..800219f 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
@@ -52,6 +52,7 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   private static final org.apache.thrift.protocol.TField TABLE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableType", org.apache.thrift.protocol.TType.STRING, (short)12);
   private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.STRUCT, (short)13);
   private static final org.apache.thrift.protocol.TField TEMPORARY_FIELD_DESC = new org.apache.thrift.protocol.TField("temporary", org.apache.thrift.protocol.TType.BOOL, (short)14);
+  private static final org.apache.thrift.protocol.TField REWRITE_ENABLED_FIELD_DESC = new org.apache.thrift.protocol.TField("rewriteEnabled", org.apache.thrift.protocol.TType.BOOL, (short)15);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -73,6 +74,7 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   private String tableType; // required
   private PrincipalPrivilegeSet privileges; // optional
   private boolean temporary; // optional
+  private boolean rewriteEnabled; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -89,7 +91,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     VIEW_EXPANDED_TEXT((short)11, "viewExpandedText"),
     TABLE_TYPE((short)12, "tableType"),
     PRIVILEGES((short)13, "privileges"),
-    TEMPORARY((short)14, "temporary");
+    TEMPORARY((short)14, "temporary"),
+    REWRITE_ENABLED((short)15, "rewriteEnabled");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -132,6 +135,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
           return PRIVILEGES;
         case 14: // TEMPORARY
           return TEMPORARY;
+        case 15: // REWRITE_ENABLED
+          return REWRITE_ENABLED;
         default:
           return null;
       }
@@ -176,8 +181,9 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
   private static final int __LASTACCESSTIME_ISSET_ID = 1;
   private static final int __RETENTION_ISSET_ID = 2;
   private static final int __TEMPORARY_ISSET_ID = 3;
+  private static final int __REWRITEENABLED_ISSET_ID = 4;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.TEMPORARY};
+  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.TEMPORARY,_Fields.REWRITE_ENABLED};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -212,6 +218,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PrincipalPrivilegeSet.class)));
     tmpMap.put(_Fields.TEMPORARY, new org.apache.thrift.meta_data.FieldMetaData("temporary", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.REWRITE_ENABLED, new org.apache.thrift.meta_data.FieldMetaData("rewriteEnabled", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Table.class, metaDataMap);
   }
@@ -297,6 +305,7 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       this.privileges = new PrincipalPrivilegeSet(other.privileges);
     }
     this.temporary = other.temporary;
+    this.rewriteEnabled = other.rewriteEnabled;
   }
 
   public Table deepCopy() {
@@ -323,6 +332,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     this.privileges = null;
     this.temporary = false;
 
+    setRewriteEnabledIsSet(false);
+    this.rewriteEnabled = false;
   }
 
   public String getTableName() {
@@ -669,6 +680,28 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TEMPORARY_ISSET_ID, value);
   }
 
+  public boolean isRewriteEnabled() {
+    return this.rewriteEnabled;
+  }
+
+  public void setRewriteEnabled(boolean rewriteEnabled) {
+    this.rewriteEnabled = rewriteEnabled;
+    setRewriteEnabledIsSet(true);
+  }
+
+  public void unsetRewriteEnabled() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REWRITEENABLED_ISSET_ID);
+  }
+
+  /** Returns true if field rewriteEnabled is set (has been assigned a value) and false otherwise */
+  public boolean isSetRewriteEnabled() {
+    return EncodingUtils.testBit(__isset_bitfield, __REWRITEENABLED_ISSET_ID);
+  }
+
+  public void setRewriteEnabledIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REWRITEENABLED_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TABLE_NAME:
@@ -783,6 +816,14 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       }
       break;
 
+    case REWRITE_ENABLED:
+      if (value == null) {
+        unsetRewriteEnabled();
+      } else {
+        setRewriteEnabled((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -830,6 +871,9 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     case TEMPORARY:
       return isTemporary();
 
+    case REWRITE_ENABLED:
+      return isRewriteEnabled();
+
     }
     throw new IllegalStateException();
   }
@@ -869,6 +913,8 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       return isSetPrivileges();
     case TEMPORARY:
       return isSetTemporary();
+    case REWRITE_ENABLED:
+      return isSetRewriteEnabled();
     }
     throw new IllegalStateException();
   }
@@ -1012,6 +1058,15 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         return false;
     }
 
+    boolean this_present_rewriteEnabled = true && this.isSetRewriteEnabled();
+    boolean that_present_rewriteEnabled = true && that.isSetRewriteEnabled();
+    if (this_present_rewriteEnabled || that_present_rewriteEnabled) {
+      if (!(this_present_rewriteEnabled && that_present_rewriteEnabled))
+        return false;
+      if (this.rewriteEnabled != that.rewriteEnabled)
+        return false;
+    }
+
     return true;
   }
 
@@ -1089,6 +1144,11 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
     if (present_temporary)
       list.add(temporary);
 
+    boolean present_rewriteEnabled = true && (isSetRewriteEnabled());
+    list.add(present_rewriteEnabled);
+    if (present_rewriteEnabled)
+      list.add(rewriteEnabled);
+
     return list.hashCode();
   }
 
@@ -1240,6 +1300,16 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetRewriteEnabled()).compareTo(other.isSetRewriteEnabled());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetRewriteEnabled()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rewriteEnabled, other.rewriteEnabled);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1359,6 +1429,12 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       sb.append(this.temporary);
       first = false;
     }
+    if (isSetRewriteEnabled()) {
+      if (!first) sb.append(", ");
+      sb.append("rewriteEnabled:");
+      sb.append(this.rewriteEnabled);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1547,6 +1623,14 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 15: // REWRITE_ENABLED
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.rewriteEnabled = iprot.readBool();
+              struct.setRewriteEnabledIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1641,6 +1725,11 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         oprot.writeBool(struct.temporary);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetRewriteEnabled()) {
+        oprot.writeFieldBegin(REWRITE_ENABLED_FIELD_DESC);
+        oprot.writeBool(struct.rewriteEnabled);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1701,7 +1790,10 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       if (struct.isSetTemporary()) {
         optionals.set(13);
       }
-      oprot.writeBitSet(optionals, 14);
+      if (struct.isSetRewriteEnabled()) {
+        optionals.set(14);
+      }
+      oprot.writeBitSet(optionals, 15);
       if (struct.isSetTableName()) {
         oprot.writeString(struct.tableName);
       }
@@ -1757,12 +1849,15 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
       if (struct.isSetTemporary()) {
         oprot.writeBool(struct.temporary);
       }
+      if (struct.isSetRewriteEnabled()) {
+        oprot.writeBool(struct.rewriteEnabled);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Table struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(14);
+      BitSet incoming = iprot.readBitSet(15);
       if (incoming.get(0)) {
         struct.tableName = iprot.readString();
         struct.setTableNameIsSet(true);
@@ -1842,6 +1937,10 @@ public class Table implements org.apache.thrift.TBase<Table, Table._Fields>, jav
         struct.temporary = iprot.readBool();
         struct.setTemporaryIsSet(true);
       }
+      if (incoming.get(14)) {
+        struct.rewriteEnabled = iprot.readBool();
+        struct.setRewriteEnabledIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index b9af4ef..595c448 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -4563,6 +4563,10 @@ class Table {
    * @var bool
    */
   public $temporary = false;
+  /**
+   * @var bool
+   */
+  public $rewriteEnabled = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -4638,6 +4642,10 @@ class Table {
           'var' => 'temporary',
           'type' => TType::BOOL,
           ),
+        15 => array(
+          'var' => 'rewriteEnabled',
+          'type' => TType::BOOL,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -4683,6 +4691,9 @@ class Table {
       if (isset($vals['temporary'])) {
         $this->temporary = $vals['temporary'];
       }
+      if (isset($vals['rewriteEnabled'])) {
+        $this->rewriteEnabled = $vals['rewriteEnabled'];
+      }
     }
   }
 
@@ -4829,6 +4840,13 @@ class Table {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 15:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->rewriteEnabled);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -4943,6 +4961,11 @@ class Table {
       $xfer += $output->writeBool($this->temporary);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->rewriteEnabled !== null) {
+      $xfer += $output->writeFieldBegin('rewriteEnabled', TType::BOOL, 15);
+      $xfer += $output->writeBool($this->rewriteEnabled);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 21c0390..7927a46 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -3133,6 +3133,7 @@ class Table:
    - tableType
    - privileges
    - temporary
+   - rewriteEnabled
   """
 
   thrift_spec = (
@@ -3151,9 +3152,10 @@ class Table:
     (12, TType.STRING, 'tableType', None, None, ), # 12
     (13, TType.STRUCT, 'privileges', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 13
     (14, TType.BOOL, 'temporary', None, False, ), # 14
+    (15, TType.BOOL, 'rewriteEnabled', None, None, ), # 15
   )
 
-  def __init__(self, tableName=None, dbName=None, owner=None, createTime=None, lastAccessTime=None, retention=None, sd=None, partitionKeys=None, parameters=None, viewOriginalText=None, viewExpandedText=None, tableType=None, privileges=None, temporary=thrift_spec[14][4],):
+  def __init__(self, tableName=None, dbName=None, owner=None, createTime=None, lastAccessTime=None, retention=None, sd=None, partitionKeys=None, parameters=None, viewOriginalText=None, viewExpandedText=None, tableType=None, privileges=None, temporary=thrift_spec[14][4], rewriteEnabled=None,):
     self.tableName = tableName
     self.dbName = dbName
     self.owner = owner
@@ -3168,6 +3170,7 @@ class Table:
     self.tableType = tableType
     self.privileges = privileges
     self.temporary = temporary
+    self.rewriteEnabled = rewriteEnabled
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -3262,6 +3265,11 @@ class Table:
           self.temporary = iprot.readBool()
         else:
           iprot.skip(ftype)
+      elif fid == 15:
+        if ftype == TType.BOOL:
+          self.rewriteEnabled = iprot.readBool()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3335,6 +3343,10 @@ class Table:
       oprot.writeFieldBegin('temporary', TType.BOOL, 14)
       oprot.writeBool(self.temporary)
       oprot.writeFieldEnd()
+    if self.rewriteEnabled is not None:
+      oprot.writeFieldBegin('rewriteEnabled', TType.BOOL, 15)
+      oprot.writeBool(self.rewriteEnabled)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3358,6 +3370,7 @@ class Table:
     value = (value * 31) ^ hash(self.tableType)
     value = (value * 31) ^ hash(self.privileges)
     value = (value * 31) ^ hash(self.temporary)
+    value = (value * 31) ^ hash(self.rewriteEnabled)
     return value
 
   def __repr__(self):

http://git-wip-us.apache.org/repos/asf/hive/blob/3da29fe7/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index c735932..c82edd6 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -734,6 +734,7 @@ class Table
   TABLETYPE = 12
   PRIVILEGES = 13
   TEMPORARY = 14
+  REWRITEENABLED = 15
 
   FIELDS = {
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
@@ -749,7 +750,8 @@ class Table
     VIEWEXPANDEDTEXT => {:type => ::Thrift::Types::STRING, :name => 'viewExpandedText'},
     TABLETYPE => {:type => ::Thrift::Types::STRING, :name => 'tableType'},
     PRIVILEGES => {:type => ::Thrift::Types::STRUCT, :name => 'privileges', :class => ::PrincipalPrivilegeSet, :optional => true},
-    TEMPORARY => {:type => ::Thrift::Types::BOOL, :name => 'temporary', :default => false, :optional => true}
+    TEMPORARY => {:type => ::Thrift::Types::BOOL, :name => 'temporary', :default => false, :optional => true},
+    REWRITEENABLED => {:type => ::Thrift::Types::BOOL, :name => 'rewriteEnabled', :optional => true}
   }
 
   def struct_fields; FIELDS; end