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

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


The following commit(s) were added to refs/heads/master by this push:
     new 73b6b7f2ba HIVE-25941: Long compilation time of complex query due to 
analysis for materialized view rewrite (Krisztian Kasa, reviewed by Zoltan 
Haindrich,  Aman Sinha)
73b6b7f2ba is described below

commit 73b6b7f2ba75ba9d78b326a73288f80dfb5a6c8d
Author: Krisztian Kasa <kasakri...@gmail.com>
AuthorDate: Tue Apr 12 10:20:32 2022 +0200

    HIVE-25941: Long compilation time of complex query due to analysis for 
materialized view rewrite (Krisztian Kasa, reviewed by Zoltan Haindrich,  Aman 
Sinha)
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   4 +-
 .../org/apache/hadoop/hive/ql/metadata/Hive.java   |   8 +-
 .../ql/metadata/HiveMaterializedViewsRegistry.java |   7 +-
 .../ql/metadata/HiveRelOptMaterialization.java     |  17 +-
 .../hive/ql/metadata/MaterializedViewsCache.java   |  75 +++++++--
 ...terializedViewASTSubQueryRewriteRexShuttle.java |  67 ++++++++
 ...eMaterializedViewASTSubQueryRewriteShuttle.java | 181 +++++++++++++++++++++
 .../rules/views/HiveMaterializedViewUtils.java     |   6 +-
 .../org/apache/hadoop/hive/ql/parse/CBOPlan.java   |   8 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java       | 122 ++++++--------
 .../apache/hadoop/hive/ql/parse/ParseUtils.java    |   2 +-
 .../ql/metadata/TestMaterializedViewsCache.java    |  71 ++++----
 .../materialized_view_rewrite_by_text_3.q          |   4 -
 .../materialized_view_rewrite_by_text_6.q          |   2 +
 .../materialized_view_rewrite_by_text_8.q          |  12 +-
 .../llap/materialized_view_rewrite_by_text_3.q.out |  15 --
 .../llap/materialized_view_rewrite_by_text_8.q.out |  50 +++---
 17 files changed, 468 insertions(+), 183 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 73bb924611..c629e8f698 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1914,10 +1914,10 @@ public class HiveConf extends Configuration {
         "Whether to try to rewrite queries using the materialized views 
enabled for rewriting"),
     
HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SQL("hive.materializedview.rewriting.sql",
 true,
         "Whether to try to rewrite queries using the materialized views 
enabled for rewriting by comparing the sql " +
-                "query text with the materialized views query text"),
+                "query syntax tree with the materialized views query syntax 
tree"),
     
HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SUBQUERY_SQL("hive.materializedview.rewriting.sql.subquery",
 true,
         "Whether to try to rewrite sub-queries using the materialized views 
enabled for rewriting by comparing the sql " +
-                "sub-query text with the materialized views query text"),
+                "sub-query syntax tree with the materialized views query 
syntax tree"),
     
HIVE_MATERIALIZED_VIEW_REWRITING_SELECTION_STRATEGY("hive.materializedview.rewriting.strategy",
 "heuristic",
         new StringSet("heuristic", "costbased"),
         "The strategy that should be used to cost and select the materialized 
view rewriting. \n" +
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 fcb7f037b5..48e859b8c8 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
@@ -191,7 +191,6 @@ import 
org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
 import org.apache.hadoop.hive.metastore.api.WriteNotificationLogBatchRequest;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.ReplChangeManager;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -209,6 +208,7 @@ import org.apache.hadoop.hive.ql.lockmgr.LockException;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewUtils;
 import 
org.apache.hadoop.hive.ql.optimizer.listbucketingpruner.ListBucketingPrunerUtils;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
@@ -2258,11 +2258,11 @@ public class Hive {
    * @return List of materialized views has matching query definition with 
querySql
    * @throws HiveException - an exception is thrown during validation or 
unable to pull transaction ids
    */
-  public List<HiveRelOptMaterialization> getMaterializedViewsBySql(
-          String querySql, Set<TableName> tablesUsed, HiveTxnManager txnMgr) 
throws HiveException {
+  public List<HiveRelOptMaterialization> getMaterializedViewsByAST(
+          ASTNode astNode, Set<TableName> tablesUsed, HiveTxnManager txnMgr) 
throws HiveException {
 
     List<HiveRelOptMaterialization> materializedViews =
-            
HiveMaterializedViewsRegistry.get().getRewritingMaterializedViews(querySql);
+            
HiveMaterializedViewsRegistry.get().getRewritingMaterializedViews(astNode);
     if (materializedViews.isEmpty()) {
       return Collections.emptyList();
     }
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
index a4c318acbd..90f58274c9 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java
@@ -63,6 +63,7 @@ import 
org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewUtils;
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.MaterializedViewIncrementalRewritingRelVisitor;
 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.CBOPlan;
 import org.apache.hadoop.hive.ql.parse.CalcitePlanner;
 import org.apache.hadoop.hive.ql.parse.ParseUtils;
@@ -238,7 +239,7 @@ public final class HiveMaterializedViewsRegistry {
             null, viewScan.getTable().getQualifiedName(),
         isBlank(plan.getInvalidAutomaticRewritingMaterializationReason()) ?
             EnumSet.allOf(HiveRelOptMaterialization.RewriteAlgorithm.class) : 
EnumSet.of(TEXT),
-            determineIncrementalRebuildMode(plan.getPlan()));
+            determineIncrementalRebuildMode(plan.getPlan()), plan.getAst());
   }
 
   private HiveRelOptMaterialization.IncrementalRebuildMode 
determineIncrementalRebuildMode(RelNode definitionPlan) {
@@ -368,8 +369,8 @@ public final class HiveMaterializedViewsRegistry {
     return materialization;
   }
 
-  public List<HiveRelOptMaterialization> getRewritingMaterializedViews(String 
querySql) {
-    return materializedViewsCache.get(querySql);
+  public List<HiveRelOptMaterialization> getRewritingMaterializedViews(ASTNode 
ast) {
+    return materializedViewsCache.get(ast);
   }
 
   public boolean isEmpty() {
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveRelOptMaterialization.java 
b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveRelOptMaterialization.java
index 1d312e1da4..b4a20cea0f 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveRelOptMaterialization.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveRelOptMaterialization.java
@@ -24,6 +24,7 @@ import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.RelNode;
 import org.apache.hadoop.hive.metastore.api.Materialization;
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewUtils;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
 
 import java.util.EnumSet;
 import java.util.List;
@@ -68,22 +69,24 @@ public class HiveRelOptMaterialization extends 
RelOptMaterialization {
   private final boolean sourceTablesUpdateDeleteModified;
   private final boolean sourceTablesCompacted;
   private final IncrementalRebuildMode rebuildMode;
+  private final ASTNode ast;
 
   public HiveRelOptMaterialization(RelNode tableRel, RelNode queryRel,
                                    RelOptTable starRelOptTable, List<String> 
qualifiedTableName,
-                                   EnumSet<RewriteAlgorithm> scope, 
IncrementalRebuildMode rebuildMode) {
-    this(tableRel, queryRel, starRelOptTable, qualifiedTableName, scope, 
false, false, rebuildMode);
+                                   EnumSet<RewriteAlgorithm> scope, 
IncrementalRebuildMode rebuildMode, ASTNode ast) {
+    this(tableRel, queryRel, starRelOptTable, qualifiedTableName, scope, 
false, false, rebuildMode, ast);
   }
 
   private HiveRelOptMaterialization(RelNode tableRel, RelNode queryRel,
                                     RelOptTable starRelOptTable, List<String> 
qualifiedTableName,
                                     EnumSet<RewriteAlgorithm> scope,
-                                    boolean sourceTablesUpdateDeleteModified, 
boolean sourceTablesCompacted, IncrementalRebuildMode rebuildMode) {
+                                    boolean sourceTablesUpdateDeleteModified, 
boolean sourceTablesCompacted, IncrementalRebuildMode rebuildMode, ASTNode ast) 
{
     super(tableRel, queryRel, starRelOptTable, qualifiedTableName);
     this.scope = scope;
     this.sourceTablesUpdateDeleteModified = sourceTablesUpdateDeleteModified;
     this.sourceTablesCompacted = sourceTablesCompacted;
     this.rebuildMode = rebuildMode;
+    this.ast = ast;
   }
 
   public EnumSet<RewriteAlgorithm> getScope() {
@@ -111,15 +114,19 @@ public class HiveRelOptMaterialization extends 
RelOptMaterialization {
     return rebuildMode;
   }
 
+  public ASTNode getAst() {
+    return ast;
+  }
+
   public HiveRelOptMaterialization updateInvalidation(Materialization 
materialization) {
     return new HiveRelOptMaterialization(tableRel, queryRel, starRelOptTable, 
qualifiedTableName, scope,
-        materialization.isSourceTablesUpdateDeleteModified(), 
materialization.isSourceTablesCompacted(), rebuildMode);
+        materialization.isSourceTablesUpdateDeleteModified(), 
materialization.isSourceTablesCompacted(), rebuildMode, ast);
   }
 
   public HiveRelOptMaterialization copyToNewCluster(RelOptCluster optCluster) {
     final RelNode newViewScan = 
HiveMaterializedViewUtils.copyNodeNewCluster(optCluster, tableRel);
     return new HiveRelOptMaterialization(newViewScan, queryRel, null, 
qualifiedTableName, scope,
-        sourceTablesUpdateDeleteModified, sourceTablesCompacted, rebuildMode);
+        sourceTablesUpdateDeleteModified, sourceTablesCompacted, rebuildMode, 
ast);
   }
 
 }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializedViewsCache.java 
b/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializedViewsCache.java
index f1b084557e..9ffe2e3d4a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializedViewsCache.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializedViewsCache.java
@@ -19,12 +19,14 @@
 package org.apache.hadoop.hive.ql.metadata;
 
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewUtils;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.function.BiFunction;
@@ -49,7 +51,7 @@ public class MaterializedViewsCache {
   private final ConcurrentMap<String, ConcurrentMap<String, 
HiveRelOptMaterialization>> materializedViews =
           new ConcurrentHashMap<>();
   // Map for looking up materialization by view query text
-  private final Map<String, List<HiveRelOptMaterialization>> 
sqlToMaterializedView = new ConcurrentHashMap<>();
+  private final Map<ASTKey, List<HiveRelOptMaterialization>> 
sqlToMaterializedView = new ConcurrentHashMap<>();
 
 
   public void putIfAbsent(Table materializedViewTable, 
HiveRelOptMaterialization materialization) {
@@ -58,7 +60,7 @@ public class MaterializedViewsCache {
     // You store the materialized view
     dbMap.computeIfAbsent(materializedViewTable.getTableName(), (mvTableName) 
-> {
       List<HiveRelOptMaterialization> materializationList = 
sqlToMaterializedView.computeIfAbsent(
-              materializedViewTable.getViewExpandedText(), s -> new 
ArrayList<>());
+              new ASTKey(materialization.getAst()), s -> new ArrayList<>());
       materializationList.add(materialization);
       return materialization;
     });
@@ -86,7 +88,7 @@ public class MaterializedViewsCache {
 
     dbMap.compute(materializedViewTable.getTableName(), (mvTableName, 
existingMaterialization) -> {
       List<HiveRelOptMaterialization> optMaterializationList = 
sqlToMaterializedView.computeIfAbsent(
-          materializedViewTable.getViewExpandedText(), s -> new ArrayList<>());
+          new ASTKey(newMaterialization.getAst()), s -> new ArrayList<>());
 
       if (existingMaterialization == null) {
         // If it was not existing, we just create it
@@ -146,14 +148,15 @@ public class MaterializedViewsCache {
       return;
     }
 
-    List<HiveRelOptMaterialization> materializationList = 
sqlToMaterializedView.get(mvTable.getViewExpandedText());
+    ASTKey ASTKey = new ASTKey(materialization.getAst());
+    List<HiveRelOptMaterialization> materializationList = 
sqlToMaterializedView.get(ASTKey);
     if (materializationList == null) {
       return;
     }
 
     materializationList.remove(materialization);
     if (materializationList.isEmpty()) {
-      sqlToMaterializedView.remove(mvTable.getViewExpandedText());
+      sqlToMaterializedView.remove(ASTKey);
     }
   }
 
@@ -188,15 +191,19 @@ public class MaterializedViewsCache {
     return null;
   }
 
-  public List<HiveRelOptMaterialization> get(String querySql) {
-    List<HiveRelOptMaterialization> relOptMaterializationList = 
sqlToMaterializedView.get(querySql);
+  public List<HiveRelOptMaterialization> get(ASTNode astNode) {
+    List<HiveRelOptMaterialization> relOptMaterializationList = 
sqlToMaterializedView.get(new ASTKey(astNode));
     if (relOptMaterializationList == null) {
-      LOG.trace("No materialized view with query text '{}' found in 
registry.", querySql);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("No materialized view with query text '{}' found in 
registry.", astNode.dump());
+      }
       LOG.debug("No materialized view with similar query text found in 
registry.");
       return emptyList();
     }
-    LOG.trace("{} materialized view(s) found with query text '{}' in registry",
-            relOptMaterializationList.size(), querySql);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("{} materialized view(s) found with query text '{}' in 
registry",
+              relOptMaterializationList.size(), astNode.dump());
+    }
     LOG.debug("{} materialized view(s) found with similar query text found in 
registry",
             relOptMaterializationList.size());
     return unmodifiableList(relOptMaterializationList);
@@ -205,4 +212,52 @@ public class MaterializedViewsCache {
   public boolean isEmpty() {
     return materializedViews.isEmpty();
   }
+
+
+  private static class ASTKey {
+    private final ASTNode root;
+
+    public ASTKey(ASTNode root) {
+      this.root = root;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      ASTKey that = (ASTKey) o;
+      return equals(root, that.root);
+    }
+
+    private boolean equals(ASTNode astNode1, ASTNode astNode2) {
+      if (!(astNode1.getType() == astNode2.getType() &&
+              astNode1.getText().equals(astNode2.getText()) &&
+              astNode1.getChildCount() == astNode2.getChildCount())) {
+        return false;
+      }
+
+      for (int i = 0; i < astNode1.getChildCount(); ++i) {
+        if (!equals((ASTNode) astNode1.getChild(i), (ASTNode) 
astNode2.getChild(i))) {
+          return false;
+        }
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return hashcode(root);
+    }
+
+    private int hashcode(ASTNode node) {
+      int result = Objects.hash(node.getType(), node.getText());
+
+      for (int i = 0; i < node.getChildCount(); ++i) {
+        result = 31 * result + hashcode((ASTNode) node.getChild(i));
+      }
+
+      return result;
+    }
+  }
 }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteRexShuttle.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteRexShuttle.java
new file mode 100644
index 0000000000..a3cacb123c
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteRexShuttle.java
@@ -0,0 +1,67 @@
+/*
+ * 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;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.sql.fun.SqlQuantifyOperator;
+
+/**
+ * Traverse expressions and tries to rewrite subQuery expressions to 
Materialized view scans.
+ */
+public class HiveMaterializedViewASTSubQueryRewriteRexShuttle extends 
RexShuttle {
+
+  private final HiveMaterializedViewASTSubQueryRewriteShuttle relShuttle;
+
+  public 
HiveMaterializedViewASTSubQueryRewriteRexShuttle(HiveMaterializedViewASTSubQueryRewriteShuttle
 relShuttle) {
+    this.relShuttle = relShuttle;
+  }
+
+  @Override
+  public RexNode visitSubQuery(RexSubQuery subQuery) {
+
+    RelNode newSubquery = subQuery.rel.accept(relShuttle);
+
+    RexNode newSubQueryRex;
+    switch (subQuery.op.kind) {
+      case IN:
+        newSubQueryRex = RexSubQuery.in(newSubquery, subQuery.operands);
+        break;
+
+      case EXISTS:
+        newSubQueryRex = RexSubQuery.exists(newSubquery);
+        break;
+
+      case SCALAR_QUERY:
+        newSubQueryRex = RexSubQuery.scalar(newSubquery);
+        break;
+
+      case SOME:
+      case ALL:
+        newSubQueryRex = RexSubQuery.some(newSubquery, subQuery.operands, 
(SqlQuantifyOperator) subQuery.op);
+        break;
+
+      default:
+        throw new RuntimeException("Unsupported op.kind " + subQuery.op.kind);
+    }
+
+    return newSubQueryRex;
+  }
+}
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteShuttle.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteShuttle.java
new file mode 100644
index 0000000000..151a047c18
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveMaterializedViewASTSubQueryRewriteShuttle.java
@@ -0,0 +1,181 @@
+/*
+ * 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;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import 
org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewUtils;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.CalcitePlanner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayDeque;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Predicate;
+
+import static java.util.Collections.singletonList;
+import static java.util.Collections.unmodifiableMap;
+import static java.util.Collections.unmodifiableSet;
+import static 
org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.NON_CALCITE;
+import static 
org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewUtils.extractTable;
+
+/**
+ * Traverse the plan and tries to rewrite subtrees of the plan to materialized 
view scans.
+ *
+ * The rewrite depends on whether the subtree's corresponding AST match with 
any materialized view
+ * definitions AST.
+ */
+public class HiveMaterializedViewASTSubQueryRewriteShuttle extends 
HiveRelShuttleImpl {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(HiveMaterializedViewASTSubQueryRewriteShuttle.class);
+
+  private final Map<RelNode, ASTNode> subQueryMap;
+  private final ASTNode originalAST;
+  private final ASTNode expandedAST;
+  private final RelBuilder relBuilder;
+  private final Hive db;
+  private final Set<TableName> tablesUsedByOriginalPlan;
+  private final HiveTxnManager txnManager;
+
+  public HiveMaterializedViewASTSubQueryRewriteShuttle(
+          Map<RelNode, ASTNode> subQueryMap,
+          ASTNode originalAST,
+          ASTNode expandedAST,
+          RelBuilder relBuilder,
+          Hive db,
+          Set<TableName> tablesUsedByOriginalPlan,
+          HiveTxnManager txnManager) {
+    this.subQueryMap = unmodifiableMap(subQueryMap);
+    this.originalAST = originalAST;
+    this.expandedAST = expandedAST;
+    this.relBuilder = relBuilder;
+    this.db = db;
+    this.tablesUsedByOriginalPlan = unmodifiableSet(tablesUsedByOriginalPlan);
+    this.txnManager = txnManager;
+  }
+
+  public RelNode rewrite(RelNode relNode) {
+    return relNode.accept(this);
+  }
+
+  @Override
+  public RelNode visit(HiveProject project) {
+    if (!subQueryMap.containsKey(project)) {
+      // No AST found for this subtree
+      return super.visit(project);
+    }
+
+    // The AST associated to the RelNode is part of the original AST, but we 
need the expanded one
+    // 1. Collect the path elements of this node in the original AST
+    ArrayDeque<Integer> path = new ArrayDeque<>();
+    ASTNode curr = subQueryMap.get(project);
+    while (curr != null && curr != originalAST) {
+      path.push(curr.getType());
+      curr = (ASTNode) curr.getParent();
+    }
+
+    // 2. Search for the node in the expanded tree
+    ASTNode expandedSubqAST = new 
CalcitePlanner.ASTSearcher().simpleBreadthFirstSearch(expandedAST, path);
+    if (expandedSubqAST == null) {
+      return super.visit(project);
+    }
+
+    // 3. Lookup MV in the registry by AST subtree coming from the expanded 
tree.
+    // Deal only with MVs which are not supported by the Calcite based rewrite 
algorithm since that algorithm
+    // also makes cost based decisions and can produce better plans.
+    RelNode match = getMaterializedViewByAST(
+            expandedSubqAST, relBuilder.getCluster(), NON_CALCITE, db, 
tablesUsedByOriginalPlan, txnManager);
+    if (match != null) {
+      return match;
+    }
+
+    return super.visit(project);
+  }
+
+    @Override
+  public RelNode visit(HiveFilter filter) {
+
+    RexNode newCond = filter.getCondition().accept(new 
HiveMaterializedViewASTSubQueryRewriteRexShuttle(this));
+    return relBuilder
+            .push(filter.getInput().accept(this))
+            .filter(newCond)
+            .build();
+  }
+
+  /**
+   * 1. Look up MVs in the registry by AST
+   * 2. Check if we have the correct privilege to scan those MVs
+   * 3. Validate if they are up-to-date
+   */
+  public static RelNode getMaterializedViewByAST(
+          ASTNode expandedAST,
+          RelOptCluster optCluster,
+          Predicate<EnumSet<HiveRelOptMaterialization.RewriteAlgorithm>> 
filter,
+          Hive db,
+          Set<TableName> tablesUsedByOriginalPlan,
+          HiveTxnManager txnManager) {
+    try {
+      List<HiveRelOptMaterialization> relOptMaterializationList = 
db.getMaterializedViewsByAST(
+              expandedAST, tablesUsedByOriginalPlan, txnManager);
+      for (HiveRelOptMaterialization relOptMaterialization : 
relOptMaterializationList) {
+        if (!filter.test(relOptMaterialization.getScope())) {
+          LOG.debug("Filter out materialized view {} scope {}",
+                  relOptMaterialization.qualifiedTableName, 
relOptMaterialization.getScope());
+          continue;
+        }
+
+        try {
+          Table hiveTableMD = extractTable(relOptMaterialization);
+          if 
(HiveMaterializedViewUtils.checkPrivilegeForMaterializedViews(singletonList(hiveTableMD)))
 {
+            Set<TableName> sourceTables = new HashSet<>(1);
+            sourceTables.add(hiveTableMD.getFullTableName());
+            if (db.validateMaterializedViewsFromRegistry(
+                    singletonList(hiveTableMD), sourceTables, txnManager)) {
+              return 
relOptMaterialization.copyToNewCluster(optCluster).tableRel;
+            }
+          } else {
+            LOG.debug("User does not have privilege to use materialized view 
{}",
+                    relOptMaterialization.qualifiedTableName);
+          }
+        } catch (HiveException e) {
+          LOG.warn("Skipping materialized view due to validation failure: " +
+                  relOptMaterialization.qualifiedTableName, e);
+        }
+      }
+    } catch (HiveException e) {
+      LOG.warn(String.format("Exception while looking up materialized views 
for query '%s'", expandedAST), e);
+    }
+
+    return null;
+  }
+}
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewUtils.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewUtils.java
index e76a6a9e0c..cde22b3f0c 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewUtils.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/views/HiveMaterializedViewUtils.java
@@ -191,7 +191,8 @@ public class HiveMaterializedViewUtils {
     augmentMaterializationPlanner.setRoot(materialization.queryRel);
     final RelNode modifiedQueryRel = 
augmentMaterializationPlanner.findBestExp();
     return new HiveRelOptMaterialization(materialization.tableRel, 
modifiedQueryRel,
-        null, materialization.qualifiedTableName, materialization.getScope(), 
materialization.getRebuildMode());
+        null, materialization.qualifiedTableName, materialization.getScope(), 
materialization.getRebuildMode(),
+            materialization.getAst());
   }
 
   /**
@@ -313,7 +314,8 @@ public class HiveMaterializedViewUtils {
       materializationList.add(
           new HiveRelOptMaterialization(newTableRel, newQueryRel, null,
               ImmutableList.of(scanTable.getDbName(), scanTable.getTableName(),
-                  "#" + materializationList.size()), 
materialization.getScope(), materialization.getRebuildMode()));
+                  "#" + materializationList.size()), 
materialization.getScope(), materialization.getRebuildMode(),
+                  materialization.getAst()));
     }
     return materializationList;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CBOPlan.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/CBOPlan.java
index 1ee1c4c4a5..4230ef07b4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CBOPlan.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CBOPlan.java
@@ -24,14 +24,20 @@ import org.apache.calcite.rel.RelNode;
  * Wrapper of Calcite plan.
  */
 public class CBOPlan {
+  private final ASTNode ast;
   private final RelNode plan;
   private final String invalidAutomaticRewritingMaterializationReason;
 
-  public CBOPlan(RelNode plan, String 
invalidAutomaticRewritingMaterializationReason) {
+  public CBOPlan(ASTNode ast, RelNode plan, String 
invalidAutomaticRewritingMaterializationReason) {
+    this.ast = ast;
     this.plan = plan;
     this.invalidAutomaticRewritingMaterializationReason = 
invalidAutomaticRewritingMaterializationReason;
   }
 
+  public ASTNode getAst() {
+    return ast;
+  }
+
   /**
    * Root node of plan.
    * @return Root {@link RelNode}
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 a76ce37461..a5633ef9e2 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
@@ -29,7 +29,6 @@ import com.google.common.collect.Multimap;
 
 import java.util.Map.Entry;
 import java.util.Optional;
-import java.util.function.Predicate;
 import java.util.regex.Pattern;
 import org.antlr.runtime.ClassicToken;
 import org.antlr.runtime.CommonToken;
@@ -137,6 +136,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryProperties;
 import org.apache.hadoop.hive.ql.QueryState;
@@ -164,6 +164,7 @@ import 
org.apache.hadoop.hive.ql.optimizer.calcite.CalciteViewSemanticException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveConfPlannerContext;
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.HiveDefaultRelMetadataProvider;
+import 
org.apache.hadoop.hive.ql.optimizer.calcite.HiveMaterializedViewASTSubQueryRewriteShuttle;
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.HiveTezModelRelMetadataProvider;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RuleEventLogger;
 import 
org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveAggregateSortLimitRule;
@@ -347,10 +348,8 @@ import java.util.stream.IntStream;
 
 import javax.sql.DataSource;
 
-import static java.util.Collections.singletonList;
+import static 
org.apache.hadoop.hive.ql.optimizer.calcite.HiveMaterializedViewASTSubQueryRewriteShuttle.getMaterializedViewByAST;
 import static 
org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.ANY;
-import static 
org.apache.hadoop.hive.ql.metadata.HiveRelOptMaterialization.RewriteAlgorithm.NON_CALCITE;
-import static 
org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.HiveMaterializedViewUtils.extractTable;
 
 
 public class CalcitePlanner extends SemanticAnalyzer {
@@ -1250,6 +1249,16 @@ public class CalcitePlanner extends SemanticAnalyzer {
   public static class ASTSearcher {
     private final LinkedList<ASTNode> searchQueue = new LinkedList<ASTNode>();
 
+    public ASTNode simpleBreadthFirstSearch(ASTNode ast, Collection<Integer> 
tokens) {
+      int[] tokenArray = new int[tokens.size()];
+      int i = 0;
+      for (Integer token : tokens) {
+        tokenArray[i] = token;
+        ++i;
+      }
+      return simpleBreadthFirstSearch(ast, tokenArray);
+    }
+
     /**
      * Performs breadth-first search of the AST for a nested set of tokens. 
Tokens
      * don't have to be each others' direct children, they can be separated by
@@ -1619,6 +1628,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
     Map<List<String>, JdbcConvention> jdbcConventionMap = new HashMap<>();
     Map<List<String>, JdbcSchema> schemaMap = new HashMap<>();
 
+    Map<RelNode, ASTNode> subQueryMap = new HashMap<>();
+
     protected CalcitePlannerAction(
             Map<String, PrunedPartitionList> partitionCache,
             StatsSource statsSource,
@@ -1668,10 +1679,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         LOG.debug("Initial CBO Plan:\n" + RelOptUtil.toString(calcitePlan));
       }
 
-      RelNode rewrittenPlan = applyMaterializedViewRewritingByText(ast, 
calcitePlan, optCluster, ANY);
-      if (rewrittenPlan != null) {
-        return rewrittenPlan;
-      }
+      calcitePlan = applyMaterializedViewRewritingByText(ast, calcitePlan, 
optCluster);
 
       // Create executor
       RexExecutor executorProvider = new HiveRexExecutorImpl();
@@ -2098,56 +2106,38 @@ public class CalcitePlanner extends SemanticAnalyzer {
     }
 
     private RelNode applyMaterializedViewRewritingByText(
-            ASTNode queryToRewrite, RelNode calciteGenPlan, RelOptCluster 
optCluster,
-            Predicate<EnumSet<HiveRelOptMaterialization.RewriteAlgorithm>> 
filter) {
+            ASTNode queryToRewriteAST, RelNode originalPlan, RelOptCluster 
optCluster) {
       if (!isMaterializedViewRewritingByTextEnabled()) {
-        return null;
+        return originalPlan;
       }
 
-      unparseTranslator.applyTranslations(ctx.getTokenRewriteStream(), 
EXPANDED_QUERY_TOKEN_REWRITE_PROGRAM);
-      String expandedQueryText = ctx.getTokenRewriteStream().toString(
-              EXPANDED_QUERY_TOKEN_REWRITE_PROGRAM,
-              queryToRewrite.getTokenStartIndex(),
-              queryToRewrite.getTokenStopIndex());
-      return getMaterializedViewByQueryText(expandedQueryText, calciteGenPlan, 
optCluster, filter);
-    }
-
-    private RelNode getMaterializedViewByQueryText(
-            String expandedQueryText, RelNode calciteGenPlan, RelOptCluster 
optCluster,
-            Predicate<EnumSet<HiveRelOptMaterialization.RewriteAlgorithm>> 
filter) {
+      String expandedQueryText = null;
       try {
-        List<HiveRelOptMaterialization> relOptMaterializationList = 
db.getMaterializedViewsBySql(
-                expandedQueryText, getTablesUsed(calciteGenPlan), getTxnMgr());
-        for (HiveRelOptMaterialization relOptMaterialization : 
relOptMaterializationList) {
-          if (!filter.test(relOptMaterialization.getScope())) {
-            LOG.debug("Filter out materialized view {} scope {}",
-                    relOptMaterialization.qualifiedTableName, 
relOptMaterialization.getScope());
-            continue;
-          }
+        unparseTranslator.applyTranslations(ctx.getTokenRewriteStream(), 
EXPANDED_QUERY_TOKEN_REWRITE_PROGRAM);
+        expandedQueryText = ctx.getTokenRewriteStream().toString(
+                EXPANDED_QUERY_TOKEN_REWRITE_PROGRAM,
+                queryToRewriteAST.getTokenStartIndex(),
+                queryToRewriteAST.getTokenStopIndex());
+
+        ASTNode expandedAST = ParseUtils.parse(expandedQueryText, new 
Context(conf));
+        Set<TableName> tablesUsedByOriginalPlan = getTablesUsed(originalPlan);
+        RelNode mvScan = getMaterializedViewByAST(expandedAST, optCluster, 
ANY, db, tablesUsedByOriginalPlan, getTxnMgr());
+        if (mvScan != null) {
+          return mvScan;
+        }
 
-          try {
-            Table hiveTableMD = extractTable(relOptMaterialization);
-            if 
(HiveMaterializedViewUtils.checkPrivilegeForMaterializedViews(singletonList(hiveTableMD)))
 {
-              Set<TableName> sourceTables = new HashSet<>(1);
-              sourceTables.add(hiveTableMD.getFullTableName());
-              if (db.validateMaterializedViewsFromRegistry(
-                      singletonList(hiveTableMD), sourceTables, getTxnMgr())) {
-                return 
relOptMaterialization.copyToNewCluster(optCluster).tableRel;
-              }
-            } else {
-              LOG.debug("User does not have privilege to use materialized view 
{}",
-                      relOptMaterialization.qualifiedTableName);
-            }
-          } catch (HiveException e) {
-            LOG.warn("Skipping materialized view due to validation failure: " +
-                    relOptMaterialization.qualifiedTableName, e);
-          }
+        if 
(!conf.getBoolVar(ConfVars.HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SUBQUERY_SQL))
 {
+          return originalPlan;
         }
-      } catch (HiveException e) {
-        LOG.warn(String.format("Exception while looking up materialized views 
for query '%s'", expandedQueryText), e);
-      }
 
-      return null;
+        return new HiveMaterializedViewASTSubQueryRewriteShuttle(subQueryMap, 
queryToRewriteAST, expandedAST,
+                HiveRelFactories.HIVE_BUILDER.create(optCluster, null),
+                db, tablesUsedByOriginalPlan, 
getTxnMgr()).rewrite(originalPlan);
+      } catch (Exception e) {
+        LOG.warn("Automatic materialized view query rewrite failed. expanded 
query text: {} AST string {} ",
+                expandedQueryText, queryToRewriteAST.toStringTree(), e);
+        return originalPlan;
+      }
     }
 
     /**
@@ -3430,22 +3420,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
             RelNode subQueryRelNode =
                 genLogicalPlan(qbSQ, false, 
relToHiveColNameCalcitePosMap.get(srcRel), relToHiveRR.get(srcRel));
 
-            if 
(conf.getBoolVar(ConfVars.HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SUBQUERY_SQL)
 &&
-                    isMaterializedViewRewritingByTextEnabled()) {
-              unparseTranslator.applyTranslations(ctx.getTokenRewriteStream(), 
EXPANDED_QUERY_TOKEN_REWRITE_PROGRAM);
-              String expandedSubQueryText = 
ctx.getTokenRewriteStream().toString(
-                      EXPANDED_QUERY_TOKEN_REWRITE_PROGRAM,
-                      subQueryRoot.getTokenStartIndex(),
-                      subQueryRoot.getTokenStopIndex());
-
-              if (expandedSubQueryText.length() >= 2) {
-                expandedSubQueryText = expandedSubQueryText.substring(1, 
expandedSubQueryText.length() - 1).trim();
-              }
-
-              RelNode mv = 
getMaterializedViewByQueryText(expandedSubQueryText, subQueryRelNode, cluster, 
NON_CALCITE);
-              if (mv != null) {
-                subQueryRelNode = mv;
-              }
+            if (subQueryRelNode instanceof HiveProject) {
+              subQueryMap.put(subQueryRelNode, subQueryRoot);
             }
 
             subQueryToRelNode.put(next, 
parseInfo.setSubQueryRelNode(subQueryRelNode));
@@ -5041,15 +5017,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
 
         ASTNode subqueryRoot = qbexpr.getSubQueryRoot();
         if (subqueryRoot != null &&
-                
conf.getBoolVar(ConfVars.HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SUBQUERY_SQL))
 {
-          RelNode mv = applyMaterializedViewRewritingByText(subqueryRoot, 
relNode, cluster, NON_CALCITE);
-          if (mv != null) {
-            RowResolver rr = relToHiveRR.remove(relNode);
-            relToHiveRR.put(mv, rr);
-            ImmutableMap<String, Integer> tmp = 
relToHiveColNameCalcitePosMap.remove(relNode);
-            relToHiveColNameCalcitePosMap.put(mv, tmp);
-            relNode = mv;
-          }
+                
conf.getBoolVar(ConfVars.HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SUBQUERY_SQL)
 &&
+                relNode instanceof HiveProject) {
+          subQueryMap.put(relNode, subqueryRoot);
         }
 
         aliasToRel.put(subqAlias, relNode);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
index 5aae2eadef..8b4fa0f461 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
@@ -522,7 +522,7 @@ public final class ParseUtils {
     final ASTNode ast = parse(viewQuery, ctx);
     final CalcitePlanner analyzer = getAnalyzer(conf, ctx);
     RelNode logicalPlan = analyzer.genLogicalPlan(ast);
-    return new CBOPlan(logicalPlan, 
analyzer.getInvalidAutomaticRewritingMaterializationReason());
+    return new CBOPlan(ast, logicalPlan, 
analyzer.getInvalidAutomaticRewritingMaterializationReason());
   }
 
   public static List<FieldSchema> parseQueryAndGetSchema(HiveConf conf, String 
viewQuery)
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMaterializedViewsCache.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMaterializedViewsCache.java
index da532e657b..34e85b4719 100644
--- 
a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMaterializedViewsCache.java
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMaterializedViewsCache.java
@@ -36,7 +36,14 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseException;
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
@@ -74,7 +81,7 @@ class TestMaterializedViewsCache {
   private HiveRelOptMaterialization db1Materialization1;
 
   @BeforeEach
-  void setUp() {
+  void setUp() throws ParseException {
     defaultMV1 = getTable("default", "mat1", "select col0 from t1 where col0 = 
'foo'");
     defaultMaterialization1 = createMaterialization(defaultMV1);
     defaultMV1Same = getTable("default", "mat_same", "select col0 from t1 
where col0 = 'foo'");
@@ -91,7 +98,9 @@ class TestMaterializedViewsCache {
   void testEmptyCache() {
     MaterializedViewsCache emptyCache = new MaterializedViewsCache();
 
-    assertThat(emptyCache.get("select 'any definition'").isEmpty(), is(true));
+    ASTNode any = (ASTNode) ParseDriver.adaptor.create(HiveParser.Identifier, 
"any");
+
+    assertThat(emptyCache.get(any).isEmpty(), is(true));
     assertThat(emptyCache.isEmpty(), is(true));
     assertThat(emptyCache.values().isEmpty(), is(true));
   }
@@ -107,7 +116,9 @@ class TestMaterializedViewsCache {
   void testQueryDoesNotMatchAnyMVDefinition() {
     materializedViewsCache.putIfAbsent(defaultMV1, defaultMaterialization1);
 
-    assertThat(materializedViewsCache.get("select 'not found'").isEmpty(), 
is(true));
+    ASTNode notFound = (ASTNode) 
ParseDriver.adaptor.create(HiveParser.Identifier, "notFound");
+
+    assertThat(materializedViewsCache.get(notFound).isEmpty(), is(true));
     assertThat(materializedViewsCache.values().size(), is(1));
   }
 
@@ -116,8 +127,8 @@ class TestMaterializedViewsCache {
     materializedViewsCache.putIfAbsent(defaultMV1, defaultMaterialization1);
 
     assertThat(materializedViewsCache.get(defaultMV1.getDbName(), 
defaultMV1.getTableName()), is(defaultMaterialization1));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).size(), 
is(1));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).get(0), 
is(defaultMaterialization1));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).size(), 
is(1));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).get(0), 
is(defaultMaterialization1));
     assertThat(materializedViewsCache.isEmpty(), is(false));
     assertThat(materializedViewsCache.values().size(), is(1));
     assertThat(materializedViewsCache.values().get(0), 
is(defaultMaterialization1));
@@ -129,8 +140,8 @@ class TestMaterializedViewsCache {
     materializedViewsCache.putIfAbsent(defaultMV1, defaultMaterialization1);
 
     assertThat(materializedViewsCache.get(defaultMV1.getDbName(), 
defaultMV1.getTableName()), is(defaultMaterialization1));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).size(), 
is(1));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).get(0), 
is(defaultMaterialization1));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).size(), 
is(1));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).get(0), 
is(defaultMaterialization1));
     assertThat(materializedViewsCache.values().size(), is(1));
     assertThat(materializedViewsCache.values().get(0), 
is(defaultMaterialization1));
   }
@@ -143,11 +154,11 @@ class TestMaterializedViewsCache {
     return table;
   }
 
-  private static HiveRelOptMaterialization createMaterialization(Table table) {
+  private static HiveRelOptMaterialization createMaterialization(Table table) 
throws ParseException {
     return new HiveRelOptMaterialization(
             new DummyRel(table), new DummyRel(table), null, 
asList(table.getDbName(), table.getTableName()),
             EnumSet.allOf(HiveRelOptMaterialization.RewriteAlgorithm.class),
-            HiveRelOptMaterialization.IncrementalRebuildMode.AVAILABLE);
+            HiveRelOptMaterialization.IncrementalRebuildMode.AVAILABLE, 
ParseUtils.parse(table.getViewExpandedText(), null));
   }
 
   @Test
@@ -157,9 +168,9 @@ class TestMaterializedViewsCache {
 
     assertThat(materializedViewsCache.get(defaultMV1.getDbName(), 
defaultMV1.getTableName()), is(defaultMaterialization1));
     assertThat(materializedViewsCache.get(defaultMV1Same.getDbName(), 
defaultMV1Same.getTableName()), is(defaultMaterialization1Same));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).size(), 
is(2));
-    assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()), 
hasItem(defaultMaterialization1));
-    assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()), 
hasItem(defaultMaterialization1Same));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).size(), 
is(2));
+    assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()), 
hasItem(defaultMaterialization1));
+    assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()), 
hasItem(defaultMaterialization1Same));
     assertThat(materializedViewsCache.isEmpty(), is(false));
     assertThat(materializedViewsCache.values().size(), is(2));
   }
@@ -171,9 +182,9 @@ class TestMaterializedViewsCache {
 
     assertThat(materializedViewsCache.get(defaultMV1.getDbName(), 
defaultMV1.getTableName()), is(defaultMaterialization1));
     assertThat(materializedViewsCache.get(db1MV1.getDbName(), 
db1MV1.getTableName()), is(db1Materialization1));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).size(), 
is(2));
-    assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()), 
hasItem(defaultMaterialization1));
-    assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()), 
hasItem(db1Materialization1));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).size(), 
is(2));
+    assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()), 
hasItem(defaultMaterialization1));
+    assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()), 
hasItem(db1Materialization1));
     assertThat(materializedViewsCache.isEmpty(), is(false));
     assertThat(materializedViewsCache.values().size(), is(2));
   }
@@ -183,8 +194,8 @@ class TestMaterializedViewsCache {
     materializedViewsCache.putIfAbsent(defaultMV1, defaultMaterialization1);
     materializedViewsCache.putIfAbsent(defaultMVUpCase, 
defaultMaterializationUpCase);
 
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).size(), 
is(1));
-    assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()), 
hasItem(defaultMaterialization1));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).size(), 
is(1));
+    assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()), 
hasItem(defaultMaterialization1));
     assertThat(materializedViewsCache.isEmpty(), is(false));
     assertThat(materializedViewsCache.values().size(), is(2));
   }
@@ -194,23 +205,23 @@ class TestMaterializedViewsCache {
     materializedViewsCache.refresh(defaultMV1, defaultMV1, 
defaultMaterialization1);
 
     assertThat(materializedViewsCache.get(defaultMV1.getDbName(), 
defaultMV1.getTableName()), is(defaultMaterialization1));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).size(), 
is(1));
-    assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()), 
hasItem(defaultMaterialization1));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).size(), 
is(1));
+    assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()), 
hasItem(defaultMaterialization1));
     assertThat(materializedViewsCache.isEmpty(), is(false));
     assertThat(materializedViewsCache.values().size(), is(1));
     assertThat(materializedViewsCache.values(), 
hasItem(defaultMaterialization1));
   }
 
   @Test
-  void testRefreshWhenMVIsCachedButWasUpdated() {
+  void testRefreshWhenMVIsCachedButWasUpdated() throws ParseException {
     materializedViewsCache.putIfAbsent(defaultMV1, defaultMaterialization1);
     HiveRelOptMaterialization newMaterialization = 
createMaterialization(defaultMV1);
     materializedViewsCache.refresh(defaultMV1, defaultMV1, newMaterialization);
 
     assertThat(newMaterialization, is(not(defaultMaterialization1)));
     assertThat(materializedViewsCache.get(defaultMV1.getDbName(), 
defaultMV1.getTableName()), is(newMaterialization));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).size(), 
is(1));
-    assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()), 
hasItem(newMaterialization));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).size(), 
is(1));
+    assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()), 
hasItem(newMaterialization));
     assertThat(materializedViewsCache.isEmpty(), is(false));
     assertThat(materializedViewsCache.values().size(), is(1));
     assertThat(materializedViewsCache.values(), hasItem(newMaterialization));
@@ -222,8 +233,8 @@ class TestMaterializedViewsCache {
     materializedViewsCache.refresh(defaultMV1Same, defaultMV1, 
defaultMaterialization1);
 
     assertThat(materializedViewsCache.get(defaultMV1.getDbName(), 
defaultMV1.getTableName()), is(defaultMaterialization1));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).size(), 
is(1));
-    assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()), 
hasItem(defaultMaterialization1));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).size(), 
is(1));
+    assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()), 
hasItem(defaultMaterialization1));
     assertThat(materializedViewsCache.isEmpty(), is(false));
     assertThat(materializedViewsCache.values().size(), is(1));
     assertThat(materializedViewsCache.values(), 
hasItem(defaultMaterialization1));
@@ -236,7 +247,7 @@ class TestMaterializedViewsCache {
     materializedViewsCache.remove(defaultMV1);
 
     assertThat(materializedViewsCache.get(defaultMV1.getDbName(), 
defaultMV1.getTableName()), is(nullValue()));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).isEmpty(),
 is(true));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).isEmpty(),
 is(true));
     assertThat(materializedViewsCache.isEmpty(), is(true));
     assertThat(materializedViewsCache.values().isEmpty(), is(true));
   }
@@ -248,14 +259,14 @@ class TestMaterializedViewsCache {
     materializedViewsCache.remove(defaultMV1.getDbName(), 
defaultMV1.getTableName());
 
     assertThat(materializedViewsCache.get(defaultMV1.getDbName(), 
defaultMV1.getTableName()), is(nullValue()));
-    
assertThat(materializedViewsCache.get(defaultMV1.getViewExpandedText()).isEmpty(),
 is(true));
+    
assertThat(materializedViewsCache.get(defaultMaterialization1.getAst()).isEmpty(),
 is(true));
     assertThat(materializedViewsCache.isEmpty(), is(true));
     assertThat(materializedViewsCache.values().isEmpty(), is(true));
   }
 
   @Disabled("Testing parallelism only")
   @Test
-  void testParallelism() {
+  void testParallelism() throws ParseException {
     int ITERATIONS = 1000000;
 
     List<Pair<Table, HiveRelOptMaterialization>> testData = new ArrayList<>();
@@ -263,7 +274,7 @@ class TestMaterializedViewsCache {
       Table table = new Table(new 
org.apache.hadoop.hive.metastore.api.Table());
       table.setDbName("default");
       table.setTableName("mat" + i);
-      table.setViewOriginalText("select col0 from t" + i);
+      table.setViewExpandedText("select col0 from t" + i);
       HiveRelOptMaterialization materialization = createMaterialization(table);
       testData.add(new Pair<>(table, materialization));
     }
@@ -271,7 +282,7 @@ class TestMaterializedViewsCache {
       Table table = new Table(new 
org.apache.hadoop.hive.metastore.api.Table());
       table.setDbName("db1");
       table.setTableName("mat" + i);
-      table.setViewOriginalText("select col0 from t" + i);
+      table.setViewExpandedText("select col0 from t" + i);
       HiveRelOptMaterialization materialization = createMaterialization(table);
       testData.add(new Pair<>(table, materialization));
     }
@@ -295,7 +306,7 @@ class TestMaterializedViewsCache {
     for (Pair<Table, HiveRelOptMaterialization> entry : testData) {
       callableList.add(() -> {
         for (int j = 0; j < ITERATIONS; ++j) {
-          materializedViewsCache.get(entry.left.getViewExpandedText());
+          materializedViewsCache.get(entry.right.getAst());
         }
         return null;
       });
diff --git 
a/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_3.q 
b/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_3.q
index cbf5eed50e..a76b030e05 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_3.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_3.q
@@ -15,8 +15,4 @@ SELECT * FROM t1 WHERE col0 = 'FOO';
 explain cbo
 SELECT * FROM t1 WHERE col0 = 'foo';
 
--- query sql text based lookup is case sensitive now -> no rewrite
-explain cbo
-select * from t1 where col0 = 'foo';
-
 drop materialized view mat1;
diff --git 
a/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_6.q 
b/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_6.q
index b61fa2efe6..ccc9030787 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_6.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_6.q
@@ -17,10 +17,12 @@ select col0 from t1 where col0 = 1 union select col0 from 
t1 where col0 = 2;
 create materialized view mat2 as
 select col0 from t1 where col0 = 3;
 
+-- This query is not rewritten because the MV mat2 is applicable for Calcite 
based rewrite algorithm but that is turned off in this test case.
 explain cbo
 select col0 from t2 where exists (
  select col0 from t1 where col0 = 3);
 
+-- These queries should be rewritten because only sql text based rewrite is 
applicable for MV mat1
 explain cbo
 select col0 from t2 where exists (select col0 from t1 where col0 = 1 union 
select col0 from t1 where col0 = 2);
 
diff --git 
a/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_8.q 
b/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_8.q
index a90971ab07..e76a74b4dc 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_8.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_by_text_8.q
@@ -6,25 +6,25 @@ create table t1(col0 int) STORED AS ORC
                           TBLPROPERTIES ('transactional'='true');
 
 create materialized view mat1 as
-select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20;
+select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where l.col0 
= 20;
 
 -- create MV mat2 contains mat1 definiton as subquery, no rewrite
 create materialized view mat2 as
 select col0 from
-  (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+  (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100;
 
 -- rewrite to scan mat2
 explain cbo
 select col0 from
-  (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+  (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100;
 
 -- rewrite subquery sub2 to scan mat2
 explain cbo
 select col0 from (
 select col0 from
-  (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+  (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 ) sub2
 where col0 = 10;
@@ -33,7 +33,7 @@ where col0 = 10;
 explain cbo
 select col0 from (
     select col0 from
-      (select col0 from t1 where col0 between 1 and 10 union select col0 from 
t1 where col0 = 20) sub
+      (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
     where col0 < 100
 ) sub2
 where col0 = 10;
@@ -42,7 +42,7 @@ where col0 = 10;
 explain cbo
 select col0 from t1 where col0 in (
   select col0 from
-    (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+    (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 );
 
diff --git 
a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_3.q.out
 
b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_3.q.out
index 10fd6f81b9..9b09f6bbd2 100644
--- 
a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_3.q.out
+++ 
b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_3.q.out
@@ -51,21 +51,6 @@ POSTHOOK: Input: default@t1
 CBO PLAN:
 HiveTableScan(table=[[default, mat1]], table:alias=[default.mat1])
 
-PREHOOK: query: explain cbo
-select * from t1 where col0 = 'foo'
-PREHOOK: type: QUERY
-PREHOOK: Input: default@t1
-#### A masked pattern was here ####
-POSTHOOK: query: explain cbo
-select * from t1 where col0 = 'foo'
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@t1
-#### A masked pattern was here ####
-CBO PLAN:
-HiveProject($f0=[CAST(_UTF-16LE'foo':VARCHAR(2147483647) CHARACTER SET 
"UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
-  HiveFilter(condition=[=($0, _UTF-16LE'foo')])
-    HiveTableScan(table=[[default, t1]], table:alias=[t1])
-
 PREHOOK: query: drop materialized view mat1
 PREHOOK: type: DROP_MATERIALIZED_VIEW
 PREHOOK: Input: default@mat1
diff --git 
a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_8.q.out
 
b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_8.q.out
index b857890f74..fa31a16a26 100644
--- 
a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_8.q.out
+++ 
b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_by_text_8.q.out
@@ -8,24 +8,26 @@ POSTHOOK: query: create table t1(col0 int) STORED AS ORC
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@t1
-Only query text based automatic rewriting is available for materialized view. 
Statement has unsupported operator: union.
+Only query text based automatic rewriting is available for materialized view. 
LEFT join type is not supported by rewriting algorithm.
+Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
 PREHOOK: query: create materialized view mat1 as
-select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20
+select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where l.col0 
= 20
 PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@t1
 PREHOOK: Output: database:default
 PREHOOK: Output: default@mat1
 POSTHOOK: query: create materialized view mat1 as
-select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20
+select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where l.col0 
= 20
 POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@mat1
-POSTHOOK: Lineage: mat1.col0 EXPRESSION [(t1)t1.FieldSchema(name:col0, 
type:int, comment:null), ]
-Only query text based automatic rewriting is available for materialized view. 
Statement has unsupported operator: union.
+POSTHOOK: Lineage: mat1.col0 SIMPLE []
+Only query text based automatic rewriting is available for materialized view. 
LEFT join type is not supported by rewriting algorithm.
+Warning: Shuffle Join MERGEJOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 
'Reducer 2' is a cross product
 PREHOOK: query: create materialized view mat2 as
 select col0 from
-  (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+  (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 PREHOOK: type: CREATE_MATERIALIZED_VIEW
 PREHOOK: Input: default@t1
@@ -33,16 +35,16 @@ PREHOOK: Output: database:default
 PREHOOK: Output: default@mat2
 POSTHOOK: query: create materialized view mat2 as
 select col0 from
-  (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+  (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 POSTHOOK: type: CREATE_MATERIALIZED_VIEW
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@mat2
-POSTHOOK: Lineage: mat2.col0 EXPRESSION [(t1)t1.FieldSchema(name:col0, 
type:int, comment:null), ]
+POSTHOOK: Lineage: mat2.col0 SIMPLE []
 PREHOOK: query: explain cbo
 select col0 from
-  (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+  (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 PREHOOK: type: QUERY
 PREHOOK: Input: default@mat2
@@ -50,7 +52,7 @@ PREHOOK: Input: default@t1
 #### A masked pattern was here ####
 POSTHOOK: query: explain cbo
 select col0 from
-  (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+  (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@mat2
@@ -62,7 +64,7 @@ HiveTableScan(table=[[default, mat2]], 
table:alias=[default.mat2])
 PREHOOK: query: explain cbo
 select col0 from (
 select col0 from
-  (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+  (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 ) sub2
 where col0 = 10
@@ -73,7 +75,7 @@ PREHOOK: Input: default@t1
 POSTHOOK: query: explain cbo
 select col0 from (
 select col0 from
-  (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+  (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 ) sub2
 where col0 = 10
@@ -89,58 +91,58 @@ HiveProject(col0=[CAST(10):INTEGER])
 PREHOOK: query: explain cbo
 select col0 from (
     select col0 from
-      (select col0 from t1 where col0 between 1 and 10 union select col0 from 
t1 where col0 = 20) sub
+      (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
     where col0 < 100
 ) sub2
 where col0 = 10
 PREHOOK: type: QUERY
-PREHOOK: Input: default@mat1
+PREHOOK: Input: default@mat2
 PREHOOK: Input: default@t1
 #### A masked pattern was here ####
 POSTHOOK: query: explain cbo
 select col0 from (
     select col0 from
-      (select col0 from t1 where col0 between 1 and 10 union select col0 from 
t1 where col0 = 20) sub
+      (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
     where col0 < 100
 ) sub2
 where col0 = 10
 POSTHOOK: type: QUERY
-POSTHOOK: Input: default@mat1
+POSTHOOK: Input: default@mat2
 POSTHOOK: Input: default@t1
 #### A masked pattern was here ####
 CBO PLAN:
 HiveProject(col0=[CAST(10):INTEGER])
   HiveFilter(condition=[=($0, 10)])
-    HiveTableScan(table=[[default, mat1]], table:alias=[default.mat1])
+    HiveTableScan(table=[[default, mat2]], table:alias=[default.mat2])
 
 PREHOOK: query: explain cbo
 select col0 from t1 where col0 in (
   select col0 from
-    (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+    (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 )
 PREHOOK: type: QUERY
-PREHOOK: Input: default@mat1
+PREHOOK: Input: default@mat2
 PREHOOK: Input: default@t1
 #### A masked pattern was here ####
 POSTHOOK: query: explain cbo
 select col0 from t1 where col0 in (
   select col0 from
-    (select col0 from t1 where col0 between 1 and 10 union select col0 from t1 
where col0 = 20) sub
+    (select l.col0 from t1 l left outer join t1 r on (l.col0 = r.col0) where 
l.col0 = 20) sub
 where col0 < 100
 )
 POSTHOOK: type: QUERY
-POSTHOOK: Input: default@mat1
+POSTHOOK: Input: default@mat2
 POSTHOOK: Input: default@t1
 #### A masked pattern was here ####
 CBO PLAN:
 HiveSemiJoin(condition=[=($0, $1)], joinType=[semi])
   HiveProject(col0=[$0])
-    HiveFilter(condition=[<($0, 100)])
+    HiveFilter(condition=[IS NOT NULL($0)])
       HiveTableScan(table=[[default, t1]], table:alias=[t1])
   HiveProject(col0=[$0])
-    HiveFilter(condition=[<($0, 100)])
-      HiveTableScan(table=[[default, mat1]], table:alias=[default.mat1])
+    HiveFilter(condition=[IS NOT NULL($0)])
+      HiveTableScan(table=[[default, mat2]], table:alias=[default.mat2])
 
 PREHOOK: query: drop materialized view mat2
 PREHOOK: type: DROP_MATERIALIZED_VIEW

Reply via email to