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 2020/02/03 18:32:09 UTC

[hive] 02/02: HIVE-22795: Create new parser and udf module from ql (Jesus Camacho Rodriguez, reviewed by Vineet Garg, Zoltan Haindrich)

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

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

commit f83bfbe4ba88498b65620b2b777a31121639d4ba
Author: Jesus Camacho Rodriguez <jc...@apache.org>
AuthorDate: Mon Feb 3 10:31:11 2020 -0800

    HIVE-22795: Create new parser and udf module from ql (Jesus Camacho Rodriguez, reviewed by Vineet Garg, Zoltan Haindrich)
    
    Close apache/hive#891
---
 accumulo-handler/pom.xml                           |   5 +
 .../predicate/AccumuloPredicateHandler.java        |  14 +-
 .../accumulo/predicate/AccumuloRangeGenerator.java |   5 +-
 .../predicate/TestAccumuloRangeGenerator.java      |  71 +++--
 .../org/apache/hadoop/hive/ql/lib/Dispatcher.java  |  38 +--
 .../org/apache/hadoop/hive/ql/lib/GraphWalker.java |  37 ++-
 .../apache/hadoop/hive/ql/lib/NodeProcessor.java   |  37 ++-
 .../java/org/apache/hadoop/hive/ql/lib/Rule.java   |  41 ++-
 itests/pom.xml                                     |  11 +
 itests/qtest-accumulo/pom.xml                      |   5 +
 itests/qtest-kudu/pom.xml                          |   5 +
 itests/qtest-spark/pom.xml                         |   5 +
 kudu-handler/pom.xml                               |   6 +
 parser/pom.xml                                     | 107 +++++++
 .../apache/hadoop/hive/ql/parse/ASTErrorUtils.java | 106 +++++++
 pom.xml                                            |   2 +
 ql/pom.xml                                         |  40 +--
 .../function/macro/create/CreateMacroAnalyzer.java |   4 +-
 .../ql/ddl/table/AbstractAlterTableAnalyzer.java   |   3 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java      | 268 ------------------
 .../apache/hadoop/hive/ql/exec/tez/DagUtils.java   |  18 +-
 .../ql/hooks/AccurateEstimatesCheckerHook.java     |  12 +-
 .../hive/ql/hooks/NoOperatorReuseCheckerHook.java  |  13 +-
 .../hive/ql/index/IndexPredicateAnalyzer.java      |  18 +-
 .../hadoop/hive/ql/lib/CompositeProcessor.java     |   8 +-
 .../hadoop/hive/ql/lib/CostLessRuleDispatcher.java |  12 +-
 .../hadoop/hive/ql/lib/DefaultGraphWalker.java     |   6 +-
 .../hadoop/hive/ql/lib/DefaultRuleDispatcher.java  |  16 +-
 .../hadoop/hive/ql/lib/ExpressionWalker.java       |   2 +-
 .../apache/hadoop/hive/ql/lib/ForwardWalker.java   |   2 +-
 .../hadoop/hive/ql/lib/LevelOrderWalker.java       |   4 +-
 .../hadoop/hive/ql/lib/PreOrderOnceWalker.java     |   2 +-
 .../apache/hadoop/hive/ql/lib/PreOrderWalker.java  |   2 +-
 .../apache/hadoop/hive/ql/lib/RuleExactMatch.java  |   2 +-
 .../org/apache/hadoop/hive/ql/lib/RuleRegExp.java  |   2 +-
 .../hive/ql/lib/SubqueryExpressionWalker.java      |   4 +-
 .../apache/hadoop/hive/ql/lib/TaskGraphWalker.java |   7 +-
 .../org/apache/hadoop/hive/ql/lib/TypeRule.java    |   2 +-
 .../hive/ql/optimizer/AbstractBucketJoinProc.java  |   4 +-
 .../hive/ql/optimizer/AbstractSMBJoinProc.java     |   4 +-
 .../hive/ql/optimizer/BucketMapJoinOptimizer.java  |  22 +-
 .../hive/ql/optimizer/BucketMapjoinProc.java       |   5 +-
 .../BucketingSortingReduceSinkOptimizer.java       |  24 +-
 .../hadoop/hive/ql/optimizer/ColumnPruner.java     |  17 +-
 .../hive/ql/optimizer/ColumnPrunerProcFactory.java |  34 +--
 .../hive/ql/optimizer/ConstantPropagate.java       |  16 +-
 .../ql/optimizer/ConstantPropagateProcFactory.java |  30 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java      |   4 +-
 .../ql/optimizer/CountDistinctRewriteProc.java     |  22 +-
 .../DynamicPartitionPruningOptimization.java       |   4 +-
 .../ql/optimizer/FixedBucketPruningOptimizer.java  |   8 +-
 .../hadoop/hive/ql/optimizer/GenMRFileSink1.java   |   7 +-
 .../hadoop/hive/ql/optimizer/GenMROperator.java    |   4 +-
 .../hadoop/hive/ql/optimizer/GenMRProcContext.java |   1 -
 .../hadoop/hive/ql/optimizer/GenMRRedSink1.java    |   5 +-
 .../hadoop/hive/ql/optimizer/GenMRRedSink2.java    |   5 +-
 .../hadoop/hive/ql/optimizer/GenMRRedSink3.java    |   5 +-
 .../hadoop/hive/ql/optimizer/GenMRTableScan1.java  |   7 +-
 .../hadoop/hive/ql/optimizer/GenMRUnion1.java      |   5 +-
 .../hadoop/hive/ql/optimizer/GroupByOptimizer.java |  26 +-
 .../hive/ql/optimizer/IdentityProjectRemover.java  |  12 +-
 .../hive/ql/optimizer/LimitPushdownOptimizer.java  |  23 +-
 .../hadoop/hive/ql/optimizer/MapJoinFactory.java   |   7 +-
 .../hadoop/hive/ql/optimizer/MapJoinProcessor.java |  30 +-
 .../hadoop/hive/ql/optimizer/MergeJoinProc.java    |   4 +-
 .../hive/ql/optimizer/NonBlockingOpDeDupProc.java  |  18 +-
 .../ql/optimizer/PartitionColumnsSeparator.java    |  24 +-
 .../hive/ql/optimizer/PointLookupOptimizer.java    |  24 +-
 .../optimizer/PrunerExpressionOperatorFactory.java |  18 +-
 .../hive/ql/optimizer/PrunerOperatorFactory.java   |  10 +-
 .../hadoop/hive/ql/optimizer/PrunerUtils.java      |  27 +-
 .../hive/ql/optimizer/ReduceSinkMapJoinProc.java   |   4 +-
 .../RedundantDynamicPruningConditionsRemoval.java  |  16 +-
 .../ql/optimizer/RemoveDynamicPruningBySize.java   |   4 +-
 .../hadoop/hive/ql/optimizer/SamplePruner.java     |  22 +-
 .../ql/optimizer/SetHashGroupByMinReduction.java   |   6 +-
 .../hive/ql/optimizer/SetReducerParallelism.java   |   7 +-
 .../hive/ql/optimizer/SimpleFetchAggregation.java  |  16 +-
 .../hive/ql/optimizer/SkewJoinOptimizer.java       |  18 +-
 .../ql/optimizer/SortedDynPartitionOptimizer.java  |  18 +-
 ...SortedDynPartitionTimeGranularityOptimizer.java |  18 +-
 .../SortedMergeBucketMapJoinOptimizer.java         |  34 +--
 .../ql/optimizer/SortedMergeBucketMapjoinProc.java |   4 +-
 .../hive/ql/optimizer/SortedMergeJoinProc.java     |   4 +-
 .../ql/optimizer/SparkRemoveDynamicPruning.java    |   4 +-
 .../hadoop/hive/ql/optimizer/StatsOptimizer.java   |  16 +-
 .../TablePropertyEnrichmentOptimizer.java          |  16 +-
 .../translator/HiveOpConverterPostProc.java        |  18 +-
 .../correlation/CorrelationOptimizer.java          |  22 +-
 .../correlation/ReduceSinkDeDuplication.java       |  26 +-
 .../correlation/ReduceSinkJoinDeDuplication.java   |  22 +-
 .../hive/ql/optimizer/lineage/ExprProcFactory.java |  28 +-
 .../hive/ql/optimizer/lineage/Generator.java       |  14 +-
 .../hive/ql/optimizer/lineage/OpProcFactory.java   |  42 +--
 .../listbucketingpruner/LBExprProcCtx.java         |   2 +-
 .../listbucketingpruner/LBExprProcFactory.java     |   4 +-
 .../LBPartitionProcFactory.java                    |   6 +-
 .../listbucketingpruner/LBProcFactory.java         |   6 +-
 .../metainfo/annotation/AnnotateWithOpTraits.java  |  14 +-
 .../annotation/OpTraitsRulesProcFactory.java       |  34 +--
 .../optimizer/pcr/PartitionConditionRemover.java   |  15 +-
 .../hive/ql/optimizer/pcr/PcrExprProcFactory.java  |  30 +-
 .../hive/ql/optimizer/pcr/PcrOpProcFactory.java    |  10 +-
 .../physical/AbstractJoinTaskDispatcher.java       |   5 +-
 .../physical/AnnotateRunTimeStatsOptimizer.java    |  19 +-
 .../BucketingSortingInferenceOptimizer.java        |  14 +-
 .../physical/BucketingSortingOpProcFactory.java    |  38 +--
 .../ql/optimizer/physical/CommonJoinResolver.java  |   4 +-
 .../physical/CommonJoinTaskDispatcher.java         |   5 +-
 .../ql/optimizer/physical/CrossProductHandler.java |  29 +-
 .../hive/ql/optimizer/physical/LlapDecider.java    |  34 ++-
 .../physical/LlapPreVectorizationPass.java         |  24 +-
 .../physical/LocalMapJoinProcFactory.java          |  26 +-
 .../ql/optimizer/physical/MapJoinResolver.java     |  18 +-
 .../hive/ql/optimizer/physical/MemoryDecider.java  |  23 +-
 .../optimizer/physical/MetadataOnlyOptimizer.java  |  18 +-
 .../ql/optimizer/physical/NullScanOptimizer.java   |  20 +-
 .../optimizer/physical/NullScanTaskDispatcher.java |  19 +-
 .../ql/optimizer/physical/SerializeFilter.java     |  23 +-
 .../ql/optimizer/physical/SkewJoinProcFactory.java |  11 +-
 .../ql/optimizer/physical/SkewJoinResolver.java    |  21 +-
 .../optimizer/physical/SortMergeJoinResolver.java  |   4 +-
 .../physical/SortMergeJoinTaskDispatcher.java      |   4 +-
 .../optimizer/physical/SparkCrossProductCheck.java |   5 +-
 .../SparkDynamicPartitionPruningResolver.java      |   6 +-
 .../optimizer/physical/SparkMapJoinResolver.java   |   6 +-
 .../hive/ql/optimizer/physical/Vectorizer.java     |   7 +-
 .../hive/ql/optimizer/ppr/ExprProcFactory.java     |   4 +-
 .../hive/ql/optimizer/ppr/OpProcFactory.java       |   6 +-
 .../spark/CombineEquivalentWorkResolver.java       |   8 +-
 .../spark/SetSparkReducerParallelism.java          |   6 +-
 .../ql/optimizer/spark/SparkJoinHintOptimizer.java |   4 +-
 .../ql/optimizer/spark/SparkJoinOptimizer.java     |   4 +-
 .../ql/optimizer/spark/SparkMapJoinOptimizer.java  |   5 +-
 .../spark/SparkReduceSinkMapJoinProc.java          |  18 +-
 .../optimizer/spark/SparkSMBJoinHintOptimizer.java |   4 +-
 .../optimizer/spark/SparkSkewJoinProcFactory.java  |   9 +-
 .../ql/optimizer/spark/SparkSkewJoinResolver.java  |  21 +-
 .../spark/SparkSortMergeJoinOptimizer.java         |   4 +-
 .../stats/annotation/AnnotateWithStatistics.java   |  14 +-
 .../stats/annotation/StatsRulesProcFactory.java    |  38 +--
 .../ql/optimizer/topnkey/TopNKeyProcessor.java     |   4 +-
 .../topnkey/TopNKeyPushdownProcessor.java          |   4 +-
 .../ql/optimizer/unionproc/UnionProcFactory.java   |  22 +-
 .../ql/optimizer/unionproc/UnionProcessor.java     |  12 +-
 .../hive/ql/parse/AppMasterEventProcessor.java     |   4 +-
 .../hadoop/hive/ql/parse/BaseSemanticAnalyzer.java |  32 ++-
 .../hadoop/hive/ql/parse/CalcitePlanner.java       |   3 +-
 .../hadoop/hive/ql/parse/DDLSemanticAnalyzer.java  |   3 +-
 .../hive/ql/parse/ExportSemanticAnalyzer.java      |   5 +-
 .../hadoop/hive/ql/parse/FileSinkProcessor.java    |   6 +-
 .../hadoop/hive/ql/parse/GenMapRedWalker.java      |   4 +-
 .../hadoop/hive/ql/parse/GenTezProcContext.java    |   3 +-
 .../apache/hadoop/hive/ql/parse/GenTezUtils.java   |  16 +-
 .../apache/hadoop/hive/ql/parse/GenTezWork.java    |   6 +-
 .../hadoop/hive/ql/parse/GenTezWorkWalker.java     |   4 +-
 .../hadoop/hive/ql/parse/LoadSemanticAnalyzer.java |  15 +-
 .../hadoop/hive/ql/parse/MapReduceCompiler.java    |  15 +-
 .../hive/ql/parse/MergeSemanticAnalyzer.java       |   2 +-
 .../hive/ql/parse/OptimizeTezProcContext.java      |   2 +-
 .../hadoop/hive/ql/parse/ProcessAnalyzeTable.java  |   7 +-
 .../apache/hadoop/hive/ql/parse/QBSubQuery.java    |  42 ++-
 .../hive/ql/parse/ReplicationSemanticAnalyzer.java |   2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java     | 116 +++++---
 .../apache/hadoop/hive/ql/parse/SubQueryUtils.java |  14 +-
 .../hadoop/hive/ql/parse/TableAccessAnalyzer.java  |  24 +-
 .../apache/hadoop/hive/ql/parse/TezCompiler.java   |  72 ++---
 .../hadoop/hive/ql/parse/UnionProcessor.java       |   4 +-
 .../hive/ql/parse/spark/GenSparkProcContext.java   |   1 -
 .../hadoop/hive/ql/parse/spark/GenSparkWork.java   |   6 +-
 .../hive/ql/parse/spark/GenSparkWorkWalker.java    |   4 +-
 .../hadoop/hive/ql/parse/spark/SparkCompiler.java  |  49 ++--
 .../ql/parse/spark/SparkFileSinkProcessor.java     |   6 +-
 .../ql/parse/spark/SparkProcessAnalyzeTable.java   |  10 +-
 .../hive/ql/parse/spark/SplitOpTreeForDPP.java     |   4 +-
 .../parse/type/JoinCondTypeCheckProcFactory.java   |  19 +-
 .../hive/ql/parse/type/TypeCheckProcFactory.java   |  98 ++++---
 .../hive/ql/plan/mapper/AuxOpTreeSignature.java    |  12 +-
 .../hadoop/hive/ql/ppd/ExprWalkerProcFactory.java  |  30 +-
 .../apache/hadoop/hive/ql/ppd/OpProcFactory.java   |  44 +--
 .../hadoop/hive/ql/ppd/PredicatePushDown.java      |  14 +-
 .../hive/ql/ppd/PredicateTransitivePropagate.java  |  16 +-
 .../hive/ql/ppd/SimplePredicatePushDown.java       |  14 +-
 .../hadoop/hive/ql/ppd/SyntheticJoinPredicate.java |  18 +-
 .../apache/hadoop/hive/ql/tools/LineageInfo.java   |  17 +-
 .../hadoop/hive/ql/exec/TestFunctionRegistry.java  |   4 +-
 udf/pom.xml                                        |  78 ++++++
 .../apache/hadoop/hive/ql/exec/MethodUtils.java    | 308 +++++++++++++++++++++
 188 files changed, 1989 insertions(+), 1632 deletions(-)

diff --git a/accumulo-handler/pom.xml b/accumulo-handler/pom.xml
index 20637c0..23433a5 100644
--- a/accumulo-handler/pom.xml
+++ b/accumulo-handler/pom.xml
@@ -96,6 +96,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-udf</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-exec</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/predicate/AccumuloPredicateHandler.java b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/predicate/AccumuloPredicateHandler.java
index 6a56618..dfa9903 100644
--- a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/predicate/AccumuloPredicateHandler.java
+++ b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/predicate/AccumuloPredicateHandler.java
@@ -52,11 +52,11 @@ import org.apache.hadoop.hive.ql.index.IndexPredicateAnalyzer;
 import org.apache.hadoop.hive.ql.index.IndexSearchCondition;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler.DecomposedPredicate;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -260,9 +260,9 @@ public class AccumuloPredicateHandler {
                                   String hiveRowIdColumnName, ExprNodeDesc root) {
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler,
         columnMapper.getRowIdMapping(), hiveRowIdColumnName);
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule, NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     List<Node> roots = new ArrayList<Node>();
     roots.add(root);
     HashMap<Node, Object> nodeOutput = new HashMap<Node, Object>();
diff --git a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/predicate/AccumuloRangeGenerator.java b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/predicate/AccumuloRangeGenerator.java
index 1796382..fd4a8cc 100644
--- a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/predicate/AccumuloRangeGenerator.java
+++ b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/predicate/AccumuloRangeGenerator.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hive.accumulo.predicate.compare.LessThan;
 import org.apache.hadoop.hive.accumulo.predicate.compare.LessThanOrEqual;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -45,7 +45,6 @@ import org.apache.hadoop.hive.serde2.lazy.LazyUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.UTF8;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,7 +59,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 /**
  *
  */
-public class AccumuloRangeGenerator implements NodeProcessor {
+public class AccumuloRangeGenerator implements SemanticNodeProcessor {
   private static final Logger LOG = LoggerFactory.getLogger(AccumuloRangeGenerator.class);
 
   private final AccumuloPredicateHandler predicateHandler;
diff --git a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/predicate/TestAccumuloRangeGenerator.java b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/predicate/TestAccumuloRangeGenerator.java
index 4975fa0..0b78556 100644
--- a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/predicate/TestAccumuloRangeGenerator.java
+++ b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/predicate/TestAccumuloRangeGenerator.java
@@ -24,22 +24,20 @@ import org.apache.hadoop.hive.accumulo.AccumuloHiveConstants;
 import org.apache.hadoop.hive.accumulo.TestAccumuloDefaultIndexScanner;
 import org.apache.hadoop.hive.accumulo.columns.ColumnEncoding;
 import org.apache.hadoop.hive.accumulo.columns.HiveAccumuloRowIdColumnMapping;
-import org.apache.hadoop.hive.accumulo.serde.AccumuloSerDeParameters;
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToString;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
@@ -47,7 +45,6 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPPlus;
-import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.junit.Assert;
 import org.junit.Before;
@@ -113,9 +110,9 @@ public class TestAccumuloRangeGenerator {
         .asList(new Range(new Key("f"), true, new Key("m\0"), false));
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "rid");
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(both);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
@@ -168,9 +165,9 @@ public class TestAccumuloRangeGenerator {
     List<Range> expectedRanges = Arrays.asList(new Range());
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "rid");
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(both);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
@@ -241,9 +238,9 @@ public class TestAccumuloRangeGenerator {
     List<Range> expectedRanges = Arrays.asList(new Range(new Key("q"), true, null, false));
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "rid");
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(both);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
@@ -296,9 +293,9 @@ public class TestAccumuloRangeGenerator {
     List<Range> expectedRanges = Arrays.asList(new Range(new Key("f"), true, null, false));
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "rid");
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(both);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
@@ -354,9 +351,9 @@ public class TestAccumuloRangeGenerator {
         "2014-07-01"), false));
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "rid");
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(both);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
@@ -401,9 +398,9 @@ public class TestAccumuloRangeGenerator {
         new GenericUDFOPEqualOrGreaterThan(), Arrays.asList(key, cast));
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "key");
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(node);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
@@ -450,9 +447,9 @@ public class TestAccumuloRangeGenerator {
         new GenericUDFOPAnd(), bothFilters);
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "rid");
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(both);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
@@ -500,9 +497,9 @@ public class TestAccumuloRangeGenerator {
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "rid");
     rangeGenerator.setIndexScanner(TestAccumuloDefaultIndexScanner.buildMockHandler(10));
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(both);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
@@ -558,9 +555,9 @@ public class TestAccumuloRangeGenerator {
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "rid");
     rangeGenerator.setIndexScanner(TestAccumuloDefaultIndexScanner.buildMockHandler(10));
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(both);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
@@ -598,9 +595,9 @@ public class TestAccumuloRangeGenerator {
 
     AccumuloRangeGenerator rangeGenerator = new AccumuloRangeGenerator(conf, handler, rowIdMapping, "rid");
     rangeGenerator.setIndexScanner(TestAccumuloDefaultIndexScanner.buildMockHandler(10));
-    Dispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
-        Collections.<Rule,NodeProcessor> emptyMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(rangeGenerator,
+        Collections.<SemanticRule, SemanticNodeProcessor> emptyMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(node);
     HashMap<Node,Object> nodeOutput = new HashMap<Node,Object>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java b/common/src/java/org/apache/hadoop/hive/ql/lib/Dispatcher.java
similarity index 55%
copy from ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java
copy to common/src/java/org/apache/hadoop/hive/ql/lib/Dispatcher.java
index 9f843fe..dec7a48 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java
+++ b/common/src/java/org/apache/hadoop/hive/ql/lib/Dispatcher.java
@@ -15,31 +15,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.hadoop.hive.ql.lib;
 
 import java.util.Stack;
 
-import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
- * CompositeProcessor. Holds a list of node processors to be fired by the same
- * rule.
- *
+ * Dispatcher interface for Operators Used in operator graph walking to dispatch
+ * process/visitor functions for operators.
  */
-public class CompositeProcessor implements NodeProcessor {
-
-  NodeProcessor[] procs;
+public interface Dispatcher {
 
-  public CompositeProcessor(NodeProcessor...nodeProcessors) {
-    procs = nodeProcessors;
-  }
+  /**
+   * Dispatcher function.
+   *
+   * @param nd
+   *          operator to process.
+   * @param stack
+   *          operator stack to process.
+   * @param nodeOutputs
+   *          The argument list of outputs from processing other nodes that are
+   *          passed to this dispatcher from the walker.
+   * @return Object The return object from the processing call.
+   * @throws HiveException
+   */
+  Object dispatch(Node nd, Stack<Node> stack, Object... nodeOutputs)
+      throws HiveException;
 
-  @Override
-  public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
-      throws SemanticException {
-    for (NodeProcessor proc: procs) {
-      proc.process(nd, stack, procCtx, nodeOutputs);
-    }
-    return null;
-  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java b/common/src/java/org/apache/hadoop/hive/ql/lib/GraphWalker.java
similarity index 58%
copy from ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java
copy to common/src/java/org/apache/hadoop/hive/ql/lib/GraphWalker.java
index 9f843fe..37bb93d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java
+++ b/common/src/java/org/apache/hadoop/hive/ql/lib/GraphWalker.java
@@ -15,31 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.hadoop.hive.ql.lib;
 
-import java.util.Stack;
+import java.util.Collection;
+import java.util.HashMap;
 
-import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
- * CompositeProcessor. Holds a list of node processors to be fired by the same
- * rule.
- *
+ * Interface for operator graph walker.
  */
-public class CompositeProcessor implements NodeProcessor {
-
-  NodeProcessor[] procs;
+public interface GraphWalker {
 
-  public CompositeProcessor(NodeProcessor...nodeProcessors) {
-    procs = nodeProcessors;
-  }
+  /**
+   * starting point for walking.
+   *
+   * @param startNodes
+   *          list of starting operators
+   * @param nodeOutput
+   *          If this parameter is not null, the call to the function returns
+   *          the map from node to objects returned by the processors.
+   * @throws HiveException
+   */
+  void startWalking(Collection<Node> startNodes,
+                    HashMap<Node, Object> nodeOutput) throws HiveException;
 
-  @Override
-  public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
-      throws SemanticException {
-    for (NodeProcessor proc: procs) {
-      proc.process(nd, stack, procCtx, nodeOutputs);
-    }
-    return null;
-  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java b/common/src/java/org/apache/hadoop/hive/ql/lib/NodeProcessor.java
similarity index 54%
copy from ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java
copy to common/src/java/org/apache/hadoop/hive/ql/lib/NodeProcessor.java
index 9f843fe..d8d1f5c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java
+++ b/common/src/java/org/apache/hadoop/hive/ql/lib/NodeProcessor.java
@@ -19,27 +19,26 @@ package org.apache.hadoop.hive.ql.lib;
 
 import java.util.Stack;
 
-import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
- * CompositeProcessor. Holds a list of node processors to be fired by the same
- * rule.
- *
+ * Base class for processing operators which is no-op. The specific processors
+ * can register their own context with the dispatcher.
  */
-public class CompositeProcessor implements NodeProcessor {
-
-  NodeProcessor[] procs;
-
-  public CompositeProcessor(NodeProcessor...nodeProcessors) {
-    procs = nodeProcessors;
-  }
+public interface NodeProcessor {
 
-  @Override
-  public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
-      throws SemanticException {
-    for (NodeProcessor proc: procs) {
-      proc.process(nd, stack, procCtx, nodeOutputs);
-    }
-    return null;
-  }
+  /**
+   * Generic process for all ops that don't have specific implementations.
+   *
+   * @param nd
+   *          operator to process
+   * @param procCtx
+   *          operator processor context
+   * @param nodeOutputs
+   *          A variable argument list of outputs from other nodes in the walk
+   * @return Object to be returned by the process call
+   * @throws HiveException
+   */
+  Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
+                 Object... nodeOutputs) throws HiveException;
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java b/common/src/java/org/apache/hadoop/hive/ql/lib/Rule.java
similarity index 62%
copy from ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java
copy to common/src/java/org/apache/hadoop/hive/ql/lib/Rule.java
index 21e527f..6594c2b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java
+++ b/common/src/java/org/apache/hadoop/hive/ql/lib/Rule.java
@@ -20,32 +20,23 @@ package org.apache.hadoop.hive.ql.lib;
 
 import java.util.Stack;
 
-import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
- * Rule that matches a particular type of node.
+ * Rule interface for Operators Used in operator dispatching to dispatch
+ * process/visitor functions for operators.
  */
-public class TypeRule implements Rule {
-
-  private Class<?> nodeClass;
-
-  public TypeRule(Class<?> nodeClass) {
-    this.nodeClass = nodeClass;
-  }
-
-  @Override
-  public int cost(Stack<Node> stack) throws SemanticException {
-    if (stack == null) {
-      return -1;
-    }
-    if (nodeClass.isInstance(stack.peek())) {
-      return 1;
-    }
-    return -1;
-  }
-
-  @Override
-  public String getName() {
-    return nodeClass.getName();
-  }
+public interface Rule {
+
+  /**
+   * @return the cost of the rule - the lower the cost, the better the rule
+   *         matches
+   * @throws HiveException
+   */
+  int cost(Stack<Node> stack) throws HiveException;
+
+  /**
+   * @return the name of the rule - may be useful for debugging
+   */
+  String getName();
 }
diff --git a/itests/pom.xml b/itests/pom.xml
index 6d8bf97..3dc95bf 100644
--- a/itests/pom.xml
+++ b/itests/pom.xml
@@ -179,6 +179,17 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hive</groupId>
+        <artifactId>hive-udf</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hive</groupId>
+        <artifactId>hive-udf</artifactId>
+        <version>${project.version}</version>
+        <classifier>tests</classifier>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hive</groupId>
         <artifactId>hive-exec</artifactId>
         <version>${project.version}</version>
       </dependency>
diff --git a/itests/qtest-accumulo/pom.xml b/itests/qtest-accumulo/pom.xml
index 825fd3b..b0373d5 100644
--- a/itests/qtest-accumulo/pom.xml
+++ b/itests/qtest-accumulo/pom.xml
@@ -112,6 +112,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-udf</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-exec</artifactId>
       <scope>test</scope>
       <classifier>core</classifier>
diff --git a/itests/qtest-kudu/pom.xml b/itests/qtest-kudu/pom.xml
index a7e60e1..132d22c 100644
--- a/itests/qtest-kudu/pom.xml
+++ b/itests/qtest-kudu/pom.xml
@@ -101,6 +101,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-udf</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-exec</artifactId>
       <scope>test</scope>
       <classifier>core</classifier>
diff --git a/itests/qtest-spark/pom.xml b/itests/qtest-spark/pom.xml
index ebd7b63..b6bbeef 100644
--- a/itests/qtest-spark/pom.xml
+++ b/itests/qtest-spark/pom.xml
@@ -162,6 +162,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-udf</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-exec</artifactId>
       <scope>test</scope>
     </dependency>
diff --git a/kudu-handler/pom.xml b/kudu-handler/pom.xml
index 6f02bd0..c4661fa 100644
--- a/kudu-handler/pom.xml
+++ b/kudu-handler/pom.xml
@@ -37,6 +37,12 @@
     <!-- intra-project -->
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-udf</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-exec</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
diff --git a/parser/pom.xml b/parser/pom.xml
new file mode 100644
index 0000000..05fd78d
--- /dev/null
+++ b/parser/pom.xml
@@ -0,0 +1,107 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive</artifactId>
+    <version>4.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>hive-parser</artifactId>
+  <packaging>jar</packaging>
+  <name>Hive Parser</name>
+
+  <properties>
+    <hive.path.to.root>..</hive.path.to.root>
+  </properties>
+
+  <dependencies>
+    <!-- dependencies are always listed in sorted order by groupId, artifectId -->
+    <!-- intra-proect -->
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-common</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.eclipse.jetty.aggregate</groupId>
+          <artifactId>jetty-all</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- inter-project -->
+    <dependency>
+      <groupId>org.antlr</groupId>
+      <artifactId>antlr-runtime</artifactId>
+      <version>${antlr.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.antlr</groupId>
+      <artifactId>ST4</artifactId>
+      <version>${ST4.version}</version>
+    </dependency>
+    <!-- test inter-project -->
+  </dependencies>
+
+  <build>
+    <sourceDirectory>${basedir}/src/java</sourceDirectory>
+    <testSourceDirectory>${basedir}/src/test</testSourceDirectory>
+    <plugins>
+      <!-- plugins are always listed in sorted order by groupId, artifectId -->
+      <plugin>
+        <groupId>org.antlr</groupId>
+        <artifactId>antlr3-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>antlr</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <sourceDirectory>${basedir}/src/java</sourceDirectory>
+          <includes>
+            <include>**/HiveLexer.g</include>
+            <include>**/HiveParser.g</include>
+            <include>**/HintParser.g</include>
+          </includes>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>add-source</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>${project.build.directory}/generated-sources/java</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/ASTErrorUtils.java b/parser/src/java/org/apache/hadoop/hive/ql/parse/ASTErrorUtils.java
new file mode 100644
index 0000000..ed05673
--- /dev/null
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/ASTErrorUtils.java
@@ -0,0 +1,106 @@
+/*
+ * 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.parse;
+
+import org.antlr.runtime.tree.Tree;
+
+public class ASTErrorUtils {
+
+  private static int getLine(ASTNode tree) {
+    if (tree.getChildCount() == 0) {
+      return tree.getToken().getLine();
+    }
+
+    return getLine((ASTNode) tree.getChild(0));
+  }
+
+  private static int getCharPositionInLine(ASTNode tree) {
+    if (tree.getChildCount() == 0) {
+      return tree.getToken().getCharPositionInLine();
+    }
+
+    return getCharPositionInLine((ASTNode) tree.getChild(0));
+  }
+
+  // Dirty hack as this will throw away spaces and other things - find a better
+  // way!
+  public static String getText(ASTNode tree) {
+    if (tree.getChildCount() == 0) {
+      return tree.getText();
+    }
+    return getText((ASTNode) tree.getChild(tree.getChildCount() - 1));
+  }
+
+  public static String getMsg(String mesg, ASTNode tree) {
+    StringBuilder sb = new StringBuilder();
+    renderPosition(sb, tree);
+    sb.append(" ");
+    sb.append(mesg);
+    sb.append(" '");
+    sb.append(getText(tree));
+    sb.append("'");
+    renderOrigin(sb, tree.getOrigin());
+    return sb.toString();
+  }
+
+  static final String LINE_SEP = System.getProperty("line.separator");
+
+  public static void renderOrigin(StringBuilder sb, ASTNodeOrigin origin) {
+    while (origin != null) {
+      sb.append(" in definition of ");
+      sb.append(origin.getObjectType());
+      sb.append(" ");
+      sb.append(origin.getObjectName());
+      sb.append(" [");
+      sb.append(LINE_SEP);
+      sb.append(origin.getObjectDefinition());
+      sb.append(LINE_SEP);
+      sb.append("] used as ");
+      sb.append(origin.getUsageAlias());
+      sb.append(" at ");
+      ASTNode usageNode = origin.getUsageNode();
+      renderPosition(sb, usageNode);
+      origin = usageNode.getOrigin();
+    }
+  }
+
+  private static void renderPosition(StringBuilder sb, ASTNode tree) {
+    sb.append("Line ");
+    sb.append(getLine(tree));
+    sb.append(":");
+    sb.append(getCharPositionInLine(tree));
+  }
+
+  public static String renderPosition(ASTNode n) {
+    StringBuilder sb = new StringBuilder();
+    renderPosition(sb, n);
+    return sb.toString();
+  }
+
+  public static String getMsg(String mesg, Tree tree) {
+    return getMsg(mesg, (ASTNode) tree);
+  }
+
+  public static String getMsg(String mesg, ASTNode tree, String reason) {
+    return getMsg(mesg, tree) + ": " + reason;
+  }
+
+  public static String getMsg(String mesg, Tree tree, String reason) {
+    return getMsg(mesg, (ASTNode) tree, reason);
+  }
+}
diff --git a/pom.xml b/pom.xml
index d4c23a0..2dd2128 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,6 +46,8 @@
     <module>hplsql</module>
     <module>jdbc</module>
     <module>metastore</module>
+    <module>parser</module>
+    <module>udf</module>
     <module>ql</module>
     <module>serde</module>
     <module>service-rpc</module>
diff --git a/ql/pom.xml b/ql/pom.xml
index 08a8ff7..3632a5e 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -65,6 +65,16 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
+      <artifactId>hive-parser</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-udf</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
       <artifactId>hive-service-rpc</artifactId>
       <version>${project.version}</version>
     </dependency>
@@ -145,16 +155,6 @@
       <version>${log4j2.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.antlr</groupId>
-      <artifactId>antlr-runtime</artifactId>
-      <version>${antlr.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.antlr</groupId>
-      <artifactId>ST4</artifactId>
-      <version>${ST4.version}</version>
-    </dependency>
-    <dependency>
       <groupId>org.apache.avro</groupId>
       <artifactId>avro</artifactId>
       <version>${avro.version}</version>
@@ -857,25 +857,6 @@
     <plugins>
       <!-- plugins are always listed in sorted order by groupId, artifectId -->
       <plugin>
-        <groupId>org.antlr</groupId>
-        <artifactId>antlr3-maven-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals>
-              <goal>antlr</goal>
-            </goals>
-          </execution>
-        </executions>
-        <configuration>
-          <sourceDirectory>${basedir}/src/java</sourceDirectory>
-          <includes>
-            <include>**/HiveLexer.g</include>
-            <include>**/HiveParser.g</include>
-            <include>**/HintParser.g</include>
-          </includes>
-        </configuration>
-      </plugin>
-      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>
         <executions>
@@ -959,6 +940,7 @@
                 <includes>
                   <!-- order is meant to be the same as the ant build -->
                   <include>org.apache.hive:hive-common</include>
+                  <include>org.apache.hive:hive-udf</include>
                   <include>org.apache.hive:hive-exec</include>
                   <include>org.apache.hive:hive-serde</include>
                   <include>org.apache.hive:hive-llap-common</include>
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/function/macro/create/CreateMacroAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/function/macro/create/CreateMacroAnalyzer.java
index 4eed5c9..a2177e0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/function/macro/create/CreateMacroAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/function/macro/create/CreateMacroAnalyzer.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.ddl.DDLSemanticAnalyzerFactory.DDLType;
 import org.apache.hadoop.hive.ql.ddl.DDLWork;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
@@ -91,7 +91,7 @@ public class CreateMacroAnalyzer extends BaseSemanticAnalyzer {
     if (!arguments.isEmpty()) {
       // Walk down expression to see which arguments are actually used.
       Node expression = (Node) root.getChild(2);
-      PreOrderWalker walker = new PreOrderWalker(new Dispatcher() {
+      PreOrderWalker walker = new PreOrderWalker(new SemanticDispatcher() {
         @Override
         public Object dispatch(Node nd, Stack<Node> stack, Object... nodeOutputs) throws SemanticException {
           if (nd instanceof ASTNode) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractAlterTableAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractAlterTableAnalyzer.java
index 105636e..81800fe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractAlterTableAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractAlterTableAnalyzer.java
@@ -189,7 +189,8 @@ public abstract class AbstractAlterTableAnalyzer extends BaseSemanticAnalyzer {
       }
     }
     if (tbl.isNonNative() && !AlterTableType.NON_NATIVE_TABLE_ALLOWED.contains(op)) {
-      throw new SemanticException(ErrorMsg.ALTER_TABLE_NON_NATIVE.getMsg(tbl.getTableName()));
+      throw new SemanticException(ErrorMsg.ALTER_TABLE_NON_NATIVE.format(
+          AlterTableType.NON_NATIVE_TABLE_ALLOWED.toString(), tbl.getTableName()));
     }
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 3e43646..db5ee8d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -1074,27 +1074,6 @@ public final class FunctionRegistry {
         system.getGenericWindowingEvaluator(name, argumentOIs, isDistinct, isAllColumns);
   }
 
-  /**
-   * This method is shared between UDFRegistry and UDAFRegistry. methodName will
-   * be "evaluate" for UDFRegistry, and "aggregate"/"evaluate"/"evaluatePartial"
-   * for UDAFRegistry.
-   * @throws UDFArgumentException
-   */
-  public static <T> Method getMethodInternal(Class<? extends T> udfClass,
-      String methodName, boolean exact, List<TypeInfo> argumentClasses)
-      throws UDFArgumentException {
-
-    List<Method> mlist = new ArrayList<Method>();
-
-    for (Method m : udfClass.getMethods()) {
-      if (m.getName().equals(methodName)) {
-        mlist.add(m);
-      }
-    }
-
-    return getMethodInternal(udfClass, mlist, exact, argumentClasses);
-  }
-
   public static GenericUDAFResolver getGenericUDAFResolver(String functionName)
       throws SemanticException {
     if (LOG.isDebugEnabled()) {
@@ -1142,253 +1121,6 @@ public final class FunctionRegistry {
   }
 
   /**
-   * Returns -1 if passed does not match accepted. Otherwise return the cost
-   * (usually 0 for no conversion and 1 for conversion).
-   */
-  public static int matchCost(TypeInfo argumentPassed,
-      TypeInfo argumentAccepted, boolean exact) {
-    if (argumentAccepted.equals(argumentPassed)
-        || TypeInfoUtils.doPrimitiveCategoriesMatch(argumentPassed, argumentAccepted)) {
-      // matches
-      return 0;
-    }
-    if (argumentPassed.equals(TypeInfoFactory.voidTypeInfo)) {
-      // passing null matches everything
-      return 0;
-    }
-    if (argumentPassed.getCategory().equals(Category.LIST)
-        && argumentAccepted.getCategory().equals(Category.LIST)) {
-      // lists are compatible if and only-if the elements are compatible
-      TypeInfo argumentPassedElement = ((ListTypeInfo) argumentPassed)
-          .getListElementTypeInfo();
-      TypeInfo argumentAcceptedElement = ((ListTypeInfo) argumentAccepted)
-          .getListElementTypeInfo();
-      return matchCost(argumentPassedElement, argumentAcceptedElement, exact);
-    }
-    if (argumentPassed.getCategory().equals(Category.MAP)
-        && argumentAccepted.getCategory().equals(Category.MAP)) {
-      // lists are compatible if and only-if the elements are compatible
-      TypeInfo argumentPassedKey = ((MapTypeInfo) argumentPassed)
-          .getMapKeyTypeInfo();
-      TypeInfo argumentAcceptedKey = ((MapTypeInfo) argumentAccepted)
-          .getMapKeyTypeInfo();
-      TypeInfo argumentPassedValue = ((MapTypeInfo) argumentPassed)
-          .getMapValueTypeInfo();
-      TypeInfo argumentAcceptedValue = ((MapTypeInfo) argumentAccepted)
-          .getMapValueTypeInfo();
-      int cost1 = matchCost(argumentPassedKey, argumentAcceptedKey, exact);
-      int cost2 = matchCost(argumentPassedValue, argumentAcceptedValue, exact);
-      if (cost1 < 0 || cost2 < 0) {
-        return -1;
-      }
-      return Math.max(cost1, cost2);
-    }
-
-    if (argumentAccepted.equals(TypeInfoFactory.unknownTypeInfo)) {
-      // accepting Object means accepting everything,
-      // but there is a conversion cost.
-      return 1;
-    }
-    if (!exact && TypeInfoUtils.implicitConvertible(argumentPassed, argumentAccepted)) {
-      return 1;
-    }
-
-    return -1;
-  }
-
-  /**
-   * Given a set of candidate methods and list of argument types, try to
-   * select the best candidate based on how close the passed argument types are
-   * to the candidate argument types.
-   * For a varchar argument, we would prefer evaluate(string) over evaluate(double).
-   * @param udfMethods  list of candidate methods
-   * @param argumentsPassed list of argument types to match to the candidate methods
-   */
-  static void filterMethodsByTypeAffinity(List<Method> udfMethods, List<TypeInfo> argumentsPassed) {
-    if (udfMethods.size() > 1) {
-      // Prefer methods with a closer signature based on the primitive grouping of each argument.
-      // Score each method based on its similarity to the passed argument types.
-      int currentScore = 0;
-      int bestMatchScore = 0;
-      Method bestMatch = null;
-      for (Method m: udfMethods) {
-        currentScore = 0;
-        List<TypeInfo> argumentsAccepted =
-            TypeInfoUtils.getParameterTypeInfos(m, argumentsPassed.size());
-        Iterator<TypeInfo> argsPassedIter = argumentsPassed.iterator();
-        for (TypeInfo acceptedType : argumentsAccepted) {
-          // Check the affinity of the argument passed in with the accepted argument,
-          // based on the PrimitiveGrouping
-          TypeInfo passedType = argsPassedIter.next();
-          if (acceptedType.getCategory() == Category.PRIMITIVE
-              && passedType.getCategory() == Category.PRIMITIVE) {
-            PrimitiveGrouping acceptedPg = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(
-                ((PrimitiveTypeInfo) acceptedType).getPrimitiveCategory());
-            PrimitiveGrouping passedPg = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(
-                ((PrimitiveTypeInfo) passedType).getPrimitiveCategory());
-            if (acceptedPg == passedPg) {
-              // The passed argument matches somewhat closely with an accepted argument
-              ++currentScore;
-            }
-          }
-        }
-        // Check if the score for this method is any better relative to others
-        if (currentScore > bestMatchScore) {
-          bestMatchScore = currentScore;
-          bestMatch = m;
-        } else if (currentScore == bestMatchScore) {
-          bestMatch = null; // no longer a best match if more than one.
-        }
-      }
-
-      if (bestMatch != null) {
-        // Found a best match during this processing, use it.
-        udfMethods.clear();
-        udfMethods.add(bestMatch);
-      }
-    }
-  }
-
-  /**
-   * Gets the closest matching method corresponding to the argument list from a
-   * list of methods.
-   *
-   * @param mlist
-   *          The list of methods to inspect.
-   * @param exact
-   *          Boolean to indicate whether this is an exact match or not.
-   * @param argumentsPassed
-   *          The classes for the argument.
-   * @return The matching method.
-   */
-  public static Method getMethodInternal(Class<?> udfClass, List<Method> mlist, boolean exact,
-      List<TypeInfo> argumentsPassed) throws UDFArgumentException {
-
-    // result
-    List<Method> udfMethods = new ArrayList<Method>();
-    // The cost of the result
-    int leastConversionCost = Integer.MAX_VALUE;
-
-    for (Method m : mlist) {
-      List<TypeInfo> argumentsAccepted = TypeInfoUtils.getParameterTypeInfos(m,
-          argumentsPassed.size());
-      if (argumentsAccepted == null) {
-        // null means the method does not accept number of arguments passed.
-        continue;
-      }
-
-      boolean match = (argumentsAccepted.size() == argumentsPassed.size());
-      int conversionCost = 0;
-
-      for (int i = 0; i < argumentsPassed.size() && match; i++) {
-        int cost = matchCost(argumentsPassed.get(i), argumentsAccepted.get(i),
-            exact);
-        if (cost == -1) {
-          match = false;
-        } else {
-          conversionCost += cost;
-        }
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Method " + (match ? "did" : "didn't") + " match: passed = "
-            + argumentsPassed + " accepted = " + argumentsAccepted +
-            " method = " + m);
-      }
-      if (match) {
-        // Always choose the function with least implicit conversions.
-        if (conversionCost < leastConversionCost) {
-          udfMethods.clear();
-          udfMethods.add(m);
-          leastConversionCost = conversionCost;
-          // Found an exact match
-          if (leastConversionCost == 0) {
-            break;
-          }
-        } else if (conversionCost == leastConversionCost) {
-          // Ambiguous call: two methods with the same number of implicit
-          // conversions
-          udfMethods.add(m);
-          // Don't break! We might find a better match later.
-        } else {
-          // do nothing if implicitConversions > leastImplicitConversions
-        }
-      }
-    }
-
-    if (udfMethods.size() == 0) {
-      // No matching methods found
-      throw new NoMatchingMethodException(udfClass, argumentsPassed, mlist);
-    }
-
-    if (udfMethods.size() > 1) {
-      // First try selecting methods based on the type affinity of the arguments passed
-      // to the candidate method arguments.
-      filterMethodsByTypeAffinity(udfMethods, argumentsPassed);
-    }
-
-    if (udfMethods.size() > 1) {
-
-      // if the only difference is numeric types, pick the method
-      // with the smallest overall numeric type.
-      int lowestNumericType = Integer.MAX_VALUE;
-      boolean multiple = true;
-      Method candidate = null;
-      List<TypeInfo> referenceArguments = null;
-
-      for (Method m: udfMethods) {
-        int maxNumericType = 0;
-
-        List<TypeInfo> argumentsAccepted = TypeInfoUtils.getParameterTypeInfos(m, argumentsPassed.size());
-
-        if (referenceArguments == null) {
-          // keep the arguments for reference - we want all the non-numeric
-          // arguments to be the same
-          referenceArguments = argumentsAccepted;
-        }
-
-        Iterator<TypeInfo> referenceIterator = referenceArguments.iterator();
-
-        for (TypeInfo accepted: argumentsAccepted) {
-          TypeInfo reference = referenceIterator.next();
-
-          boolean acceptedIsPrimitive = false;
-          PrimitiveCategory acceptedPrimCat = PrimitiveCategory.UNKNOWN;
-          if (accepted.getCategory() == Category.PRIMITIVE) {
-            acceptedIsPrimitive = true;
-            acceptedPrimCat = ((PrimitiveTypeInfo) accepted).getPrimitiveCategory();
-          }
-          if (acceptedIsPrimitive && TypeInfoUtils.numericTypes.containsKey(acceptedPrimCat)) {
-            // We're looking for the udf with the smallest maximum numeric type.
-            int typeValue = TypeInfoUtils.numericTypes.get(acceptedPrimCat);
-            maxNumericType = typeValue > maxNumericType ? typeValue : maxNumericType;
-          } else if (!accepted.equals(reference)) {
-            // There are non-numeric arguments that don't match from one UDF to
-            // another. We give up at this point.
-            throw new AmbiguousMethodException(udfClass, argumentsPassed, mlist);
-          }
-        }
-
-        if (lowestNumericType > maxNumericType) {
-          multiple = false;
-          lowestNumericType = maxNumericType;
-          candidate = m;
-        } else if (maxNumericType == lowestNumericType) {
-          // multiple udfs with the same max type. Unless we find a lower one
-          // we'll give up.
-          multiple = true;
-        }
-      }
-
-      if (!multiple) {
-        return candidate;
-      } else {
-        throw new AmbiguousMethodException(udfClass, argumentsPassed, mlist);
-      }
-    }
-    return udfMethods.get(0);
-  }
-
-  /**
    * A shortcut to get the "index" GenericUDF. This is used for getting elements
    * out of array and getting values out of map.
    */
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
index 067c0f0..08eec19 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
@@ -87,12 +87,12 @@ import org.apache.hadoop.hive.ql.io.merge.MergeFileOutputFormat;
 import org.apache.hadoop.hive.ql.io.merge.MergeFileWork;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -188,7 +188,7 @@ public class DagUtils {
    */
   private final ConcurrentHashMap<String, Object> copyNotifiers = new ConcurrentHashMap<>();
 
-  class CollectFileSinkUrisNodeProcessor implements NodeProcessor {
+  class CollectFileSinkUrisNodeProcessor implements SemanticNodeProcessor {
 
     private final Set<URI> uris;
 
@@ -218,7 +218,7 @@ public class DagUtils {
     }
   }
 
-  private void addCollectFileSinkUrisRules(Map<Rule, NodeProcessor> opRules, NodeProcessor np) {
+  private void addCollectFileSinkUrisRules(Map<SemanticRule, SemanticNodeProcessor> opRules, SemanticNodeProcessor np) {
     opRules.put(new RuleRegExp("R1", FileSinkOperator.getOperatorName() + ".*"), np);
   }
 
@@ -226,11 +226,11 @@ public class DagUtils {
 
     CollectFileSinkUrisNodeProcessor np = new CollectFileSinkUrisNodeProcessor(uris);
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     addCollectFileSinkUrisRules(opRules, np);
 
-    Dispatcher disp = new DefaultRuleDispatcher(np, opRules, null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(np, opRules, null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     try {
       ogw.startWalking(topNodes, null);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/AccurateEstimatesCheckerHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/AccurateEstimatesCheckerHook.java
index 8299894..90264c1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/AccurateEstimatesCheckerHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/AccurateEstimatesCheckerHook.java
@@ -29,10 +29,10 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook;
@@ -56,7 +56,7 @@ public class AccurateEstimatesCheckerHook extends AbstractSemanticAnalyzerHook {
   private double absErr;
   private double relErr;
 
-  class EstimateCheckerHook implements NodeProcessor {
+  class EstimateCheckerHook implements SemanticNodeProcessor {
 
     Map<String, Operator<?>> opMap = new HashMap<>();
 
@@ -136,8 +136,8 @@ public class AccurateEstimatesCheckerHook extends AbstractSemanticAnalyzerHook {
       return;
     }
 
-    Dispatcher disp = new DefaultRuleDispatcher(new EstimateCheckerHook(), new HashMap<>(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(new EstimateCheckerHook(), new HashMap<>(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     HashMap<Node, Object> nodeOutput = new HashMap<Node, Object>();
     ogw.startWalking(rootOps, nodeOutput);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/NoOperatorReuseCheckerHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/NoOperatorReuseCheckerHook.java
index 9a6a2e9..ca9a954 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/NoOperatorReuseCheckerHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/NoOperatorReuseCheckerHook.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.hooks;
 
-import java.io.Serializable;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -29,10 +28,10 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
@@ -46,7 +45,7 @@ import org.apache.hadoop.hive.ql.plan.TezWork;
  */
 public class NoOperatorReuseCheckerHook implements ExecuteWithHookContext {
 
-  static class UniqueOpIdChecker implements NodeProcessor {
+  static class UniqueOpIdChecker implements SemanticNodeProcessor {
 
     Map<String, Operator<?>> opMap = new HashMap<>();
 
@@ -94,8 +93,8 @@ public class NoOperatorReuseCheckerHook implements ExecuteWithHookContext {
       return;
     }
 
-    Dispatcher disp = new DefaultRuleDispatcher(new UniqueOpIdChecker(), new HashMap<>(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(new UniqueOpIdChecker(), new HashMap<>(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     HashMap<Node, Object> nodeOutput = new HashMap<Node, Object>();
     ogw.startWalking(rootOps, nodeOutput);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java
index f39ba87..e0b85de 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java
@@ -20,12 +20,12 @@ package org.apache.hadoop.hive.ql.index;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
@@ -142,11 +142,11 @@ public class IndexPredicateAnalyzer {
     ExprNodeDesc predicate,
     final List<IndexSearchCondition> searchConditions) {
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
-    NodeProcessor nodeProcessor = new NodeProcessor() {
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
+    SemanticNodeProcessor nodeProcessor = new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack,
-        NodeProcessorCtx procCtx, Object... nodeOutputs)
+                            NodeProcessorCtx procCtx, Object... nodeOutputs)
         throws SemanticException {
 
         // We can only push down stuff which appears as part of
@@ -164,9 +164,9 @@ public class IndexPredicateAnalyzer {
       }
     };
 
-    Dispatcher disp = new DefaultRuleDispatcher(
+    SemanticDispatcher disp = new DefaultRuleDispatcher(
       nodeProcessor, opRules, null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.add(predicate);
     HashMap<Node, Object> nodeOutput = new HashMap<Node, Object>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java
index 9f843fe..a4afad4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/CompositeProcessor.java
@@ -26,18 +26,18 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
  * rule.
  *
  */
-public class CompositeProcessor implements NodeProcessor {
+public class CompositeProcessor implements SemanticNodeProcessor {
 
-  NodeProcessor[] procs;
+  SemanticNodeProcessor[] procs;
 
-  public CompositeProcessor(NodeProcessor...nodeProcessors) {
+  public CompositeProcessor(SemanticNodeProcessor...nodeProcessors) {
     procs = nodeProcessors;
   }
 
   @Override
   public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
       throws SemanticException {
-    for (NodeProcessor proc: procs) {
+    for (SemanticNodeProcessor proc: procs) {
       proc.process(nd, stack, procCtx, nodeOutputs);
     }
     return null;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/CostLessRuleDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/CostLessRuleDispatcher.java
index a67044e..7e060e2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/CostLessRuleDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/CostLessRuleDispatcher.java
@@ -28,11 +28,11 @@ import org.apache.hadoop.hive.ql.parse.ASTNode;
  * Dispatches calls to relevant method in processor. The user registers various
  * rules with the dispatcher, and the processor corresponding to the type of node
  */
-public class CostLessRuleDispatcher implements Dispatcher {
+public class CostLessRuleDispatcher implements SemanticDispatcher {
 
-  private final SetMultimap<Integer, NodeProcessor> procRules;
+  private final SetMultimap<Integer, SemanticNodeProcessor> procRules;
   private final NodeProcessorCtx procCtx;
-  private final NodeProcessor defaultProc;
+  private final SemanticNodeProcessor defaultProc;
 
   /**
    * Constructor.
@@ -41,8 +41,8 @@ public class CostLessRuleDispatcher implements Dispatcher {
    * @param rules       Map mapping the node's type to processor
    * @param procCtx     operator processor context, which is opaque to the dispatcher
    */
-  public CostLessRuleDispatcher(NodeProcessor defaultProc, SetMultimap<Integer, NodeProcessor> rules,
-      NodeProcessorCtx procCtx) {
+  public CostLessRuleDispatcher(SemanticNodeProcessor defaultProc, SetMultimap<Integer, SemanticNodeProcessor> rules,
+                                NodeProcessorCtx procCtx) {
     this.defaultProc = defaultProc;
     procRules = rules;
     this.procCtx = procCtx;
@@ -59,7 +59,7 @@ public class CostLessRuleDispatcher implements Dispatcher {
       throws SemanticException {
 
     int nodeType = ((ASTNode) nd).getType();
-    NodeProcessor processor = this.defaultProc;
+    SemanticNodeProcessor processor = this.defaultProc;
     if (this.procRules.containsKey(nodeType)) {
       processor = this.procRules.get(((ASTNode) nd).getType()).iterator().next();
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultGraphWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultGraphWalker.java
index e45b5a9..eb92a25 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultGraphWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultGraphWalker.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
  * (dispatchedList) and a list of operators that are discovered but not yet
  * dispatched
  */
-public class DefaultGraphWalker implements GraphWalker {
+public class DefaultGraphWalker implements SemanticGraphWalker {
 
   /**
    * opStack keeps the nodes that have been visited, but have not been
@@ -55,7 +55,7 @@ public class DefaultGraphWalker implements GraphWalker {
    */
   protected final List<Node> toWalk = new ArrayList<Node>();
   protected final IdentityHashMap<Node, Object> retMap = new  IdentityHashMap<Node, Object>();
-  protected final Dispatcher dispatcher;
+  protected final SemanticDispatcher dispatcher;
 
   /**
    * Constructor.
@@ -63,7 +63,7 @@ public class DefaultGraphWalker implements GraphWalker {
    * @param disp
    *          dispatcher to call for each op encountered
    */
-  public DefaultGraphWalker(Dispatcher disp) {
+  public DefaultGraphWalker(SemanticDispatcher disp) {
     dispatcher = disp;
     opStack = new Stack<Node>();
     opQueue = new LinkedList<Node>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultRuleDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultRuleDispatcher.java
index d0bb77f..0cb816b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultRuleDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/DefaultRuleDispatcher.java
@@ -28,11 +28,11 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
  * rules with the dispatcher, and the processor corresponding to closest
  * matching rule is fired.
  */
-public class DefaultRuleDispatcher implements Dispatcher {
+public class DefaultRuleDispatcher implements SemanticDispatcher {
 
-  private final Map<Rule, NodeProcessor> procRules;
+  private final Map<SemanticRule, SemanticNodeProcessor> procRules;
   private final NodeProcessorCtx procCtx;
-  private final NodeProcessor defaultProc;
+  private final SemanticNodeProcessor defaultProc;
 
   /**
    * Constructor.
@@ -44,8 +44,8 @@ public class DefaultRuleDispatcher implements Dispatcher {
    * @param procCtx
    *          operator processor context, which is opaque to the dispatcher
    */
-  public DefaultRuleDispatcher(NodeProcessor defaultProc,
-      Map<Rule, NodeProcessor> rules, NodeProcessorCtx procCtx) {
+  public DefaultRuleDispatcher(SemanticNodeProcessor defaultProc,
+                               Map<SemanticRule, SemanticNodeProcessor> rules, NodeProcessorCtx procCtx) {
     this.defaultProc = defaultProc;
     procRules = rules;
     this.procCtx = procCtx;
@@ -66,9 +66,9 @@ public class DefaultRuleDispatcher implements Dispatcher {
 
     // find the firing rule
     // find the rule from the stack specified
-    Rule rule = null;
+    SemanticRule rule = null;
     int minCost = Integer.MAX_VALUE;
-    for (Rule r : procRules.keySet()) {
+    for (SemanticRule r : procRules.keySet()) {
       int cost = r.cost(ndStack);
       if ((cost >= 0) && (cost <= minCost)) {
         minCost = cost;
@@ -76,7 +76,7 @@ public class DefaultRuleDispatcher implements Dispatcher {
       }
     }
 
-    NodeProcessor proc;
+    SemanticNodeProcessor proc;
 
     if (rule == null) {
       proc = defaultProc;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/ExpressionWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/ExpressionWalker.java
index 7b19314..bb5b2ec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/ExpressionWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/ExpressionWalker.java
@@ -36,7 +36,7 @@ public class ExpressionWalker extends DefaultGraphWalker{
    * @param disp
    * dispatcher to call for each op encountered
    */
-  public ExpressionWalker(Dispatcher disp) {
+  public ExpressionWalker(SemanticDispatcher disp) {
     super(disp);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/ForwardWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/ForwardWalker.java
index d64d694..45ebd18 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/ForwardWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/ForwardWalker.java
@@ -30,7 +30,7 @@ public class ForwardWalker extends DefaultGraphWalker {
    * @param disp
    * dispatcher to call for each op encountered
    */
-  public ForwardWalker(Dispatcher disp) {
+  public ForwardWalker(SemanticDispatcher disp) {
     super(disp);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/LevelOrderWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/LevelOrderWalker.java
index bccd9fb..96ed10d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/LevelOrderWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/LevelOrderWalker.java
@@ -60,7 +60,7 @@ public class LevelOrderWalker extends DefaultGraphWalker {
    *
    * @param disp Dispatcher to call for each op encountered
    */
-  public LevelOrderWalker(Dispatcher disp) {
+  public LevelOrderWalker(SemanticDispatcher disp) {
     super(disp);
     this.numLevels = Integer.MAX_VALUE;
   }
@@ -72,7 +72,7 @@ public class LevelOrderWalker extends DefaultGraphWalker {
    * @param disp Dispatcher to call for each op encountered
    * @param numLevels Number of ancestor levels
    */
-  public LevelOrderWalker(Dispatcher disp, int numLevels) {
+  public LevelOrderWalker(SemanticDispatcher disp, int numLevels) {
     super(disp);
     this.numLevels = numLevels;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderOnceWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderOnceWalker.java
index 06e144e..ad3356f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderOnceWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderOnceWalker.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
  */
 public class PreOrderOnceWalker extends PreOrderWalker {
 
-  public PreOrderOnceWalker(Dispatcher disp) {
+  public PreOrderOnceWalker(SemanticDispatcher disp) {
     super(disp);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java
index be31656..12e2f17 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/PreOrderWalker.java
@@ -39,7 +39,7 @@ public class PreOrderWalker extends DefaultGraphWalker {
    * @param disp
    *          dispatcher to call for each op encountered
    */
-  public PreOrderWalker(Dispatcher disp) {
+  public PreOrderWalker(SemanticDispatcher disp) {
     super(disp);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleExactMatch.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleExactMatch.java
index 10409b6..fe407c1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleExactMatch.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleExactMatch.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
  * process/visitor functions for Nodes.  The cost method returns 1 if there is an exact
  * match between the expression and the stack, otherwise -1.
  */
-public class RuleExactMatch implements Rule {
+public class RuleExactMatch implements SemanticRule {
 
   private final String ruleName;
   private final String[] pattern;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleRegExp.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleRegExp.java
index db62db2..fdfc599 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleRegExp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleRegExp.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
  * Rule interface for Nodes Used in Node dispatching to dispatch process/visitor
  * functions for Nodes.
  */
-public class RuleRegExp implements Rule {
+public class RuleRegExp implements SemanticRule {
 
   private final String ruleName;
   private final Pattern patternWithWildCardChar;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/SubqueryExpressionWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/SubqueryExpressionWalker.java
index 75f09e4..86b7efe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/SubqueryExpressionWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/SubqueryExpressionWalker.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.ql.lib;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 
-public class SubqueryExpressionWalker extends ExpressionWalker{
+public class SubqueryExpressionWalker extends ExpressionWalker {
 
   /**
    * Constructor.
@@ -29,7 +29,7 @@ public class SubqueryExpressionWalker extends ExpressionWalker{
    * @param disp
    * dispatcher to call for each op encountered
    */
-  public SubqueryExpressionWalker(Dispatcher disp) {
+  public SubqueryExpressionWalker(SemanticDispatcher disp) {
     super(disp);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/TaskGraphWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/TaskGraphWalker.java
index 23cdb62..b1c7adc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/TaskGraphWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/TaskGraphWalker.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.lib;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -37,7 +36,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
  * (dispatchedList) and a list of operators that are discovered but not yet
  * dispatched
  */
-public class TaskGraphWalker implements GraphWalker {
+public class TaskGraphWalker implements SemanticGraphWalker {
 
 
   public class TaskGraphWalkerContext{
@@ -56,7 +55,7 @@ public class TaskGraphWalker implements GraphWalker {
   protected Stack<Node> opStack;
   private final List<Node> toWalk = new ArrayList<Node>();
   private final HashMap<Node, Object> retMap = new HashMap<Node, Object>();
-  private final Dispatcher dispatcher;
+  private final SemanticDispatcher dispatcher;
   private final  TaskGraphWalkerContext walkerCtx;
 
   /**
@@ -65,7 +64,7 @@ public class TaskGraphWalker implements GraphWalker {
    * @param disp
    *          dispatcher to call for each op encountered
    */
-  public TaskGraphWalker(Dispatcher disp) {
+  public TaskGraphWalker(SemanticDispatcher disp) {
     dispatcher = disp;
     opStack = new Stack<Node>();
     walkerCtx = new TaskGraphWalkerContext(retMap);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java
index 21e527f..cf26204 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/TypeRule.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
 /**
  * Rule that matches a particular type of node.
  */
-public class TypeRule implements Rule {
+public class TypeRule implements SemanticRule {
 
   private Class<?> nodeClass;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java
index d3fb91e..d69df77 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -59,7 +59,7 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 /**
  * this transformation does bucket map join optimization.
  */
-abstract public class AbstractBucketJoinProc implements NodeProcessor {
+abstract public class AbstractBucketJoinProc implements SemanticNodeProcessor {
 
   protected ParseContext pGraphContext;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
index e17a17f..3f30c8c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -56,7 +56,7 @@ import org.apache.hadoop.hive.ql.plan.SMBJoinDesc;
 import org.apache.hadoop.util.ReflectionUtils;
 
 //try to replace a bucket map join with a sorted merge map join
-abstract public class AbstractSMBJoinProc extends AbstractBucketJoinProc implements NodeProcessor {
+abstract public class AbstractSMBJoinProc extends AbstractBucketJoinProc implements SemanticNodeProcessor {
 
   public AbstractSMBJoinProc(ParseContext pctx) {
     super(pctx);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java
index bacd6bb..7ffeba4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java
@@ -28,12 +28,12 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -51,7 +51,7 @@ public class BucketMapJoinOptimizer extends Transform {
 
   public ParseContext transform(ParseContext pctx) throws SemanticException {
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     BucketJoinProcCtx bucketMapJoinOptimizeCtx =
         new BucketJoinProcCtx(pctx.getConf());
 
@@ -62,9 +62,9 @@ public class BucketMapJoinOptimizer extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules,
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules,
         bucketMapJoinOptimizeCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     List<Node> topNodes = new ArrayList<Node>();
@@ -74,15 +74,15 @@ public class BucketMapJoinOptimizer extends Transform {
     return pctx;
   }
 
-  private NodeProcessor getBucketMapjoinProc(ParseContext pctx) {
+  private SemanticNodeProcessor getBucketMapjoinProc(ParseContext pctx) {
     return new BucketMapjoinProc(pctx);
   }
 
-  private NodeProcessor getDefaultProc() {
-    return new NodeProcessor() {
+  private SemanticNodeProcessor getDefaultProc() {
+    return new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack,
-          NodeProcessorCtx procCtx, Object... nodeOutputs)
+                            NodeProcessorCtx procCtx, Object... nodeOutputs)
           throws SemanticException {
         return null;
       }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapjoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapjoinProc.java
index adcf618..5d4e6cd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapjoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapjoinProc.java
@@ -25,14 +25,13 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
-import org.apache.hadoop.hive.ql.parse.QBJoinTree;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 
-public class BucketMapjoinProc extends AbstractBucketJoinProc implements NodeProcessor {
+public class BucketMapjoinProc extends AbstractBucketJoinProc implements SemanticNodeProcessor {
   public BucketMapjoinProc(ParseContext pGraphContext) {
     super(pGraphContext);
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
index e733b70..f74ac2f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
@@ -38,12 +38,12 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -80,7 +80,7 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
   @Override
   public ParseContext transform(ParseContext pctx) throws SemanticException {
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
     // process reduce sink added by hive.enforce.bucketing or hive.enforce.sorting
     opRules.put(new RuleRegExp("R1",
@@ -90,8 +90,8 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
         getBucketSortReduceSinkProc(pctx));
 
     // The dispatcher fires the processor corresponding to the closest matching rule
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of top nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -101,17 +101,17 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
     return pctx;
   }
 
-  private NodeProcessor getDefaultProc() {
-    return new NodeProcessor() {
+  private SemanticNodeProcessor getDefaultProc() {
+    return new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack,
-          NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
+                            NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
         return null;
       }
     };
   }
 
-  private NodeProcessor getBucketSortReduceSinkProc(ParseContext pctx) {
+  private SemanticNodeProcessor getBucketSortReduceSinkProc(ParseContext pctx) {
     return new BucketSortReduceSinkProcessor(pctx);
   }
 
@@ -119,7 +119,7 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
    * BucketSortReduceSinkProcessor.
    *
    */
-  public class BucketSortReduceSinkProcessor implements NodeProcessor {
+  public class BucketSortReduceSinkProcessor implements SemanticNodeProcessor {
     private final Logger LOG = LoggerFactory.getLogger(BucketSortReduceSinkProcessor.class);
     protected ParseContext pGraphContext;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
index ef94ea4..bd59b95 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
@@ -38,13 +38,12 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
-import org.apache.hadoop.hive.ql.parse.ColumnAccessInfo;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 
@@ -84,7 +83,7 @@ public class ColumnPruner extends Transform {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack. The dispatcher
     // generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1",
       FilterOperator.getOperatorName() + "%"),
       ColumnPrunerProcFactory.getFilterProc());
@@ -126,9 +125,9 @@ public class ColumnPruner extends Transform {
         ColumnPrunerProcFactory.getUnionProc());
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(ColumnPrunerProcFactory
+    SemanticDispatcher disp = new DefaultRuleDispatcher(ColumnPrunerProcFactory
         .getDefaultProc(), opRules, cppCtx);
-    GraphWalker ogw = new ColumnPrunerWalker(disp);
+    SemanticGraphWalker ogw = new ColumnPrunerWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -146,7 +145,7 @@ public class ColumnPruner extends Transform {
    */
   public static class ColumnPrunerWalker extends DefaultGraphWalker {
 
-    public ColumnPrunerWalker(Dispatcher disp) {
+    public ColumnPrunerWalker(SemanticDispatcher disp) {
       super(disp);
     }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
index d3749fd..5dc6bd0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.hive.ql.exec.UDTFOperator;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
@@ -99,7 +99,7 @@ public final class ColumnPrunerProcFactory {
   /**
    * Node Processor for Column Pruning on Filter Operators.
    */
-  public static class ColumnPrunerFilterProc implements NodeProcessor {
+  public static class ColumnPrunerFilterProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
@@ -130,7 +130,7 @@ public final class ColumnPrunerProcFactory {
   /**
    * Node Processor for Column Pruning on Group By Operators.
    */
-  public static class ColumnPrunerGroupByProc implements NodeProcessor {
+  public static class ColumnPrunerGroupByProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
@@ -220,7 +220,7 @@ public final class ColumnPrunerProcFactory {
     return new ColumnPrunerGroupByProc();
   }
 
-  public static class ColumnPrunerScriptProc implements NodeProcessor {
+  public static class ColumnPrunerScriptProc implements SemanticNodeProcessor {
     @Override
     @SuppressWarnings("unchecked")
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
@@ -453,7 +453,7 @@ public final class ColumnPrunerProcFactory {
   /**
    * The Default Node Processor for Column Pruning.
    */
-  public static class ColumnPrunerDefaultProc implements NodeProcessor {
+  public static class ColumnPrunerDefaultProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
@@ -478,7 +478,7 @@ public final class ColumnPrunerProcFactory {
    * The Node Processor for Column Pruning on Table Scan Operators. It will
    * store needed columns in tableScanDesc.
    */
-  public static class ColumnPrunerTableScanProc implements NodeProcessor {
+  public static class ColumnPrunerTableScanProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
@@ -566,7 +566,7 @@ public final class ColumnPrunerProcFactory {
   /**
    * The Node Processor for Column Pruning on Reduce Sink Operators.
    */
-  public static class ColumnPrunerReduceSinkProc implements NodeProcessor {
+  public static class ColumnPrunerReduceSinkProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
@@ -646,7 +646,7 @@ public final class ColumnPrunerProcFactory {
   /**
    * The Node Processor for Column Pruning on Lateral View Join Operators.
    */
-  public static class ColumnPrunerLateralViewJoinProc implements NodeProcessor {
+  public static class ColumnPrunerLateralViewJoinProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
@@ -742,7 +742,7 @@ public final class ColumnPrunerProcFactory {
   /**
    * The Node Processor for Column Pruning on Select Operators.
    */
-  public static class ColumnPrunerSelectProc implements NodeProcessor {
+  public static class ColumnPrunerSelectProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
@@ -969,7 +969,7 @@ public final class ColumnPrunerProcFactory {
   /**
    * The Node Processor for Column Pruning on Join Operators.
    */
-  public static class ColumnPrunerJoinProc implements NodeProcessor {
+  public static class ColumnPrunerJoinProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
@@ -992,7 +992,7 @@ public final class ColumnPrunerProcFactory {
   /**
    * The Node Processor for Column Pruning on Map Join Operators.
    */
-  public static class ColumnPrunerMapJoinProc implements NodeProcessor {
+  public static class ColumnPrunerMapJoinProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
@@ -1015,7 +1015,7 @@ public final class ColumnPrunerProcFactory {
   /**
    * The Node Processor for Column Pruning on Union Operators.
    */
-  public static class ColumnPrunerUnionProc implements NodeProcessor {
+  public static class ColumnPrunerUnionProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
@@ -1042,8 +1042,8 @@ public final class ColumnPrunerProcFactory {
   }
 
   private static void pruneOperator(NodeProcessorCtx ctx,
-      Operator<? extends OperatorDesc> op,
-      List<FieldNode> cols)
+                                    Operator<? extends OperatorDesc> op,
+                                    List<FieldNode> cols)
       throws SemanticException {
     // the pruning needs to preserve the order of columns in the input schema
     RowSchema inputSchema = op.getSchema();
@@ -1086,9 +1086,9 @@ public final class ColumnPrunerProcFactory {
   }
 
   private static void pruneJoinOperator(NodeProcessorCtx ctx,
-      CommonJoinOperator op, JoinDesc conf,
-      Map<String, ExprNodeDesc> columnExprMap,
-      Map<Byte, List<Integer>> retainMap, boolean mapJoin) throws SemanticException {
+                                        CommonJoinOperator op, JoinDesc conf,
+                                        Map<String, ExprNodeDesc> columnExprMap,
+                                        Map<Byte, List<Integer>> retainMap, boolean mapJoin) throws SemanticException {
     ColumnPrunerProcCtx cppCtx = (ColumnPrunerProcCtx) ctx;
     List<Operator<? extends OperatorDesc>> childOperators = op
         .getChildOperators();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
index ef61187..47d9ec7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
@@ -37,11 +37,11 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcCtx.ConstantPropagateOption;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -89,7 +89,7 @@ public class ConstantPropagate extends Transform {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack. The dispatcher
     // generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
     opRules.put(new RuleRegExp("R1", FilterOperator.getOperatorName() + "%"),
         ConstantPropagateProcFactory.getFilterProc());
@@ -110,9 +110,9 @@ public class ConstantPropagate extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(ConstantPropagateProcFactory
+    SemanticDispatcher disp = new DefaultRuleDispatcher(ConstantPropagateProcFactory
         .getDefaultProc(), opRules, cppCtx);
-    GraphWalker ogw = new ConstantPropagateWalker(disp);
+    SemanticGraphWalker ogw = new ConstantPropagateWalker(disp);
 
     // Create a list of operator nodes to start the walking.
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -135,7 +135,7 @@ public class ConstantPropagate extends Transform {
    */
   public static class ConstantPropagateWalker extends DefaultGraphWalker {
 
-    public ConstantPropagateWalker(Dispatcher disp) {
+    public ConstantPropagateWalker(SemanticDispatcher disp) {
       super(disp);
     }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
index f6a8e91..1a26ca5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
@@ -1043,7 +1043,7 @@ public final class ConstantPropagateProcFactory {
    * Node Processor for Constant Propagation on Filter Operators. The processor is to fold
    * conditional expressions and extract assignment expressions and propagate them.
    */
-  public static class ConstantPropagateFilterProc implements NodeProcessor {
+  public static class ConstantPropagateFilterProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
@@ -1098,7 +1098,7 @@ public final class ConstantPropagateProcFactory {
   /**
    * Node Processor for Constant Propagate for Group By Operators.
    */
-  public static class ConstantPropagateGroupByProc implements NodeProcessor {
+  public static class ConstantPropagateGroupByProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
@@ -1147,7 +1147,7 @@ public final class ConstantPropagateProcFactory {
   /**
    * The Default Node Processor for Constant Propagation.
    */
-  public static class ConstantPropagateDefaultProc implements NodeProcessor {
+  public static class ConstantPropagateDefaultProc implements SemanticNodeProcessor {
     @Override
     @SuppressWarnings("unchecked")
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
@@ -1187,7 +1187,7 @@ public final class ConstantPropagateProcFactory {
   /**
    * The Node Processor for Constant Propagation for Select Operators.
    */
-  public static class ConstantPropagateSelectProc implements NodeProcessor {
+  public static class ConstantPropagateSelectProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
@@ -1255,7 +1255,7 @@ public final class ConstantPropagateProcFactory {
    * The Node Processor for constant propagation for FileSink Operators. In addition to constant
    * propagation, this processor also prunes dynamic partitions to static partitions if possible.
    */
-  public static class ConstantPropagateFileSinkProc implements NodeProcessor {
+  public static class ConstantPropagateFileSinkProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
@@ -1301,7 +1301,7 @@ public final class ConstantPropagateProcFactory {
     }
   }
 
-  public static NodeProcessor getFileSinkProc() {
+  public static SemanticNodeProcessor getFileSinkProc() {
     return new ConstantPropagateFileSinkProc();
   }
 
@@ -1309,7 +1309,7 @@ public final class ConstantPropagateProcFactory {
    * The Node Processor for Constant Propagation for Operators which is designed to stop propagate.
    * Currently these kinds of Operators include UnionOperator and ScriptOperator.
    */
-  public static class ConstantPropagateStopProc implements NodeProcessor {
+  public static class ConstantPropagateStopProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
@@ -1323,7 +1323,7 @@ public final class ConstantPropagateProcFactory {
     }
   }
 
-  public static NodeProcessor getStopProc() {
+  public static SemanticNodeProcessor getStopProc() {
     return new ConstantPropagateStopProc();
   }
 
@@ -1332,7 +1332,7 @@ public final class ConstantPropagateProcFactory {
    * a join, then only those constants from inner join tables, or from the 'inner side' of a outer
    * join (left table for left outer join and vice versa) can be propagated.
    */
-  public static class ConstantPropagateReduceSinkProc implements NodeProcessor {
+  public static class ConstantPropagateReduceSinkProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
@@ -1429,14 +1429,14 @@ public final class ConstantPropagateProcFactory {
 
   }
 
-  public static NodeProcessor getReduceSinkProc() {
+  public static SemanticNodeProcessor getReduceSinkProc() {
     return new ConstantPropagateReduceSinkProc();
   }
 
   /**
    * The Node Processor for Constant Propagation for Join Operators.
    */
-  public static class ConstantPropagateJoinProc implements NodeProcessor {
+  public static class ConstantPropagateJoinProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
@@ -1503,14 +1503,14 @@ public final class ConstantPropagateProcFactory {
 
   }
 
-  public static NodeProcessor getJoinProc() {
+  public static SemanticNodeProcessor getJoinProc() {
     return new ConstantPropagateJoinProc();
   }
 
   /**
    * The Node Processor for Constant Propagation for Table Scan Operators.
    */
-  public static class ConstantPropagateTableScanProc implements NodeProcessor {
+  public static class ConstantPropagateTableScanProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
@@ -1534,7 +1534,7 @@ public final class ConstantPropagateProcFactory {
     }
   }
 
-  public static NodeProcessor getTableScanProc() {
+  public static SemanticNodeProcessor getTableScanProc() {
     return new ConstantPropagateTableScanProc();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index 256a139..d8d8cae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.TezDummyStoreOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.physical.LlapClusterStateForCompile;
 import org.apache.hadoop.hive.ql.parse.GenTezUtils;
@@ -88,7 +88,7 @@ import com.google.common.math.DoubleMath;
  * converted (e.g.: full outer joins cannot be handled as map joins) as well
  * as memory restrictions (one side of the join has to fit into memory).
  */
-public class ConvertJoinMapJoin implements NodeProcessor {
+public class ConvertJoinMapJoin implements SemanticNodeProcessor {
 
   private static final Logger LOG = LoggerFactory.getLogger(ConvertJoinMapJoin.class.getName());
   private static final int DEFAULT_MAX_EXECUTORS_PER_QUERY_CONTAINER_MODE = 3;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/CountDistinctRewriteProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/CountDistinctRewriteProc.java
index f0cf2f1..32edacb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/CountDistinctRewriteProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/CountDistinctRewriteProc.java
@@ -40,12 +40,12 @@ import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
@@ -90,7 +90,7 @@ public class CountDistinctRewriteProc extends Transform {
   @Override
   public ParseContext transform(ParseContext pctx) throws SemanticException {
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     // process group-by pattern
     opRules
         .put(
@@ -100,8 +100,8 @@ public class CountDistinctRewriteProc extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     List<Node> topNodes = new ArrayList<Node>();
@@ -111,8 +111,8 @@ public class CountDistinctRewriteProc extends Transform {
     return pctx;
   }
 
-  private NodeProcessor getDefaultProc() {
-    return new NodeProcessor() {
+  private SemanticNodeProcessor getDefaultProc() {
+    return new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
           Object... nodeOutputs) throws SemanticException {
@@ -121,7 +121,7 @@ public class CountDistinctRewriteProc extends Transform {
     };
   }
 
-  private NodeProcessor getCountDistinctProc(ParseContext pctx) {
+  private SemanticNodeProcessor getCountDistinctProc(ParseContext pctx) {
     return new CountDistinctProcessor(pctx);
   }
 
@@ -129,7 +129,7 @@ public class CountDistinctRewriteProc extends Transform {
    * CountDistinctProcessor.
    *
    */
-  public class CountDistinctProcessor implements NodeProcessor {
+  public class CountDistinctProcessor implements SemanticNodeProcessor {
 
     protected ParseContext pGraphContext;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
index e0ab6ce..eca5a8b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities;
 import org.apache.hadoop.hive.ql.io.AcidUtils.Operation;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -95,7 +95,7 @@ import com.google.common.base.Preconditions;
  * can use an existing join to dynamically prune partitions. This class sets up
  * the infrastructure for that.
  */
-public class DynamicPartitionPruningOptimization implements NodeProcessor {
+public class DynamicPartitionPruningOptimization implements SemanticNodeProcessor {
 
   static final private Logger LOG = LoggerFactory.getLogger(DynamicPartitionPruningOptimization.class
       .getName());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FixedBucketPruningOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FixedBucketPruningOptimizer.java
index c5553fb..5818e6b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FixedBucketPruningOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FixedBucketPruningOptimizer.java
@@ -35,13 +35,11 @@ import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree.Operator;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.optimizer.PrunerOperatorFactory.FilterPruner;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExcept;
 import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
@@ -69,7 +67,7 @@ public class FixedBucketPruningOptimizer extends Transform {
     this.compat = compat;
   }
 
-  public class NoopWalker implements NodeProcessor {
+  public class NoopWalker implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -82,7 +80,7 @@ public class FixedBucketPruningOptimizer extends Transform {
 
     @Override
     protected void generatePredicate(NodeProcessorCtx procCtx,
-        FilterOperator fop, TableScanOperator top) throws SemanticException{
+                                     FilterOperator fop, TableScanOperator top) throws SemanticException{
       FixedBucketPruningOptimizerCtxt ctxt = ((FixedBucketPruningOptimizerCtxt) procCtx);
       Table tbl = top.getConf().getTableMetadata();
       int numBuckets = tbl.getNumBuckets();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
index 10a0405..f90aa84 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.optimizer;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -36,7 +35,7 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -48,7 +47,7 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 /**
  * Processor for the rule - table scan followed by reduce sink.
  */
-public class GenMRFileSink1 implements NodeProcessor {
+public class GenMRFileSink1 implements SemanticNodeProcessor {
 
   private static final Logger LOG = LoggerFactory.getLogger(GenMRFileSink1.class.getName());
 
@@ -173,7 +172,7 @@ public class GenMRFileSink1 implements NodeProcessor {
    * @throws SemanticException
    */
   private Path processFS(FileSinkOperator fsOp, Stack<Node> stack,
-      NodeProcessorCtx opProcCtx, boolean chDir) throws SemanticException {
+                         NodeProcessorCtx opProcCtx, boolean chDir) throws SemanticException {
 
     GenMRProcContext ctx = (GenMRProcContext) opProcCtx;
     Task<?> currTask = ctx.getCurrTask();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMROperator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMROperator.java
index 93288e1..c477a6a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMROperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMROperator.java
@@ -23,7 +23,7 @@ import java.util.Stack;
 
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 /**
  * Processor for the rule - no specific rule fired.
  */
-public class GenMROperator implements NodeProcessor {
+public class GenMROperator implements SemanticNodeProcessor {
 
   public GenMROperator() {
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
index bbda668..8ebdb01 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.optimizer;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink1.java
index 033cbdc..db08e56 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink1.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink1.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.optimizer;
 
-import java.io.Serializable;
 import java.util.Map;
 import java.util.Stack;
 
@@ -26,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -36,7 +35,7 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 /**
  * Processor for the rule - table scan followed by reduce sink.
  */
-public class GenMRRedSink1 implements NodeProcessor {
+public class GenMRRedSink1 implements SemanticNodeProcessor {
 
   public GenMRRedSink1() {
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink2.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink2.java
index 8c997c7..ef85be9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink2.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink2.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.optimizer;
 
-import java.io.Serializable;
 import java.util.Map;
 import java.util.Stack;
 
@@ -26,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -35,7 +34,7 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 /**
  * Processor for the rule - reduce sink followed by reduce sink.
  */
-public class GenMRRedSink2 implements NodeProcessor {
+public class GenMRRedSink2 implements SemanticNodeProcessor {
 
   public GenMRRedSink2() {
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink3.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink3.java
index ed4bb30..f437610 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink3.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRRedSink3.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.optimizer;
 
-import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Stack;
@@ -28,7 +27,7 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.Utils;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
@@ -39,7 +38,7 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 /**
  * Processor for the rule - union followed by reduce sink.
  */
-public class GenMRRedSink3 implements NodeProcessor {
+public class GenMRRedSink3 implements SemanticNodeProcessor {
 
   public GenMRRedSink3() {
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
index bb53ce8..5f060ec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
@@ -28,11 +28,8 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.mr.MapRedTask;
-import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
-import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -50,7 +47,7 @@ import org.apache.hadoop.mapred.InputFormat;
 /**
  * Processor for the rule - table scan.
  */
-public class GenMRTableScan1 implements NodeProcessor {
+public class GenMRTableScan1 implements SemanticNodeProcessor {
   public GenMRTableScan1() {
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java
index abf363a..c09d22f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.optimizer;
 
-import java.io.Serializable;
 import java.util.Map;
 import java.util.Stack;
 
@@ -30,7 +29,7 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMRUnionCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
@@ -47,7 +46,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc;
 /**
  * Processor for the rule - TableScan followed by Union.
  */
-public class GenMRUnion1 implements NodeProcessor {
+public class GenMRUnion1 implements SemanticNodeProcessor {
 
   public GenMRUnion1() {
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
index bbbc4f8..f52d565 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
@@ -43,12 +43,12 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -80,7 +80,7 @@ public class GroupByOptimizer extends Transform {
   @Override
   public ParseContext transform(ParseContext pctx) throws SemanticException {
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     HiveConf conf = pctx.getConf();
 
     if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEGROUPBYSKEW)) {
@@ -103,10 +103,10 @@ public class GroupByOptimizer extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp =
+    SemanticDispatcher disp =
         new DefaultRuleDispatcher(getDefaultProc(), opRules,
             new GroupByOptimizerContext(conf));
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     List<Node> topNodes = new ArrayList<Node>();
@@ -116,21 +116,21 @@ public class GroupByOptimizer extends Transform {
     return pctx;
   }
 
-  private NodeProcessor getDefaultProc() {
-    return new NodeProcessor() {
+  private SemanticNodeProcessor getDefaultProc() {
+    return new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack,
-          NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
+                            NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
         return null;
       }
     };
   }
 
-  private NodeProcessor getMapSortedGroupbyProc(ParseContext pctx) {
+  private SemanticNodeProcessor getMapSortedGroupbyProc(ParseContext pctx) {
     return new SortGroupByProcessor(pctx);
   }
 
-  private NodeProcessor getMapSortedGroupbySkewProc(ParseContext pctx) {
+  private SemanticNodeProcessor getMapSortedGroupbySkewProc(ParseContext pctx) {
     return new SortGroupBySkewProcessor(pctx);
   }
 
@@ -146,7 +146,7 @@ public class GroupByOptimizer extends Transform {
    * SortGroupByProcessor.
    *
    */
-  public class SortGroupByProcessor implements NodeProcessor {
+  public class SortGroupByProcessor implements SemanticNodeProcessor {
 
     protected ParseContext pGraphContext;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IdentityProjectRemover.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IdentityProjectRemover.java
index 7c841ba..a5fe3bb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IdentityProjectRemover.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IdentityProjectRemover.java
@@ -38,11 +38,11 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -83,17 +83,17 @@ public class IdentityProjectRemover extends Transform {
     }
 
     // 1. We apply the transformation
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1",
       "(" + SelectOperator.getOperatorName() + "%)"), new ProjectRemover());
-    GraphWalker ogw = new DefaultGraphWalker(new DefaultRuleDispatcher(null, opRules, null));
+    SemanticGraphWalker ogw = new DefaultGraphWalker(new DefaultRuleDispatcher(null, opRules, null));
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getTopOps().values());
     ogw.startWalking(topNodes, null);
     return pctx;
   }
 
-  private static class ProjectRemover implements NodeProcessor {
+  private static class ProjectRemover implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java
index 59ca3f7..4cae3b2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java
@@ -28,16 +28,15 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.LimitOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -92,7 +91,7 @@ public class LimitPushdownOptimizer extends Transform {
 
   @Override
   public ParseContext transform(ParseContext pctx) throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1",
         ReduceSinkOperator.getOperatorName() + "%" +
         ".*" +
@@ -105,19 +104,19 @@ public class LimitPushdownOptimizer extends Transform {
         new TopNPropagator());
 
     LimitPushdownContext context = new LimitPushdownContext(pctx.getConf());
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, context);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, context);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     List<Node> topNodes = new ArrayList<Node>(pctx.getTopOps().values());
     ogw.startWalking(topNodes, null);
     return pctx;
   }
 
-  private static class TopNReducer implements NodeProcessor {
+  private static class TopNReducer implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack,
-        NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
+                          NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
       ReduceSinkOperator rs = null;
       for (int i = stack.size() - 2 ; i >= 0; i--) {
         Operator<?> operator = (Operator<?>) stack.get(i);
@@ -158,11 +157,11 @@ public class LimitPushdownOptimizer extends Transform {
     }
   }
 
-  private static class TopNPropagator implements NodeProcessor {
+  private static class TopNPropagator implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack,
-        NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
+                          NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
       ReduceSinkOperator cRS = (ReduceSinkOperator) nd;
       if (cRS.getConf().getTopN() == -1) {
         // No limit, nothing to propagate, we just bail out
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java
index 21d792e..6a73336 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.optimizer;
 
-import java.io.Serializable;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -29,7 +28,7 @@ import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -77,7 +76,7 @@ public final class MapJoinFactory {
    * may be performed as a bucketized map-side join (or sort-merge join), the map join operator
    * is enhanced to contain the bucketing info. when it is encountered.
    */
-  private static class TableScanMapJoinProcessor implements NodeProcessor {
+  private static class TableScanMapJoinProcessor implements SemanticNodeProcessor {
 
     public static void setupBucketMapJoinInfo(MapWork plan,
         AbstractMapJoinOperator<? extends MapJoinDesc> currMapJoinOp) {
@@ -233,7 +232,7 @@ public final class MapJoinFactory {
     }
   }
 
-  public static NodeProcessor getTableScanMapJoin() {
+  public static SemanticNodeProcessor getTableScanMapJoin() {
     return new TableScanMapJoinProcessor();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
index a9506c8..4f1c9b2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
@@ -51,12 +51,12 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinKey;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.EnabledOverride;
 import org.apache.hadoop.hive.ql.parse.GenMapRedWalker;
@@ -932,7 +932,7 @@ public class MapJoinProcessor extends Transform {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack.
     // The dispatcher generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R0",
       MapJoinOperator.getOperatorName() + "%"),
       getCurrentMapJoin());
@@ -948,10 +948,10 @@ public class MapJoinProcessor extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(getDefault(), opRules, new MapJoinWalkerCtx(
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefault(), opRules, new MapJoinWalkerCtx(
         listMapJoinOpsNoRed, pactx));
 
-    GraphWalker ogw = new GenMapRedWalker(disp);
+    SemanticGraphWalker ogw = new GenMapRedWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(listMapJoinOps);
     ogw.startWalking(topNodes, null);
@@ -964,7 +964,7 @@ public class MapJoinProcessor extends Transform {
    * CurrentMapJoin.
    *
    */
-  public static class CurrentMapJoin implements NodeProcessor {
+  public static class CurrentMapJoin implements SemanticNodeProcessor {
 
     /**
      * Store the current mapjoin in the context.
@@ -1074,7 +1074,7 @@ public class MapJoinProcessor extends Transform {
    * MapJoinFS.
    *
    */
-  public static class MapJoinFS implements NodeProcessor {
+  public static class MapJoinFS implements SemanticNodeProcessor {
 
     /**
      * Store the current mapjoin in a list of mapjoins followed by a filesink.
@@ -1101,7 +1101,7 @@ public class MapJoinProcessor extends Transform {
    * MapJoinDefault.
    *
    */
-  public static class MapJoinDefault implements NodeProcessor {
+  public static class MapJoinDefault implements SemanticNodeProcessor {
 
     /**
      * Store the mapjoin in a rejected list.
@@ -1120,7 +1120,7 @@ public class MapJoinProcessor extends Transform {
    * Default.
    *
    */
-  public static class Default implements NodeProcessor {
+  public static class Default implements SemanticNodeProcessor {
 
     /**
      * Nothing to do.
@@ -1132,19 +1132,19 @@ public class MapJoinProcessor extends Transform {
     }
   }
 
-  public static NodeProcessor getMapJoinFS() {
+  public static SemanticNodeProcessor getMapJoinFS() {
     return new MapJoinFS();
   }
 
-  public static NodeProcessor getMapJoinDefault() {
+  public static SemanticNodeProcessor getMapJoinDefault() {
     return new MapJoinDefault();
   }
 
-  public static NodeProcessor getDefault() {
+  public static SemanticNodeProcessor getDefault() {
     return new Default();
   }
 
-  public static NodeProcessor getCurrentMapJoin() {
+  public static SemanticNodeProcessor getCurrentMapJoin() {
     return new CurrentMapJoin();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MergeJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MergeJoinProc.java
index 327e16d..4284396 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MergeJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MergeJoinProc.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator;
 import org.apache.hadoop.hive.ql.exec.DummyStoreOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.GenTezProcContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -35,7 +35,7 @@ import org.apache.hadoop.hive.ql.plan.TezEdgeProperty;
 import org.apache.hadoop.hive.ql.plan.TezWork;
 import org.apache.hadoop.hive.ql.plan.TezWork.VertexType;
 
-public class MergeJoinProc implements NodeProcessor {
+public class MergeJoinProc implements SemanticNodeProcessor {
   @Override
   public Object
       process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java
index b8f8494..a5972d0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java
@@ -35,12 +35,12 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -60,12 +60,12 @@ public class NonBlockingOpDeDupProc extends Transform {
     // 1. We apply the transformation
     String SEL = SelectOperator.getOperatorName();
     String FIL = FilterOperator.getOperatorName();
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", SEL + "%" + SEL + "%"), new SelectDedup(pctx));
     opRules.put(new RuleRegExp("R2", FIL + "%" + FIL + "%"), new FilterDedup());
 
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getTopOps().values());
@@ -73,7 +73,7 @@ public class NonBlockingOpDeDupProc extends Transform {
     return pctx;
   }
 
-  private class SelectDedup implements NodeProcessor {
+  private class SelectDedup implements SemanticNodeProcessor {
 
     private ParseContext pctx;
 
@@ -211,7 +211,7 @@ public class NonBlockingOpDeDupProc extends Transform {
     }
   }
 
-  private class FilterDedup implements NodeProcessor {
+  private class FilterDedup implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PartitionColumnsSeparator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PartitionColumnsSeparator.java
index 5fc29d2..49d915c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PartitionColumnsSeparator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PartitionColumnsSeparator.java
@@ -32,14 +32,14 @@ import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.ForwardWalker;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.PreOrderOnceWalker;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TypeRule;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -75,11 +75,11 @@ public class PartitionColumnsSeparator extends Transform {
   @Override
   public ParseContext transform(ParseContext pctx) throws SemanticException {
     // 1. Trigger transformation
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", FilterOperator.getOperatorName() + "%"), new StructInTransformer());
 
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
-    GraphWalker ogw = new ForwardWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+    SemanticGraphWalker ogw = new ForwardWalker(disp);
 
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getTopOps().values());
@@ -87,7 +87,7 @@ public class PartitionColumnsSeparator extends Transform {
     return pctx;
   }
 
-  private class StructInTransformer implements NodeProcessor {
+  private class StructInTransformer implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -117,13 +117,13 @@ public class PartitionColumnsSeparator extends Transform {
     }
 
     private ExprNodeDesc generateInClauses(ExprNodeDesc predicate) throws SemanticException {
-      Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> exprRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       exprRules.put(new TypeRule(ExprNodeGenericFuncDesc.class), new StructInExprProcessor());
 
       // The dispatcher fires the processor corresponding to the closest matching
       // rule and passes the context along
-      Dispatcher disp = new DefaultRuleDispatcher(null, exprRules, null);
-      GraphWalker egw = new PreOrderOnceWalker(disp);
+      SemanticDispatcher disp = new DefaultRuleDispatcher(null, exprRules, null);
+      SemanticGraphWalker egw = new PreOrderOnceWalker(disp);
 
       List<Node> startNodes = new ArrayList<Node>();
       startNodes.add(predicate);
@@ -147,7 +147,7 @@ public class PartitionColumnsSeparator extends Transform {
    * part of the given query. Once the partitions are pruned, the partition condition
    * remover is expected to remove the redundant predicates from the plan.
    */
-  private class StructInExprProcessor implements NodeProcessor {
+  private class StructInExprProcessor implements SemanticNodeProcessor {
 
     /** TableInfo is populated in PASS 1 of process(). It contains the information required
      * to generate an IN clause of the following format:
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PointLookupOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PointLookupOptimizer.java
index 3eb224d..f09c8af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PointLookupOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PointLookupOptimizer.java
@@ -31,14 +31,14 @@ import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.ForwardWalker;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.PreOrderOnceWalker;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TypeRule;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -84,11 +84,11 @@ public class PointLookupOptimizer extends Transform {
   @Override
   public ParseContext transform(ParseContext pctx) throws SemanticException {
     // 1. Trigger transformation
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", FilterOperator.getOperatorName() + "%"), new FilterTransformer());
 
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
-    GraphWalker ogw = new ForwardWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+    SemanticGraphWalker ogw = new ForwardWalker(disp);
 
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getTopOps().values());
@@ -96,7 +96,7 @@ public class PointLookupOptimizer extends Transform {
     return pctx;
   }
 
-  private class FilterTransformer implements NodeProcessor {
+  private class FilterTransformer implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -118,13 +118,13 @@ public class PointLookupOptimizer extends Transform {
     }
 
     private ExprNodeDesc generateInClause(ExprNodeDesc predicate) throws SemanticException {
-      Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> exprRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       exprRules.put(new TypeRule(ExprNodeGenericFuncDesc.class), new OrExprProcessor());
 
       // The dispatcher fires the processor corresponding to the closest matching
       // rule and passes the context along
-      Dispatcher disp = new DefaultRuleDispatcher(null, exprRules, null);
-      GraphWalker egw = new PreOrderOnceWalker(disp);
+      SemanticDispatcher disp = new DefaultRuleDispatcher(null, exprRules, null);
+      SemanticGraphWalker egw = new PreOrderOnceWalker(disp);
 
       List<Node> startNodes = new ArrayList<Node>();
       startNodes.add(predicate);
@@ -135,7 +135,7 @@ public class PointLookupOptimizer extends Transform {
     }
   }
 
-  private class OrExprProcessor implements NodeProcessor {
+  private class OrExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
index 3e81c2b..6962dcb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
@@ -22,7 +22,7 @@ import java.util.Stack;
 
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -44,7 +44,7 @@ public abstract class PrunerExpressionOperatorFactory {
    * expr is a candidate else it is not a candidate but its children could be
    * final candidates.
    */
-  public static class GenericFuncExprProcessor implements NodeProcessor {
+  public static class GenericFuncExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -108,7 +108,7 @@ public abstract class PrunerExpressionOperatorFactory {
    * FieldExprProcessor.
    *
    */
-  public static class FieldExprProcessor implements NodeProcessor {
+  public static class FieldExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -145,7 +145,7 @@ public abstract class PrunerExpressionOperatorFactory {
   /**
    * Processor for column expressions.
    */
-  public static abstract class ColumnExprProcessor implements NodeProcessor {
+  public static abstract class ColumnExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -166,7 +166,7 @@ public abstract class PrunerExpressionOperatorFactory {
      * @return
      */
     protected abstract ExprNodeDesc processColumnDesc(NodeProcessorCtx procCtx,
-        ExprNodeColumnDesc cd);
+                                                      ExprNodeColumnDesc cd);
 
   }
 
@@ -174,7 +174,7 @@ public abstract class PrunerExpressionOperatorFactory {
    * Processor for constants and null expressions. For such expressions the
    * processor simply clones the exprNodeDesc and returns it.
    */
-  public static class DefaultExprProcessor implements NodeProcessor {
+  public static class DefaultExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -191,7 +191,7 @@ public abstract class PrunerExpressionOperatorFactory {
    * Instantiate default expression processor.
    * @return
    */
-  public static final NodeProcessor getDefaultExprProcessor() {
+  public static final SemanticNodeProcessor getDefaultExprProcessor() {
     return new DefaultExprProcessor();
   }
 
@@ -200,7 +200,7 @@ public abstract class PrunerExpressionOperatorFactory {
    *
    * @return
    */
-  public static final NodeProcessor getGenericFuncProcessor() {
+  public static final SemanticNodeProcessor getGenericFuncProcessor() {
     return new GenericFuncExprProcessor();
   }
 
@@ -209,7 +209,7 @@ public abstract class PrunerExpressionOperatorFactory {
    *
    * @return
    */
-  public static final NodeProcessor getFieldProcessor() {
+  public static final SemanticNodeProcessor getFieldProcessor() {
     return new FieldExprProcessor();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerOperatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerOperatorFactory.java
index b5742c6..e04a2b1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerOperatorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerOperatorFactory.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -48,7 +48,7 @@ public abstract class PrunerOperatorFactory {
    * Determines the partition pruner for the filter. This is called only when
    * the filter follows a table scan operator.
    */
-  public static abstract class FilterPruner implements NodeProcessor {
+  public static abstract class FilterPruner implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -101,7 +101,7 @@ public abstract class PrunerOperatorFactory {
      * @throws UDFArgumentException
      */
     protected abstract void generatePredicate(NodeProcessorCtx procCtx, FilterOperator fop,
-        TableScanOperator top) throws SemanticException;
+                                              TableScanOperator top) throws SemanticException;
     /**
      * Add pruning predicate.
      *
@@ -172,7 +172,7 @@ public abstract class PrunerOperatorFactory {
   /**
    * Default processor which just merges its children.
    */
-  public static class DefaultPruner implements NodeProcessor {
+  public static class DefaultPruner implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -189,7 +189,7 @@ public abstract class PrunerOperatorFactory {
    *
    * @return
    */
-  public final static NodeProcessor getDefaultProc() {
+  public final static SemanticNodeProcessor getDefaultProc() {
     return new DefaultPruner();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerUtils.java
index 8cfea50..6d233ff 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerUtils.java
@@ -27,14 +27,13 @@ import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleExactMatch;
-import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TypeRule;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -62,8 +61,8 @@ public final class PrunerUtils {
    * @throws SemanticException
    */
   public static void walkOperatorTree(ParseContext pctx, NodeProcessorCtx opWalkerCtx,
-      NodeProcessor filterProc, NodeProcessor defaultProc) throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+                                      SemanticNodeProcessor filterProc, SemanticNodeProcessor defaultProc) throws SemanticException {
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
     // Build regular expression for operator rule.
     // "(TS%FIL%)|(TS%FIL%FIL%)"
@@ -75,8 +74,8 @@ public final class PrunerUtils {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(defaultProc, opRules, opWalkerCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(defaultProc, opRules, opWalkerCtx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -97,21 +96,21 @@ public final class PrunerUtils {
    * @throws SemanticException
    */
   public static Map<Node, Object> walkExprTree(ExprNodeDesc pred, NodeProcessorCtx ctx,
-      NodeProcessor colProc, NodeProcessor fieldProc, NodeProcessor genFuncProc,
-      NodeProcessor defProc)
+                                               SemanticNodeProcessor colProc, SemanticNodeProcessor fieldProc, SemanticNodeProcessor genFuncProc,
+                                               SemanticNodeProcessor defProc)
       throws SemanticException {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack. The dispatcher
     // generates the plan from the operator tree
-    Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> exprRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     exprRules.put(new TypeRule(ExprNodeColumnDesc.class) , colProc);
     exprRules.put(new TypeRule(ExprNodeFieldDesc.class), fieldProc);
     exprRules.put(new TypeRule(ExprNodeGenericFuncDesc.class), genFuncProc);
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(defProc, exprRules, ctx);
-    GraphWalker egw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(defProc, exprRules, ctx);
+    SemanticGraphWalker egw = new DefaultGraphWalker(disp);
 
     List<Node> startNodes = new ArrayList<Node>();
     startNodes.add(pred);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
index b591b73..bd6c418 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.GenTezProcContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -64,7 +64,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Sets;
 
-public class ReduceSinkMapJoinProc implements NodeProcessor {
+public class ReduceSinkMapJoinProc implements SemanticNodeProcessor {
 
   private final static Logger LOG = LoggerFactory.getLogger(ReduceSinkMapJoinProc.class.getName());
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RedundantDynamicPruningConditionsRemoval.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RedundantDynamicPruningConditionsRemoval.java
index 4a60158..f50afbc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RedundantDynamicPruningConditionsRemoval.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RedundantDynamicPruningConditionsRemoval.java
@@ -30,12 +30,12 @@ import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -78,12 +78,12 @@ public class RedundantDynamicPruningConditionsRemoval extends Transform {
   public ParseContext transform(ParseContext pctx) throws SemanticException {
     // Make sure semijoin is not enabled. If it is, then do not remove the dynamic partition pruning predicates.
     if (!pctx.getConf().getBoolVar(HiveConf.ConfVars.TEZ_DYNAMIC_SEMIJOIN_REDUCTION)) {
-      Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       opRules.put(new RuleRegExp("R1", TableScanOperator.getOperatorName() + "%" +
               FilterOperator.getOperatorName() + "%"), new FilterTransformer());
 
-      Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
-      GraphWalker ogw = new DefaultGraphWalker(disp);
+      SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+      SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
       List<Node> topNodes = new ArrayList<Node>();
       topNodes.addAll(pctx.getTopOps().values());
@@ -92,7 +92,7 @@ public class RedundantDynamicPruningConditionsRemoval extends Transform {
     return pctx;
   }
 
-  private class FilterTransformer implements NodeProcessor {
+  private class FilterTransformer implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java
index c0f5699..52614d3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java
@@ -25,7 +25,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.GenTezUtils;
 import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext;
@@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
  * If we expect the number of keys for dynamic pruning to be too large we
  * disable it.
  */
-public class RemoveDynamicPruningBySize implements NodeProcessor {
+public class RemoveDynamicPruningBySize implements SemanticNodeProcessor {
 
   static final private Logger LOG = LoggerFactory.getLogger(RemoveDynamicPruningBySize.class.getName());
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SamplePruner.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SamplePruner.java
index bb96814..b3aac5c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SamplePruner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SamplePruner.java
@@ -37,12 +37,12 @@ import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -103,7 +103,7 @@ public class SamplePruner extends Transform {
     // create a the context for walking operators
     SamplePrunerCtx samplePrunerCtx = new SamplePrunerCtx(pctx.getOpToSamplePruner());
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1",
       "(" + TableScanOperator.getOperatorName() + "%"
       + FilterOperator.getOperatorName() + "%"
@@ -113,9 +113,9 @@ public class SamplePruner extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules,
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules,
         samplePrunerCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -128,7 +128,7 @@ public class SamplePruner extends Transform {
    * FilterPPR filter processor.
    *
    */
-  public static class FilterPPR implements NodeProcessor {
+  public static class FilterPPR implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -150,7 +150,7 @@ public class SamplePruner extends Transform {
     }
   }
 
-  public static NodeProcessor getFilterProc() {
+  public static SemanticNodeProcessor getFilterProc() {
     return new FilterPPR();
   }
 
@@ -158,7 +158,7 @@ public class SamplePruner extends Transform {
    * DefaultPPR default processor which does nothing.
    *
    */
-  public static class DefaultPPR implements NodeProcessor {
+  public static class DefaultPPR implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -168,7 +168,7 @@ public class SamplePruner extends Transform {
     }
   }
 
-  public static NodeProcessor getDefaultProc() {
+  public static SemanticNodeProcessor getDefaultProc() {
     return new DefaultPPR();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetHashGroupByMinReduction.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetHashGroupByMinReduction.java
index df3707a..1a7d1f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetHashGroupByMinReduction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetHashGroupByMinReduction.java
@@ -24,7 +24,7 @@ import java.util.Stack;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ColStatistics;
@@ -41,14 +41,14 @@ import org.slf4j.LoggerFactory;
  * SetHashGroupByMinReduction determines the min reduction to perform
  * a hash aggregation for a group by.
  */
-public class SetHashGroupByMinReduction implements NodeProcessor {
+public class SetHashGroupByMinReduction implements SemanticNodeProcessor {
 
   private static final Logger LOG = LoggerFactory.getLogger(SetHashGroupByMinReduction.class.getName());
 
   @SuppressWarnings("unchecked")
   @Override
   public Object process(Node nd, Stack<Node> stack,
-      NodeProcessorCtx procContext, Object... nodeOutputs)
+                        NodeProcessorCtx procContext, Object... nodeOutputs)
       throws SemanticException {
 
     GroupByOperator groupByOperator = (GroupByOperator) nd;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java
index 3229171..c21dd19 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java
@@ -29,9 +29,8 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.optimizer.stats.annotation.StatsRulesProcFactory;
 import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc.ExprNodeDescEqualityWrapper;
@@ -47,14 +46,14 @@ import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.FIXED;
  * SetReducerParallelism determines how many reducers should
  * be run for a given reduce sink.
  */
-public class SetReducerParallelism implements NodeProcessor {
+public class SetReducerParallelism implements SemanticNodeProcessor {
 
   private static final Logger LOG = LoggerFactory.getLogger(SetReducerParallelism.class.getName());
 
   @SuppressWarnings("unchecked")
   @Override
   public Object process(Node nd, Stack<Node> stack,
-      NodeProcessorCtx procContext, Object... nodeOutputs)
+                        NodeProcessorCtx procContext, Object... nodeOutputs)
       throws SemanticException {
 
     OptimizeTezProcContext context = (OptimizeTezProcContext) procContext;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchAggregation.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchAggregation.java
index 916151f..d14bafb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchAggregation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchAggregation.java
@@ -33,12 +33,12 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -65,12 +65,12 @@ public class SimpleFetchAggregation extends Transform {
     String SEL = SelectOperator.getOperatorName() + "%";
     String FS = FileSinkOperator.getOperatorName() + "%";
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", GBY + RS + GBY + SEL + FS), new SingleGBYProcessor(pctx));
     opRules.put(new RuleRegExp("R2", GBY + RS + GBY + FS), new SingleGBYProcessor(pctx));
 
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getTopOps().values());
@@ -78,7 +78,7 @@ public class SimpleFetchAggregation extends Transform {
     return pctx;
   }
 
-  static class SingleGBYProcessor implements NodeProcessor {
+  static class SingleGBYProcessor implements SemanticNodeProcessor {
 
     private ParseContext pctx;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
index 40aaafe..8cbc83e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
@@ -38,12 +38,12 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -79,7 +79,7 @@ public class SkewJoinOptimizer extends Transform {
 
   private static final Logger LOG = LoggerFactory.getLogger(SkewJoinOptimizer.class.getName());
 
-  public static class SkewJoinProc implements NodeProcessor {
+  public static class SkewJoinProc implements SemanticNodeProcessor {
     private ParseContext parseContext;
 
     public SkewJoinProc(ParseContext parseContext) {
@@ -661,15 +661,15 @@ public class SkewJoinOptimizer extends Transform {
    */
   @Override
   public ParseContext transform(ParseContext pctx) throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
     opRules.put(new RuleRegExp("R1", "TS%.*RS%JOIN%"), getSkewJoinProc(pctx));
     SkewJoinOptProcCtx skewJoinOptProcCtx = new SkewJoinOptProcCtx(pctx);
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(
+    SemanticDispatcher disp = new DefaultRuleDispatcher(
       null, opRules, skewJoinOptProcCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     List<Node> topNodes = new ArrayList<Node>();
@@ -678,7 +678,7 @@ public class SkewJoinOptimizer extends Transform {
     return pctx;
   }
 
-  private NodeProcessor getSkewJoinProc(ParseContext parseContext) {
+  private SemanticNodeProcessor getSkewJoinProc(ParseContext parseContext) {
     return new SkewJoinProc(parseContext);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
index c6798e9..c1d659e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
@@ -48,12 +48,12 @@ import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.RecordIdentifier;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
@@ -99,14 +99,14 @@ public class SortedDynPartitionOptimizer extends Transform {
 
     // create a walker which walks the tree in a DFS manner while maintaining the
     // operator stack. The dispatcher generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
     String FS = FileSinkOperator.getOperatorName() + "%";
 
     opRules.put(new RuleRegExp("Sorted Dynamic Partition", FS), getSortDynPartProc(pCtx));
 
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pCtx.getTopOps().values());
@@ -115,11 +115,11 @@ public class SortedDynPartitionOptimizer extends Transform {
     return pCtx;
   }
 
-  private NodeProcessor getSortDynPartProc(ParseContext pCtx) {
+  private SemanticNodeProcessor getSortDynPartProc(ParseContext pCtx) {
     return new SortedDynamicPartitionProc(pCtx);
   }
 
-  class SortedDynamicPartitionProc implements NodeProcessor {
+  class SortedDynamicPartitionProc implements SemanticNodeProcessor {
 
     private final Logger LOG = LoggerFactory.getLogger(SortedDynPartitionOptimizer.class);
     protected ParseContext parseCtx;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
index 384c857..d458ebb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
@@ -34,12 +34,12 @@ import org.apache.hadoop.hive.ql.exec.Utilities.ReduceField;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -98,14 +98,14 @@ public class SortedDynPartitionTimeGranularityOptimizer extends Transform {
   public ParseContext transform(ParseContext pCtx) throws SemanticException {
     // create a walker which walks the tree in a DFS manner while maintaining the
     // operator stack. The dispatcher generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
     String FS = FileSinkOperator.getOperatorName() + "%";
 
     opRules.put(new RuleRegExp("Sorted Dynamic Partition Time Granularity", FS), getSortDynPartProc(pCtx));
 
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pCtx.getTopOps().values());
@@ -114,11 +114,11 @@ public class SortedDynPartitionTimeGranularityOptimizer extends Transform {
     return pCtx;
   }
 
-  private NodeProcessor getSortDynPartProc(ParseContext pCtx) {
+  private SemanticNodeProcessor getSortDynPartProc(ParseContext pCtx) {
     return new SortedDynamicPartitionProc(pCtx);
   }
 
-  class SortedDynamicPartitionProc implements NodeProcessor {
+  class SortedDynamicPartitionProc implements SemanticNodeProcessor {
 
     private final Logger LOG = LoggerFactory.getLogger(SortedDynPartitionTimeGranularityOptimizer.class);
     protected ParseContext parseCtx;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java
index ffa74af..f18f15a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java
@@ -32,12 +32,12 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -58,14 +58,14 @@ public class SortedMergeBucketMapJoinOptimizer extends Transform {
 
     // Go through all joins - it should only contain selects and filters between
     // tablescan and join operators.
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", JoinOperator.getOperatorName() + "%"),
       getCheckCandidateJoin());
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, smbJoinContext);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, smbJoinContext);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -85,7 +85,7 @@ public class SortedMergeBucketMapJoinOptimizer extends Transform {
     // dictates which operator is allowed
     getListOfRejectedJoins(pctx, smbJoinContext);
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     // go through all map joins and find out all which have enabled bucket map
     // join.
     opRules.put(new RuleRegExp("R1", MapJoinOperator.getOperatorName() + "%"),
@@ -100,8 +100,8 @@ public class SortedMergeBucketMapJoinOptimizer extends Transform {
         getSortedMergeJoinProc(pctx));
     }
 
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, smbJoinContext);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, smbJoinContext);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -111,19 +111,19 @@ public class SortedMergeBucketMapJoinOptimizer extends Transform {
     return pctx;
   }
 
-  private NodeProcessor getSortedMergeBucketMapjoinProc(ParseContext pctx) {
+  private SemanticNodeProcessor getSortedMergeBucketMapjoinProc(ParseContext pctx) {
     return new SortedMergeBucketMapjoinProc(pctx);
   }
 
-  private NodeProcessor getSortedMergeJoinProc(ParseContext pctx) {
+  private SemanticNodeProcessor getSortedMergeJoinProc(ParseContext pctx) {
     return new SortedMergeJoinProc(pctx);
   }
 
-  private NodeProcessor getDefaultProc() {
-    return new NodeProcessor() {
+  private SemanticNodeProcessor getDefaultProc() {
+    return new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack,
-          NodeProcessorCtx procCtx, Object... nodeOutputs)
+                            NodeProcessorCtx procCtx, Object... nodeOutputs)
           throws SemanticException {
         return null;
       }
@@ -132,8 +132,8 @@ public class SortedMergeBucketMapJoinOptimizer extends Transform {
 
   // check if the join operator encountered is a candidate for being converted
   // to a sort-merge join
-  private NodeProcessor getCheckCandidateJoin() {
-    return new NodeProcessor() {
+  private SemanticNodeProcessor getCheckCandidateJoin() {
+    return new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapjoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapjoinProc.java
index 9e33a94..cbfb749 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapjoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapjoinProc.java
@@ -25,12 +25,12 @@ import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 
-public class SortedMergeBucketMapjoinProc extends AbstractSMBJoinProc implements NodeProcessor {
+public class SortedMergeBucketMapjoinProc extends AbstractSMBJoinProc implements SemanticNodeProcessor {
   public SortedMergeBucketMapjoinProc(ParseContext pctx) {
     super(pctx);
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeJoinProc.java
index 0489ffc..4110a6f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeJoinProc.java
@@ -22,12 +22,12 @@ import java.util.Stack;
 
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 
-public class SortedMergeJoinProc extends AbstractSMBJoinProc implements NodeProcessor {
+public class SortedMergeJoinProc extends AbstractSMBJoinProc implements SemanticNodeProcessor {
 
   public SortedMergeJoinProc(ParseContext pctx) {
     super(pctx);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruning.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruning.java
index a117a65..de2a58a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruning.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruning.java
@@ -25,7 +25,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -40,7 +40,7 @@ import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
  *
  * Cloned from RemoveDynamicPruningBySize
  */
-public class SparkRemoveDynamicPruning implements NodeProcessor {
+public class SparkRemoveDynamicPruning implements SemanticNodeProcessor {
 
   static final private Logger LOG =
       LoggerFactory.getLogger(SparkRemoveDynamicPruning.class.getName());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
index 54de1d7..784dd31 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
@@ -40,12 +40,12 @@ import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lockmgr.LockException;
 import org.apache.hadoop.hive.ql.metadata.Hive;
@@ -121,15 +121,15 @@ public class StatsOptimizer extends Transform {
     String SEL = SelectOperator.getOperatorName() + "%";
     String FS = FileSinkOperator.getOperatorName() + "%";
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", TS + SEL + GBY + RS + GBY + SEL + FS),
         new MetaDataProcessor(pctx));
     opRules.put(new RuleRegExp("R2", TS + SEL + GBY + RS + GBY + FS),
             new MetaDataProcessor(pctx));
 
     NodeProcessorCtx soProcCtx = new StatsOptimizerProcContext();
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, soProcCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, soProcCtx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getTopOps().values());
@@ -141,7 +141,7 @@ public class StatsOptimizer extends Transform {
     boolean stopProcess = false;
   }
 
-  private static class MetaDataProcessor implements NodeProcessor {
+  private static class MetaDataProcessor implements SemanticNodeProcessor {
 
     private final ParseContext pctx;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
index a904182..387d34c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
@@ -32,13 +32,13 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -96,7 +96,7 @@ class TablePropertyEnrichmentOptimizer extends Transform {
     return originalTableParameters;
   }
 
-  private static class Processor implements NodeProcessor {
+  private static class Processor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
@@ -150,17 +150,17 @@ class TablePropertyEnrichmentOptimizer extends Transform {
 
     LOG.info("TablePropertyEnrichmentOptimizer::transform().");
 
-    Map<Rule, NodeProcessor> opRules = Maps.newLinkedHashMap();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = Maps.newLinkedHashMap();
     opRules.put(new RuleRegExp("R1", TableScanOperator.getOperatorName() + "%"),
         new Processor());
 
     WalkerCtx context = new WalkerCtx(pctx.getConf());
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, context);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, context);
 
     List<Node> topNodes = Lists.newArrayList();
     topNodes.addAll(pctx.getTopOps().values());
 
-    GraphWalker walker = new PreOrderWalker(disp);
+    SemanticGraphWalker walker = new PreOrderWalker(disp);
     walker.startWalking(topNodes, null);
 
     LOG.info("TablePropertyEnrichmentOptimizer::transform() complete!");
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java
index 7e6a361..f41b844 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java
@@ -31,13 +31,13 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.ForwardWalker;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -65,12 +65,12 @@ public class HiveOpConverterPostProc extends Transform {
     this.aliasToOpInfo = new HashMap<String, Operator<? extends OperatorDesc>>();
 
     // 2. Trigger transformation
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", JoinOperator.getOperatorName() + "%"), new JoinAnnotate());
     opRules.put(new RuleRegExp("R2", TableScanOperator.getOperatorName() + "%"), new TableScanAnnotate());
 
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
-    GraphWalker ogw = new ForwardWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+    SemanticGraphWalker ogw = new ForwardWalker(disp);
 
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getTopOps().values());
@@ -78,7 +78,7 @@ public class HiveOpConverterPostProc extends Transform {
     return pctx;
   }
 
-  private class JoinAnnotate implements NodeProcessor {
+  private class JoinAnnotate implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -126,7 +126,7 @@ public class HiveOpConverterPostProc extends Transform {
   }
 
 
-  private class TableScanAnnotate implements NodeProcessor {
+  private class TableScanAnnotate implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java
index 9377fd2..c33f39d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java
@@ -50,12 +50,12 @@ import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.optimizer.MapJoinProcessor;
@@ -218,12 +218,12 @@ public class CorrelationOptimizer extends Transform {
 
     // detect correlations
     CorrelationNodeProcCtx corrCtx = new CorrelationNodeProcCtx(pCtx);
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", ReduceSinkOperator.getOperatorName() + "%"),
         new CorrelationNodeProc());
 
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, corrCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, corrCtx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topOp nodes
     List<Node> topNodes = new ArrayList<Node>();
@@ -249,7 +249,7 @@ public class CorrelationOptimizer extends Transform {
     return pCtx;
   }
 
-  private class CorrelationNodeProc implements NodeProcessor {
+  private class CorrelationNodeProc implements SemanticNodeProcessor {
 
     private void analyzeReduceSinkOperatorsOfJoinOperator(JoinCondDesc[] joinConds,
         List<Operator<? extends OperatorDesc>> rsOps, Operator<? extends OperatorDesc> curentRsOp,
@@ -630,11 +630,11 @@ public class CorrelationOptimizer extends Transform {
     }
   }
 
-  private NodeProcessor getDefaultProc() {
-    return new NodeProcessor() {
+  private SemanticNodeProcessor getDefaultProc() {
+    return new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack,
-          NodeProcessorCtx ctx, Object... nodeOutputs) throws SemanticException {
+                            NodeProcessorCtx ctx, Object... nodeOutputs) throws SemanticException {
         Operator<? extends OperatorDesc> op = (Operator<? extends OperatorDesc>) nd;
         LOG.info("Walk to operator " + op.getIdentifier() + " "
             + op.getName() + ". No actual work to do");
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java
index b25bcf0..c4c70d6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java
@@ -35,12 +35,12 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -80,7 +80,7 @@ public class ReduceSinkDeDuplication extends Transform {
 
     // If multiple rules can be matched with same cost, last rule will be choosen as a processor
     // see DefaultRuleDispatcher#dispatch()
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", RS + "%.*%" + RS + "%"),
         ReduceSinkDeduplicateProcFactory.getReducerReducerProc());
     opRules.put(new RuleRegExp("R2", RS + "%" + GBY + "%.*%" + RS + "%"),
@@ -93,9 +93,9 @@ public class ReduceSinkDeDuplication extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(ReduceSinkDeduplicateProcFactory
+    SemanticDispatcher disp = new DefaultRuleDispatcher(ReduceSinkDeduplicateProcFactory
         .getDefaultProc(), opRules, cppCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -113,19 +113,19 @@ public class ReduceSinkDeDuplication extends Transform {
 
   static class ReduceSinkDeduplicateProcFactory {
 
-    public static NodeProcessor getReducerReducerProc() {
+    public static SemanticNodeProcessor getReducerReducerProc() {
       return new ReducerReducerProc();
     }
 
-    public static NodeProcessor getGroupbyReducerProc() {
+    public static SemanticNodeProcessor getGroupbyReducerProc() {
       return new GroupbyReducerProc();
     }
 
-    public static NodeProcessor getJoinReducerProc() {
+    public static SemanticNodeProcessor getJoinReducerProc() {
       return new JoinReducerProc();
     }
 
-    public static NodeProcessor getDefaultProc() {
+    public static SemanticNodeProcessor getDefaultProc() {
       return new DefaultProc();
     }
   }
@@ -133,7 +133,7 @@ public class ReduceSinkDeDuplication extends Transform {
   /*
    * do nothing.
    */
-  static class DefaultProc implements NodeProcessor {
+  static class DefaultProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -141,7 +141,7 @@ public class ReduceSinkDeDuplication extends Transform {
     }
   }
 
-  public abstract static class AbsctractReducerReducerProc implements NodeProcessor {
+  public abstract static class AbsctractReducerReducerProc implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkJoinDeDuplication.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkJoinDeDuplication.java
index 0e8c2fb..d3e3c4d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkJoinDeDuplication.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkJoinDeDuplication.java
@@ -29,13 +29,13 @@ import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.ForwardWalker;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -72,13 +72,13 @@ public class ReduceSinkJoinDeDuplication extends Transform {
 
     ReduceSinkJoinDeDuplicateProcCtx cppCtx = new ReduceSinkJoinDeDuplicateProcCtx(pGraphContext);
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", ReduceSinkOperator.getOperatorName() + "%"),
         ReduceSinkJoinDeDuplicateProcFactory.getReducerMapJoinProc());
 
-    Dispatcher disp = new DefaultRuleDispatcher(
+    SemanticDispatcher disp = new DefaultRuleDispatcher(
         ReduceSinkJoinDeDuplicateProcFactory.getDefaultProc(), opRules, cppCtx);
-    GraphWalker ogw = new ForwardWalker(disp);
+    SemanticGraphWalker ogw = new ForwardWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -96,11 +96,11 @@ public class ReduceSinkJoinDeDuplication extends Transform {
 
   static class ReduceSinkJoinDeDuplicateProcFactory {
 
-    public static NodeProcessor getReducerMapJoinProc() {
+    public static SemanticNodeProcessor getReducerMapJoinProc() {
       return new ReducerProc();
     }
 
-    public static NodeProcessor getDefaultProc() {
+    public static SemanticNodeProcessor getDefaultProc() {
       return new DefaultProc();
     }
   }
@@ -108,7 +108,7 @@ public class ReduceSinkJoinDeDuplication extends Transform {
   /*
    * do nothing.
    */
-  static class DefaultProc implements NodeProcessor {
+  static class DefaultProc implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -116,7 +116,7 @@ public class ReduceSinkJoinDeDuplication extends Transform {
     }
   }
 
-  static class ReducerProc implements NodeProcessor {
+  static class ReducerProc implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java
index 07e2aef..3501cdd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/ExprProcFactory.java
@@ -42,12 +42,12 @@ import org.apache.hadoop.hive.ql.hooks.LineageInfo.Predicate;
 import org.apache.hadoop.hive.ql.hooks.LineageInfo.TableAliasInfo;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -70,7 +70,7 @@ public class ExprProcFactory {
   private static final String exprNodeFieldDescRegExp = ExprNodeFieldDesc.class.getName() + "%";
   private static final String exprNodeGenFuncDescRegExp = ExprNodeGenericFuncDesc.class.getName() + "%";
 
-  private static final Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
+  private static final Map<SemanticRule, SemanticNodeProcessor> exprRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
   static {
     exprRules.put(new RuleRegExp("R1", exprNodeColDescRegExp), getColumnProcessor());
@@ -81,7 +81,7 @@ public class ExprProcFactory {
   /**
    * Processor for column expressions.
    */
-  public static class ColumnExprProcessor implements NodeProcessor {
+  public static class ColumnExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -113,7 +113,7 @@ public class ExprProcFactory {
   /**
    * Processor for any function or field expression.
    */
-  public static class GenericExprProcessor implements NodeProcessor {
+  public static class GenericExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -150,7 +150,7 @@ public class ExprProcFactory {
    * Processor for constants and null expressions. For such expressions the
    * processor simply returns a null dependency vector.
    */
-  public static class DefaultExprProcessor implements NodeProcessor {
+  public static class DefaultExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -166,19 +166,19 @@ public class ExprProcFactory {
     }
   }
 
-  public static NodeProcessor getDefaultExprProcessor() {
+  public static SemanticNodeProcessor getDefaultExprProcessor() {
     return new DefaultExprProcessor();
   }
 
-  public static NodeProcessor getGenericFuncProcessor() {
+  public static SemanticNodeProcessor getGenericFuncProcessor() {
     return new GenericExprProcessor();
   }
 
-  public static NodeProcessor getFieldProcessor() {
+  public static SemanticNodeProcessor getFieldProcessor() {
     return new GenericExprProcessor();
   }
 
-  public static NodeProcessor getColumnProcessor() {
+  public static SemanticNodeProcessor getColumnProcessor() {
     return new ColumnExprProcessor();
   }
 
@@ -315,9 +315,9 @@ public class ExprProcFactory {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultExprProcessor(),
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultExprProcessor(),
         exprRules, exprCtx);
-    GraphWalker egw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker egw = new DefaultGraphWalker(disp);
 
     List<Node> startNodes = Collections.singletonList((Node)expr);
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java
index e131e3d..3a4522f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/Generator.java
@@ -36,12 +36,12 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDTFOperator;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.optimizer.lineage.LineageCtx.Index;
@@ -93,7 +93,7 @@ public class Generator extends Transform {
     // Create the lineage context
     LineageCtx lCtx = new LineageCtx(pctx, index);
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", TableScanOperator.getOperatorName() + "%"),
       OpProcFactory.getTSProc());
     opRules.put(new RuleRegExp("R2", ScriptOperator.getOperatorName() + "%"),
@@ -119,8 +119,8 @@ public class Generator extends Transform {
       OpProcFactory.getFilterProc());
 
     // The dispatcher fires the processor corresponding to the closest matching rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(OpProcFactory.getDefaultProc(), opRules, lCtx);
-    GraphWalker ogw = new LevelOrderWalker(disp, 2);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(OpProcFactory.getDefaultProc(), opRules, lCtx);
+    SemanticGraphWalker ogw = new LevelOrderWalker(disp, 2);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/OpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/OpProcFactory.java
index 32e419a..f8c011a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/OpProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/OpProcFactory.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hive.ql.hooks.LineageInfo.DependencyType;
 import org.apache.hadoop.hive.ql.hooks.LineageInfo.Predicate;
 import org.apache.hadoop.hive.ql.hooks.LineageInfo.TableAliasInfo;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.Utils;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
@@ -85,7 +85,7 @@ public class OpProcFactory {
   /**
    * Processor for Script and UDTF Operators.
    */
-  public static class TransformLineage extends DefaultLineage implements NodeProcessor {
+  public static class TransformLineage extends DefaultLineage implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -144,7 +144,7 @@ public class OpProcFactory {
   /**
    * Processor for TableScan Operator. This actually creates the base column mappings.
    */
-  public static class TableScanLineage extends DefaultLineage implements NodeProcessor {
+  public static class TableScanLineage extends DefaultLineage implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -201,7 +201,7 @@ public class OpProcFactory {
   /**
    * Processor for Join Operator.
    */
-  public static class JoinLineage extends DefaultLineage implements NodeProcessor {
+  public static class JoinLineage extends DefaultLineage implements SemanticNodeProcessor {
 
     private final HashMap<Node, Object> outputMap = new HashMap<Node, Object>();
 
@@ -305,7 +305,7 @@ public class OpProcFactory {
   /**
    * Processor for Join Operator.
    */
-  public static class LateralViewJoinLineage extends DefaultLineage implements NodeProcessor {
+  public static class LateralViewJoinLineage extends DefaultLineage implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -350,7 +350,7 @@ public class OpProcFactory {
   /**
    * Processor for Select operator.
    */
-  public static class SelectLineage extends DefaultLineage implements NodeProcessor {
+  public static class SelectLineage extends DefaultLineage implements SemanticNodeProcessor {
 
     private final HashMap<Node, Object> outputMap = new HashMap<Node, Object>();
 
@@ -406,7 +406,7 @@ public class OpProcFactory {
   /**
    * Processor for GroupBy operator.
    */
-  public static class GroupByLineage extends DefaultLineage implements NodeProcessor {
+  public static class GroupByLineage extends DefaultLineage implements SemanticNodeProcessor {
 
     private final HashMap<Node, Object> outputMap = new HashMap<Node, Object>();
 
@@ -531,7 +531,7 @@ public class OpProcFactory {
    * In this case we call mergeDependency as opposed to putDependency
    * in order to account for visits from different parents.
    */
-  public static class UnionLineage extends DefaultLineage implements NodeProcessor {
+  public static class UnionLineage extends DefaultLineage implements SemanticNodeProcessor {
 
     @SuppressWarnings("unchecked")
     @Override
@@ -568,7 +568,7 @@ public class OpProcFactory {
   /**
    * ReduceSink processor.
    */
-  public static class ReduceSinkLineage implements NodeProcessor {
+  public static class ReduceSinkLineage implements SemanticNodeProcessor {
 
     private final HashMap<Node, Object> outputMap = new HashMap<Node, Object>();
 
@@ -639,7 +639,7 @@ public class OpProcFactory {
   /**
    * Filter processor.
    */
-  public static class FilterLineage implements NodeProcessor {
+  public static class FilterLineage implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -680,7 +680,7 @@ public class OpProcFactory {
    * Default processor. This basically passes the input dependencies as such
    * to the output dependencies.
    */
-  public static class DefaultLineage implements NodeProcessor {
+  public static class DefaultLineage implements SemanticNodeProcessor {
 
     @SuppressWarnings("unchecked")
     @Override
@@ -709,43 +709,43 @@ public class OpProcFactory {
     }
   }
 
-  public static NodeProcessor getJoinProc() {
+  public static SemanticNodeProcessor getJoinProc() {
     return new JoinLineage();
   }
 
-  public static NodeProcessor getLateralViewJoinProc() {
+  public static SemanticNodeProcessor getLateralViewJoinProc() {
     return new LateralViewJoinLineage();
   }
 
-  public static NodeProcessor getTSProc() {
+  public static SemanticNodeProcessor getTSProc() {
     return new TableScanLineage();
   }
 
-  public static NodeProcessor getTransformProc() {
+  public static SemanticNodeProcessor getTransformProc() {
     return new TransformLineage();
   }
 
-  public static NodeProcessor getSelProc() {
+  public static SemanticNodeProcessor getSelProc() {
     return new SelectLineage();
   }
 
-  public static NodeProcessor getGroupByProc() {
+  public static SemanticNodeProcessor getGroupByProc() {
     return new GroupByLineage();
   }
 
-  public static NodeProcessor getUnionProc() {
+  public static SemanticNodeProcessor getUnionProc() {
     return new UnionLineage();
   }
 
-  public static NodeProcessor getReduceSinkProc() {
+  public static SemanticNodeProcessor getReduceSinkProc() {
     return new ReduceSinkLineage();
   }
 
-  public static NodeProcessor getDefaultProc() {
+  public static SemanticNodeProcessor getDefaultProc() {
     return new DefaultLineage();
   }
 
-  public static NodeProcessor getFilterProc() {
+  public static SemanticNodeProcessor getFilterProc() {
     return new FilterLineage();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBExprProcCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBExprProcCtx.java
index d680b06..0eaaac1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBExprProcCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBExprProcCtx.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.metadata.Partition;
  * The processor context for list bucketing pruner. This contains the table alias
  * that is being currently processed.
  */
-public class LBExprProcCtx implements NodeProcessorCtx{
+public class LBExprProcCtx implements NodeProcessorCtx {
 
   /**
    * The table alias that is being currently processed.
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBExprProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBExprProcFactory.java
index 1349823..7e51e8e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBExprProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBExprProcFactory.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer.listbucketingpruner;
 import java.util.Map;
 
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.optimizer.PrunerExpressionOperatorFactory;
@@ -104,7 +104,7 @@ public class LBExprProcFactory extends PrunerExpressionOperatorFactory {
    *
    * @return
    */
-  public static NodeProcessor getColumnProcessor() {
+  public static SemanticNodeProcessor getColumnProcessor() {
     return new LBPRColumnExprProcessor();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBPartitionProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBPartitionProcFactory.java
index ef20520..6ea2401 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBPartitionProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBPartitionProcFactory.java
@@ -22,7 +22,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -50,7 +50,7 @@ public class LBPartitionProcFactory extends PrunerOperatorFactory {
 
     @Override
     protected void generatePredicate(NodeProcessorCtx procCtx, FilterOperator fop,
-        TableScanOperator top) throws SemanticException, UDFArgumentException {
+                                     TableScanOperator top) throws SemanticException, UDFArgumentException {
       LBOpPartitionWalkerCtx owc = (LBOpPartitionWalkerCtx) procCtx;
       Table tbl = top.getConf().getTableMetadata();
       if (tbl.isPartitioned()) {
@@ -74,7 +74,7 @@ public class LBPartitionProcFactory extends PrunerOperatorFactory {
 
   }
 
-  public static NodeProcessor getFilterProc() {
+  public static SemanticNodeProcessor getFilterProc() {
     return new LBPRPartitionFilterPruner();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBProcFactory.java
index dac9716..eb0d8aa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/LBProcFactory.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer.listbucketingpruner;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.PrunerOperatorFactory;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -42,7 +42,7 @@ public final class LBProcFactory extends PrunerOperatorFactory {
 
     @Override
     protected void generatePredicate(NodeProcessorCtx procCtx, FilterOperator fop,
-        TableScanOperator top) throws SemanticException, UDFArgumentException {
+                                     TableScanOperator top) throws SemanticException, UDFArgumentException {
       LBOpWalkerCtx owc = (LBOpWalkerCtx) procCtx;
       // Otherwise this is not a sampling predicate and we need to
       ExprNodeDesc predicate = fop.getConf().getPredicate();
@@ -59,7 +59,7 @@ public final class LBProcFactory extends PrunerOperatorFactory {
 
   }
 
-  public static NodeProcessor getFilterProc() {
+  public static SemanticNodeProcessor getFilterProc() {
     return new LBPRFilterPruner();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/AnnotateWithOpTraits.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/AnnotateWithOpTraits.java
index 7ab53b2..b54b15a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/AnnotateWithOpTraits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/AnnotateWithOpTraits.java
@@ -34,12 +34,12 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.PTFOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -57,7 +57,7 @@ public class AnnotateWithOpTraits extends Transform {
 
     // create a walker which walks the tree in a BFS manner while maintaining the
     // operator stack. The dispatcher generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("TS", TableScanOperator.getOperatorName() + "%"),
         OpTraitsRulesProcFactory.getTableScanRule());
     opRules.put(new RuleRegExp("RS", ReduceSinkOperator.getOperatorName() + "%"),
@@ -83,9 +83,9 @@ public class AnnotateWithOpTraits extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(OpTraitsRulesProcFactory.getDefaultRule(), opRules,
+    SemanticDispatcher disp = new DefaultRuleDispatcher(OpTraitsRulesProcFactory.getDefaultRule(), opRules,
         annotateCtx);
-    GraphWalker ogw = new LevelOrderWalker(disp, 0);
+    SemanticGraphWalker ogw = new LevelOrderWalker(disp, 0);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/OpTraitsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/OpTraitsRulesProcFactory.java
index a9d6a5c..c935b74 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/OpTraitsRulesProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/OpTraitsRulesProcFactory.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.PTFOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -69,7 +69,7 @@ import org.apache.hadoop.hive.ql.plan.ptf.PartitionDef;
 
 public class OpTraitsRulesProcFactory {
 
-  public static class DefaultRule implements NodeProcessor {
+  public static class DefaultRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -86,7 +86,7 @@ public class OpTraitsRulesProcFactory {
    * Reduce sink operator is the de-facto operator
    * for determining keyCols (emit keys of a map phase)
    */
-  public static class ReduceSinkRule implements NodeProcessor {
+  public static class ReduceSinkRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -173,7 +173,7 @@ public class OpTraitsRulesProcFactory {
    * Table scan has the table object and pruned partitions that has information
    * such as bucketing, sorting, etc. that is used later for optimization.
    */
-  public static class TableScanRule implements NodeProcessor {
+  public static class TableScanRule implements SemanticNodeProcessor {
 
     public boolean checkBucketedTable(Table tbl, ParseContext pGraphContext,
         PrunedPartitionList prunedParts) throws SemanticException {
@@ -252,7 +252,7 @@ public class OpTraitsRulesProcFactory {
   /*
    * Group-by re-orders the keys emitted hence, the keyCols would change.
    */
-  public static class GroupByRule implements NodeProcessor {
+  public static class GroupByRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -287,7 +287,7 @@ public class OpTraitsRulesProcFactory {
   /*
    * PTFOperator re-orders the keys just like Group By Operator does.
    */
-  public static class PTFRule implements NodeProcessor {
+  public static class PTFRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -328,7 +328,7 @@ public class OpTraitsRulesProcFactory {
     }
   }
 
-  public static class SelectRule implements NodeProcessor {
+  public static class SelectRule implements SemanticNodeProcessor {
 
     // For bucket columns
     // If all the columns match to the parent, put them in the bucket cols
@@ -410,7 +410,7 @@ public class OpTraitsRulesProcFactory {
     }
   }
 
-  public static class JoinRule implements NodeProcessor {
+  public static class JoinRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -487,7 +487,7 @@ public class OpTraitsRulesProcFactory {
    * When we have operators that have multiple parents, it is not clear which
    * parent's traits we need to propagate forward.
    */
-  public static class MultiParentRule implements NodeProcessor {
+  public static class MultiParentRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -521,35 +521,35 @@ public class OpTraitsRulesProcFactory {
     }
   }
 
-  public static NodeProcessor getTableScanRule() {
+  public static SemanticNodeProcessor getTableScanRule() {
     return new TableScanRule();
   }
 
-  public static NodeProcessor getReduceSinkRule() {
+  public static SemanticNodeProcessor getReduceSinkRule() {
     return new ReduceSinkRule();
   }
 
-  public static NodeProcessor getSelectRule() {
+  public static SemanticNodeProcessor getSelectRule() {
     return new SelectRule();
   }
 
-  public static NodeProcessor getDefaultRule() {
+  public static SemanticNodeProcessor getDefaultRule() {
     return new DefaultRule();
   }
 
-  public static NodeProcessor getMultiParentRule() {
+  public static SemanticNodeProcessor getMultiParentRule() {
     return new MultiParentRule();
   }
 
-  public static NodeProcessor getGroupByRule() {
+  public static SemanticNodeProcessor getGroupByRule() {
     return new GroupByRule();
   }
 
-  public static NodeProcessor getPTFRule() {
+  public static SemanticNodeProcessor getPTFRule() {
     return new PTFRule();
   }
 
-  public static NodeProcessor getJoinRule() {
+  public static SemanticNodeProcessor getJoinRule() {
     return new JoinRule();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PartitionConditionRemover.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PartitionConditionRemover.java
index ac93594..37afabe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PartitionConditionRemover.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PartitionConditionRemover.java
@@ -25,16 +25,15 @@ import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -65,7 +64,7 @@ public class PartitionConditionRemover extends Transform {
         new ArrayList<PcrOpWalkerCtx.OpToDeleteInfo>();
     PcrOpWalkerCtx opWalkerCtx = new PcrOpWalkerCtx(pctx, opToRemove);
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1",
       "(" + TableScanOperator.getOperatorName() + "%"
       + FilterOperator.getOperatorName() + "%)|("
@@ -76,9 +75,9 @@ public class PartitionConditionRemover extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(PcrOpProcFactory.getDefaultProc(),
+    SemanticDispatcher disp = new DefaultRuleDispatcher(PcrOpProcFactory.getDefaultProc(),
         opRules, opWalkerCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
index a689ab5..54a3048 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
@@ -29,13 +29,13 @@ import java.util.Stack;
 
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.ExpressionWalker;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -206,7 +206,7 @@ public final class PcrExprProcFactory {
   /**
    * Processor for column expressions.
    */
-  public static class ColumnExprProcessor implements NodeProcessor {
+  public static class ColumnExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -250,7 +250,7 @@ public final class PcrExprProcFactory {
    * If it is deterministic, we evaluate result vector if any of the children
    * is partition column. Otherwise, we pass it as it is.
    */
-  public static class GenericFuncExprProcessor implements NodeProcessor {
+  public static class GenericFuncExprProcessor implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -497,7 +497,7 @@ public final class PcrExprProcFactory {
    * FieldExprProcessor.
    *
    */
-  public static class FieldExprProcessor implements NodeProcessor {
+  public static class FieldExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -525,7 +525,7 @@ public final class PcrExprProcFactory {
    * Processor for constants and null expressions. For such expressions the
    * processor simply returns.
    */
-  public static class DefaultExprProcessor implements NodeProcessor {
+  public static class DefaultExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -538,19 +538,19 @@ public final class PcrExprProcFactory {
     }
   }
 
-  public static NodeProcessor getDefaultExprProcessor() {
+  public static SemanticNodeProcessor getDefaultExprProcessor() {
     return new DefaultExprProcessor();
   }
 
-  public static NodeProcessor getGenericFuncProcessor() {
+  public static SemanticNodeProcessor getGenericFuncProcessor() {
     return new GenericFuncExprProcessor();
   }
 
-  public static NodeProcessor getFieldProcessor() {
+  public static SemanticNodeProcessor getFieldProcessor() {
     return new FieldExprProcessor();
   }
 
-  public static NodeProcessor getColumnProcessor() {
+  public static SemanticNodeProcessor getColumnProcessor() {
     return new ColumnExprProcessor();
   }
 
@@ -574,7 +574,7 @@ public final class PcrExprProcFactory {
     // Create the walker, the rules dispatcher and the context.
     PcrExprProcCtx pprCtx = new PcrExprProcCtx(tabAlias, parts, vcs);
 
-    Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> exprRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     exprRules.put(
         new RuleRegExp("R1", ExprNodeColumnDesc.class.getName() + "%"),
         getColumnProcessor());
@@ -586,9 +586,9 @@ public final class PcrExprProcFactory {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultExprProcessor(),
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultExprProcessor(),
         exprRules, pprCtx);
-    GraphWalker egw = new ExpressionWalker(disp);
+    SemanticGraphWalker egw = new ExpressionWalker(disp);
 
     List<Node> startNodes = new ArrayList<Node>();
     startNodes.add(pred);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrOpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrOpProcFactory.java
index 20d4783..ff41c09 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrOpProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrOpProcFactory.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -56,7 +56,7 @@ public final class PcrOpProcFactory {
    * Remove partition condition in a filter operator when possible. This is
    * called only when the filter follows a table scan operator.
    */
-  public static class FilterPCR implements NodeProcessor {
+  public static class FilterPCR implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -158,7 +158,7 @@ public final class PcrOpProcFactory {
   /**
    * Default processor which does nothing
    */
-  public static class DefaultPCR implements NodeProcessor {
+  public static class DefaultPCR implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -168,11 +168,11 @@ public final class PcrOpProcFactory {
     }
   }
 
-  public static NodeProcessor getFilterProc() {
+  public static SemanticNodeProcessor getFilterProc() {
     return new FilterPCR();
   }
 
-  public static NodeProcessor getDefaultProc() {
+  public static SemanticNodeProcessor getDefaultProc() {
     return new DefaultPCR();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java
index ba05ec7..2658133 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
-import java.io.Serializable;
 import java.util.List;
 import java.util.Map;
 import java.util.Stack;
@@ -29,7 +28,7 @@ import org.apache.hadoop.hive.ql.exec.ConditionalTask;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.mr.MapRedTask;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker.TaskGraphWalkerContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -38,7 +37,7 @@ import org.apache.hadoop.hive.ql.plan.MapWork;
 /**
  * Common iteration methods for converting joins and sort-merge joins.
  */
-public abstract class AbstractJoinTaskDispatcher implements Dispatcher {
+public abstract class AbstractJoinTaskDispatcher implements SemanticDispatcher {
 
   protected final PhysicalContext physicalContext;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java
index 2727e80..21969f6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
@@ -36,11 +35,11 @@ import org.apache.hadoop.hive.ql.exec.mr.MapRedTask;
 import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
 import org.apache.hadoop.hive.ql.exec.tez.TezTask;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.ExplainConfiguration.AnalyzeState;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -56,11 +55,11 @@ import org.apache.hadoop.hive.ql.stats.fs.FSStatsPublisher;
 public class AnnotateRunTimeStatsOptimizer implements PhysicalPlanResolver {
   private static final Logger LOG = LoggerFactory.getLogger(AnnotateRunTimeStatsOptimizer.class);
 
-  private class AnnotateRunTimeStatsDispatcher implements Dispatcher {
+  private class AnnotateRunTimeStatsDispatcher implements SemanticDispatcher {
 
     private final PhysicalContext physicalContext;
 
-    public AnnotateRunTimeStatsDispatcher(PhysicalContext context, Map<Rule, NodeProcessor> rules) {
+    public AnnotateRunTimeStatsDispatcher(PhysicalContext context, Map<SemanticRule, SemanticNodeProcessor> rules) {
       super();
       physicalContext = context;
     }
@@ -143,9 +142,9 @@ public class AnnotateRunTimeStatsOptimizer implements PhysicalPlanResolver {
 
   @Override
   public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
-    Dispatcher disp = new AnnotateRunTimeStatsDispatcher(pctx, opRules);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
+    SemanticDispatcher disp = new AnnotateRunTimeStatsDispatcher(pctx, opRules);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getRootTasks());
     ogw.startWalking(topNodes, null);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java
index a5a1fb0..514efdb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java
@@ -36,12 +36,12 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.mr.ExecDriver;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleExactMatch;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -99,7 +99,7 @@ public class BucketingSortingInferenceOptimizer implements PhysicalPlanResolver
       // In particular, this guarantees that the first operator is the reducer
       // (and its parent(s) are ReduceSinkOperators) since it begins walking the tree from
       // the reducer.
-      Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       opRules.put(new RuleRegExp("R1", SelectOperator.getOperatorName() + "%"),
           BucketingSortingOpProcFactory.getSelProc());
       // Matches only GroupByOperators which are reducers, rather than map group by operators,
@@ -132,9 +132,9 @@ public class BucketingSortingInferenceOptimizer implements PhysicalPlanResolver
 
       // The dispatcher fires the processor corresponding to the closest matching rule and passes
       // the context along
-      Dispatcher disp = new DefaultRuleDispatcher(BucketingSortingOpProcFactory.getDefaultProc(),
+      SemanticDispatcher disp = new DefaultRuleDispatcher(BucketingSortingOpProcFactory.getDefaultProc(),
           opRules, bCtx);
-      GraphWalker ogw = new PreOrderWalker(disp);
+      SemanticGraphWalker ogw = new PreOrderWalker(disp);
 
       // Create a list of topop nodes
       ArrayList<Node> topNodes = new ArrayList<Node>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java
index 01fb734..bcbfb64 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.Utils;
 import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.BucketCol;
@@ -58,7 +58,7 @@ import org.slf4j.LoggerFactory;
 public class BucketingSortingOpProcFactory {
   private static final Logger LOG = LoggerFactory.getLogger(BucketingSortingOpProcFactory.class);
 
-  public static class DefaultInferrer implements NodeProcessor {
+  public static class DefaultInferrer implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -130,7 +130,7 @@ public class BucketingSortingOpProcFactory {
    * ReduceSink Operator
    *
    */
-  public static class JoinInferrer extends DefaultInferrer implements NodeProcessor {
+  public static class JoinInferrer extends DefaultInferrer implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -326,7 +326,7 @@ public class BucketingSortingOpProcFactory {
   /**
    * Processor for Select operator.
    */
-  public static class SelectInferrer extends DefaultInferrer implements NodeProcessor {
+  public static class SelectInferrer extends DefaultInferrer implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -456,7 +456,7 @@ public class BucketingSortingOpProcFactory {
   /**
    * Processor for FileSink operator.
    */
-  public static class FileSinkInferrer extends DefaultInferrer implements NodeProcessor {
+  public static class FileSinkInferrer extends DefaultInferrer implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -578,7 +578,7 @@ public class BucketingSortingOpProcFactory {
    *
    */
 
-  public static class MultiGroupByInferrer extends GroupByInferrer implements NodeProcessor {
+  public static class MultiGroupByInferrer extends GroupByInferrer implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -620,7 +620,7 @@ public class BucketingSortingOpProcFactory {
    *
    * It is up to the caller to guarantee the tree matches this pattern.
    */
-  public static class GroupByInferrer extends DefaultInferrer implements NodeProcessor {
+  public static class GroupByInferrer extends DefaultInferrer implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -716,7 +716,7 @@ public class BucketingSortingOpProcFactory {
   /**
    * Filter processor
    */
-  public static class ForwardingInferrer extends DefaultInferrer implements NodeProcessor {
+  public static class ForwardingInferrer extends DefaultInferrer implements SemanticNodeProcessor {
     @SuppressWarnings("unchecked")
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -729,47 +729,47 @@ public class BucketingSortingOpProcFactory {
     }
   }
 
-  public static NodeProcessor getDefaultProc() {
+  public static SemanticNodeProcessor getDefaultProc() {
     return new DefaultInferrer();
   }
 
-  public static NodeProcessor getJoinProc() {
+  public static SemanticNodeProcessor getJoinProc() {
     return new JoinInferrer();
   }
 
-  public static NodeProcessor getSelProc() {
+  public static SemanticNodeProcessor getSelProc() {
     return new SelectInferrer();
   }
 
-  public static NodeProcessor getGroupByProc() {
+  public static SemanticNodeProcessor getGroupByProc() {
     return new GroupByInferrer();
   }
 
-  public static NodeProcessor getFileSinkProc() {
+  public static SemanticNodeProcessor getFileSinkProc() {
     return new FileSinkInferrer();
   }
 
-  public static NodeProcessor getFilterProc() {
+  public static SemanticNodeProcessor getFilterProc() {
     return new ForwardingInferrer();
   }
 
-  public static NodeProcessor getLimitProc() {
+  public static SemanticNodeProcessor getLimitProc() {
     return new ForwardingInferrer();
   }
 
-  public static NodeProcessor getLateralViewForwardProc() {
+  public static SemanticNodeProcessor getLateralViewForwardProc() {
     return new ForwardingInferrer();
   }
 
-  public static NodeProcessor getLateralViewJoinProc() {
+  public static SemanticNodeProcessor getLateralViewJoinProc() {
     return new ForwardingInferrer();
   }
 
-  public static NodeProcessor getForwardProc() {
+  public static SemanticNodeProcessor getForwardProc() {
     return new ForwardingInferrer();
   }
 
-  public static NodeProcessor getMultiGroupByProc() {
+  public static SemanticNodeProcessor getMultiGroupByProc() {
     return new MultiGroupByInferrer();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java
index a84534a..be86f7c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer.physical;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -68,7 +68,7 @@ public class CommonJoinResolver implements PhysicalPlanResolver {
   public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException {
 
     // create dispatcher and graph walker
-    Dispatcher disp = new CommonJoinTaskDispatcher(pctx);
+    SemanticDispatcher disp = new CommonJoinTaskDispatcher(pctx);
     TaskGraphWalker ogw = new TaskGraphWalker(disp);
 
     // get all the tasks nodes from root task
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
index 9c4a0c2..035a5fd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
@@ -45,10 +45,9 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.mr.MapRedTask;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
 import org.apache.hadoop.hive.ql.optimizer.MapJoinProcessor;
-import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ConditionalResolverCommonJoin;
 import org.apache.hadoop.hive.ql.plan.ConditionalResolverCommonJoin.ConditionalResolverCommonJoinCtx;
@@ -104,7 +103,7 @@ import org.apache.hadoop.hive.ql.plan.ReduceWork;
  * MapredLocalTask. then make this new generated task depends on current task's parent task, and
  * make current task depends on this new generated task
  */
-public class CommonJoinTaskDispatcher extends AbstractJoinTaskDispatcher implements Dispatcher {
+public class CommonJoinTaskDispatcher extends AbstractJoinTaskDispatcher implements SemanticDispatcher {
 
   HashMap<String, Long> aliasToSize = null;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CrossProductHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CrossProductHandler.java
index d970316..6b744dc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CrossProductHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CrossProductHandler.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -48,12 +47,12 @@ import org.apache.hadoop.hive.ql.exec.tez.TezTask;
 import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -79,7 +78,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
  * If the keys expr list on the mapJoin Desc is an empty list for any input,
  * this implies a cross product.
  */
-public class CrossProductHandler implements PhysicalPlanResolver, Dispatcher {
+public class CrossProductHandler implements PhysicalPlanResolver, SemanticDispatcher {
 
   protected static transient final Logger LOG = LoggerFactory
       .getLogger(CrossProductHandler.class);
@@ -268,7 +267,7 @@ public class CrossProductHandler implements PhysicalPlanResolver, Dispatcher {
    * <p>
    * For MR the taskname is the StageName, for Tez it is the vertex name.
    */
-  public static class MapJoinCheck implements NodeProcessor, NodeProcessorCtx {
+  public static class MapJoinCheck implements SemanticNodeProcessor, NodeProcessorCtx {
 
     final List<String> warnings;
     final String taskName;
@@ -279,11 +278,11 @@ public class CrossProductHandler implements PhysicalPlanResolver, Dispatcher {
     }
 
     List<String> analyze(BaseWork work) throws SemanticException {
-      Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       opRules.put(new RuleRegExp("R1", MapJoinOperator.getOperatorName()
           + "%"), this);
-      Dispatcher disp = new DefaultRuleDispatcher(new NoopProcessor(), opRules, this);
-      GraphWalker ogw = new DefaultGraphWalker(disp);
+      SemanticDispatcher disp = new DefaultRuleDispatcher(new NoopProcessor(), opRules, this);
+      SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
       ArrayList<Node> topNodes = new ArrayList<Node>();
       topNodes.addAll(work.getAllRootOperators());
       ogw.startWalking(topNodes, null);
@@ -329,7 +328,7 @@ public class CrossProductHandler implements PhysicalPlanResolver, Dispatcher {
    * in the Work. For Tez, you can restrict it to ReduceSinks for a particular output
    * vertex.
    */
-  public static class ExtractReduceSinkInfo implements NodeProcessor, NodeProcessorCtx {
+  public static class ExtractReduceSinkInfo implements SemanticNodeProcessor, NodeProcessorCtx {
 
     static class Info {
       List<ExprNodeDesc> keyCols;
@@ -355,11 +354,11 @@ public class CrossProductHandler implements PhysicalPlanResolver, Dispatcher {
     }
 
     Map<Integer, Info> analyze(BaseWork work) throws SemanticException {
-      Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       opRules.put(new RuleRegExp("R1", ReduceSinkOperator.getOperatorName()
           + "%"), this);
-      Dispatcher disp = new DefaultRuleDispatcher(new NoopProcessor(), opRules, this);
-      GraphWalker ogw = new DefaultGraphWalker(disp);
+      SemanticDispatcher disp = new DefaultRuleDispatcher(new NoopProcessor(), opRules, this);
+      SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
       ArrayList<Node> topNodes = new ArrayList<Node>();
       topNodes.addAll(work.getAllRootOperators());
       ogw.startWalking(topNodes, null);
@@ -386,7 +385,7 @@ public class CrossProductHandler implements PhysicalPlanResolver, Dispatcher {
     }
   }
 
-  static class NoopProcessor implements NodeProcessor {
+  static class NoopProcessor implements SemanticNodeProcessor {
     @Override
     public final Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java
index 6c370f6..0faedb6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapDecider.java
@@ -24,7 +24,6 @@ import static org.apache.hadoop.hive.ql.optimizer.physical.LlapDecider.LlapMode.
 import static org.apache.hadoop.hive.ql.optimizer.physical.LlapDecider.LlapMode.map;
 import static org.apache.hadoop.hive.ql.optimizer.physical.LlapDecider.LlapMode.none;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Deque;
@@ -53,12 +52,12 @@ import org.apache.hadoop.hive.ql.exec.tez.TezTask;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -73,7 +72,6 @@ import org.apache.hadoop.hive.ql.plan.ReduceWork;
 import org.apache.hadoop.hive.ql.plan.SelectDesc;
 import org.apache.hadoop.hive.ql.plan.Statistics;
 import org.apache.hadoop.hive.ql.plan.TezWork;
-import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -112,7 +110,7 @@ public class LlapDecider implements PhysicalPlanResolver {
   }
 
 
-  class LlapDecisionDispatcher implements Dispatcher {
+  class LlapDecisionDispatcher implements SemanticDispatcher {
     private final HiveConf conf;
     private final boolean doSkipUdfCheck;
     private final boolean arePermanentFnsAllowed;
@@ -120,7 +118,7 @@ public class LlapDecider implements PhysicalPlanResolver {
     private final float minReducersPerExec;
     private final int executorsPerNode;
     private List<MapJoinOperator> mapJoinOpList;
-    private final Map<Rule, NodeProcessor> rules;
+    private final Map<SemanticRule, SemanticNodeProcessor> rules;
 
     public LlapDecisionDispatcher(PhysicalContext pctx, LlapMode mode) {
       conf = pctx.getConf();
@@ -377,10 +375,10 @@ public class LlapDecider implements PhysicalPlanResolver {
       return true;
     }
 
-    private Map<Rule, NodeProcessor> getRules() {
-      Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    private Map<SemanticRule, SemanticNodeProcessor> getRules() {
+      Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       opRules.put(new RuleRegExp("No scripts", ScriptOperator.getOperatorName() + "%"),
-          new NodeProcessor() {
+          new SemanticNodeProcessor() {
           @Override
           public Object process(Node n, Stack<Node> s, NodeProcessorCtx c,
               Object... os) {
@@ -388,7 +386,7 @@ public class LlapDecider implements PhysicalPlanResolver {
               return Boolean.FALSE;
           }
         });
-      opRules.put(new RuleRegExp("No user code in fil", FilterOperator.getOperatorName() + "%"), new NodeProcessor() {
+      opRules.put(new RuleRegExp("No user code in fil", FilterOperator.getOperatorName() + "%"), new SemanticNodeProcessor() {
         @Override
         public Object process(Node n, Stack<Node> s, NodeProcessorCtx c, Object... os) {
           ExprNodeDesc expr = ((FilterOperator) n).getConf().getPredicate();
@@ -399,7 +397,7 @@ public class LlapDecider implements PhysicalPlanResolver {
           return Boolean.valueOf(retval);
         }
       });
-      opRules.put(new RuleRegExp("No user code in gby", GroupByOperator.getOperatorName() + "%"), new NodeProcessor() {
+      opRules.put(new RuleRegExp("No user code in gby", GroupByOperator.getOperatorName() + "%"), new SemanticNodeProcessor() {
         @Override
         public Object process(Node n, Stack<Node> s, NodeProcessorCtx c, Object... os) {
           @SuppressWarnings("unchecked")
@@ -412,7 +410,7 @@ public class LlapDecider implements PhysicalPlanResolver {
         }
       });
       opRules.put(new RuleRegExp("No user code in select", SelectOperator.getOperatorName() + "%"),
-          new NodeProcessor() {
+          new SemanticNodeProcessor() {
             @Override
             public Object process(Node n, Stack<Node> s, NodeProcessorCtx c, Object... os) {
               @SuppressWarnings({"unchecked"})
@@ -427,7 +425,7 @@ public class LlapDecider implements PhysicalPlanResolver {
 
       if (!conf.getBoolVar(HiveConf.ConfVars.LLAP_ENABLE_GRACE_JOIN_IN_LLAP)) {
         opRules.put(new RuleRegExp("Disable grace hash join if LLAP mode and not dynamic partition hash join",
-            MapJoinOperator.getOperatorName() + "%"), new NodeProcessor() {
+            MapJoinOperator.getOperatorName() + "%"), new SemanticNodeProcessor() {
               @Override
               public Object process(Node n, Stack<Node> s, NodeProcessorCtx c, Object... os) {
                 MapJoinOperator mapJoinOp = (MapJoinOperator) n;
@@ -445,8 +443,8 @@ public class LlapDecider implements PhysicalPlanResolver {
     private boolean evaluateOperators(BaseWork work) throws SemanticException {
       // lets take a look at the operators. we're checking for user
       // code in those. we will not run that in llap.
-      Dispatcher disp = new DefaultRuleDispatcher(null, rules, null);
-      GraphWalker ogw = new DefaultGraphWalker(disp);
+      SemanticDispatcher disp = new DefaultRuleDispatcher(null, rules, null);
+      SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
       ArrayList<Node> topNodes = new ArrayList<Node>();
       topNodes.addAll(work.getAllRootOperators());
@@ -541,7 +539,7 @@ public class LlapDecider implements PhysicalPlanResolver {
     }
 
     // create dispatcher and graph walker
-    Dispatcher disp = new LlapDecisionDispatcher(pctx, mode);
+    SemanticDispatcher disp = new LlapDecisionDispatcher(pctx, mode);
     TaskGraphWalker ogw = new TaskGraphWalker(disp);
 
     // get all the tasks nodes from root task
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapPreVectorizationPass.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapPreVectorizationPass.java
index 16fbe95..b5b8e14 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapPreVectorizationPass.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapPreVectorizationPass.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hive.ql.optimizer.physical;
 
 import static org.apache.hadoop.hive.ql.optimizer.physical.LlapDecider.LlapMode.none;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -31,19 +30,18 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.tez.TezTask;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.physical.LlapDecider.LlapMode;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.TezWork;
 
-import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,8 +60,8 @@ public class LlapPreVectorizationPass implements PhysicalPlanResolver {
       return pctx;
     }
 
-    Dispatcher disp = new LlapPreVectorizationPassDispatcher(pctx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new LlapPreVectorizationPassDispatcher(pctx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getRootTasks());
     ogw.startWalking(topNodes, null);
@@ -71,7 +69,7 @@ public class LlapPreVectorizationPass implements PhysicalPlanResolver {
     return pctx;
   }
 
-  class LlapPreVectorizationPassDispatcher implements Dispatcher {
+  class LlapPreVectorizationPassDispatcher implements SemanticDispatcher {
     HiveConf conf;
 
     LlapPreVectorizationPassDispatcher(PhysicalContext pctx) {
@@ -94,7 +92,7 @@ public class LlapPreVectorizationPass implements PhysicalPlanResolver {
 
     private void handleWork(TezWork tezWork, BaseWork work)
         throws SemanticException {
-      Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
       if (conf.getVar(HiveConf.ConfVars.LLAP_EXECUTION_MODE).equals("only")
           && !conf.getBoolVar(HiveConf.ConfVars.LLAP_ENABLE_GRACE_JOIN_IN_LLAP)) {
@@ -102,7 +100,7 @@ public class LlapPreVectorizationPass implements PhysicalPlanResolver {
         // Since the presence of Grace Hash Join disables some "native" vectorization optimizations,
         // we will disable the grace hash join now, before vectorization is done.
         opRules.put(new RuleRegExp("Disable grace hash join if LLAP mode and not dynamic partition hash join",
-            MapJoinOperator.getOperatorName() + "%"), new NodeProcessor() {
+            MapJoinOperator.getOperatorName() + "%"), new SemanticNodeProcessor() {
               @Override
               public Object process(Node n, Stack<Node> s, NodeProcessorCtx c, Object... os) {
                 MapJoinOperator mapJoinOp = (MapJoinOperator) n;
@@ -115,8 +113,8 @@ public class LlapPreVectorizationPass implements PhysicalPlanResolver {
       }
 
       if (!opRules.isEmpty()) {
-        Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
-        GraphWalker ogw = new DefaultGraphWalker(disp);
+        SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
+        SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
         ArrayList<Node> topNodes = new ArrayList<Node>();
         topNodes.addAll(work.getAllRootOperators());
         ogw.startWalking(topNodes, null);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java
index 3e81ab5..29112f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java
@@ -37,12 +37,12 @@ import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.physical.MapJoinResolver.LocalMapJoinProcCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -64,16 +64,16 @@ import org.apache.hadoop.hive.ql.plan.TableDesc;
 public final class LocalMapJoinProcFactory {
   private static final Logger LOG = LoggerFactory.getLogger(LocalMapJoinProcFactory.class);
 
-  public static NodeProcessor getJoinProc() {
+  public static SemanticNodeProcessor getJoinProc() {
     return new LocalMapJoinProcessor();
   }
 
-  public static NodeProcessor getGroupByProc() {
+  public static SemanticNodeProcessor getGroupByProc() {
     return new MapJoinFollowedByGroupByProcessor();
   }
 
-  public static NodeProcessor getDefaultProc() {
-    return new NodeProcessor() {
+  public static SemanticNodeProcessor getDefaultProc() {
+    return new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
           Object... nodeOutputs) throws SemanticException {
@@ -86,7 +86,7 @@ public final class LocalMapJoinProcFactory {
    * MapJoinFollowByProcessor.
    *
    */
-  public static class MapJoinFollowedByGroupByProcessor implements NodeProcessor {
+  public static class MapJoinFollowedByGroupByProcessor implements SemanticNodeProcessor {
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
       LocalMapJoinProcCtx context = (LocalMapJoinProcCtx) ctx;
@@ -106,7 +106,7 @@ public final class LocalMapJoinProcFactory {
    * LocalMapJoinProcessor.
    *
    */
-  public static class LocalMapJoinProcessor implements NodeProcessor {
+  public static class LocalMapJoinProcessor implements SemanticNodeProcessor {
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
         throws SemanticException {
       LocalMapJoinProcCtx context = (LocalMapJoinProcCtx) ctx;
@@ -252,14 +252,14 @@ public final class LocalMapJoinProcFactory {
     public void hasGroupBy(Operator<? extends OperatorDesc> mapJoinOp,
         LocalMapJoinProcCtx localMapJoinProcCtx) throws Exception {
       List<Operator<? extends OperatorDesc>> childOps = mapJoinOp.getChildOperators();
-      Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       opRules.put(new RuleRegExp("R1", GroupByOperator.getOperatorName() + "%"),
         LocalMapJoinProcFactory.getGroupByProc());
       // The dispatcher fires the processor corresponding to the closest
       // matching rule and passes the context along
-      Dispatcher disp = new DefaultRuleDispatcher(LocalMapJoinProcFactory.getDefaultProc(),
+      SemanticDispatcher disp = new DefaultRuleDispatcher(LocalMapJoinProcFactory.getDefaultProc(),
           opRules, localMapJoinProcCtx);
-      GraphWalker ogw = new DefaultGraphWalker(disp);
+      SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
       // iterator the reducer operator tree
       ArrayList<Node> topNodes = new ArrayList<Node>();
       topNodes.addAll(childOps);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java
index 484369b..ec966f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java
@@ -39,12 +39,12 @@ import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -70,7 +70,7 @@ public class MapJoinResolver implements PhysicalPlanResolver {
   public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException {
 
     // create dispatcher and graph walker
-    Dispatcher disp = new LocalMapJoinTaskDispatcher(pctx);
+    SemanticDispatcher disp = new LocalMapJoinTaskDispatcher(pctx);
     TaskGraphWalker ogw = new TaskGraphWalker(disp);
 
     // get all the tasks nodes from root task
@@ -87,7 +87,7 @@ public class MapJoinResolver implements PhysicalPlanResolver {
    * MapredLocalTask. then make this new generated task depends on current task's parent task, and
    * make current task depends on this new generated task
    */
-  class LocalMapJoinTaskDispatcher implements Dispatcher {
+  class LocalMapJoinTaskDispatcher implements SemanticDispatcher {
 
     private PhysicalContext physicalContext;
 
@@ -250,14 +250,14 @@ public class MapJoinResolver implements PhysicalPlanResolver {
         throws SemanticException {
       LocalMapJoinProcCtx localMapJoinProcCtx = new LocalMapJoinProcCtx(task, physicalContext
           .getParseContext());
-      Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       opRules.put(new RuleRegExp("R1", MapJoinOperator.getOperatorName() + "%"),
         LocalMapJoinProcFactory.getJoinProc());
       // The dispatcher fires the processor corresponding to the closest
       // matching rule and passes the context along
-      Dispatcher disp = new DefaultRuleDispatcher(LocalMapJoinProcFactory.getDefaultProc(),
+      SemanticDispatcher disp = new DefaultRuleDispatcher(LocalMapJoinProcFactory.getDefaultProc(),
           opRules, localMapJoinProcCtx);
-      GraphWalker ogw = new DefaultGraphWalker(disp);
+      SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
       // iterator the reducer operator tree
       ArrayList<Node> topNodes = new ArrayList<Node>();
       topNodes.addAll(task.getWork().getAliasToWork().values());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MemoryDecider.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MemoryDecider.java
index 11dba77..a87164f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MemoryDecider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MemoryDecider.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -37,12 +36,12 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.tez.TezTask;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -65,7 +64,7 @@ public class MemoryDecider implements PhysicalPlanResolver {
 
   protected static transient final Logger LOG = LoggerFactory.getLogger(MemoryDecider.class);
 
-  public class MemoryCalculator implements Dispatcher {
+  public class MemoryCalculator implements SemanticDispatcher {
 
     private final long totalAvailableMemory; // how much to we have
     private final long minimumHashTableSize; // minimum size of ht completely in memory
@@ -126,12 +125,12 @@ public class MemoryDecider implements PhysicalPlanResolver {
 
     private void evaluateOperators(BaseWork w, PhysicalContext pctx) throws SemanticException {
       // lets take a look at the operator memory requirements.
-      Dispatcher disp = null;
+      SemanticDispatcher disp = null;
       final Set<MapJoinOperator> mapJoins = new LinkedHashSet<MapJoinOperator>();
 
-      LinkedHashMap<Rule, NodeProcessor> rules = new LinkedHashMap<Rule, NodeProcessor>();
+      LinkedHashMap<SemanticRule, SemanticNodeProcessor> rules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       rules.put(new RuleRegExp("Map join memory estimator",
-              MapJoinOperator.getOperatorName() + "%"), new NodeProcessor() {
+              MapJoinOperator.getOperatorName() + "%"), new SemanticNodeProcessor() {
           @Override
           public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
               Object... nodeOutputs) {
@@ -141,7 +140,7 @@ public class MemoryDecider implements PhysicalPlanResolver {
         });
       disp = new DefaultRuleDispatcher(null, rules, null);
 
-      GraphWalker ogw = new DefaultGraphWalker(disp);
+      SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
       ArrayList<Node> topNodes = new ArrayList<Node>();
       topNodes.addAll(w.getAllRootOperators());
@@ -268,7 +267,7 @@ public class MemoryDecider implements PhysicalPlanResolver {
       return size;
     }
 
-    public class DefaultRule implements NodeProcessor {
+    public class DefaultRule implements SemanticNodeProcessor {
 
       @Override
       public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -284,7 +283,7 @@ public class MemoryDecider implements PhysicalPlanResolver {
     pctx.getConf();
 
     // create dispatcher and graph walker
-    Dispatcher disp = new MemoryCalculator(pctx);
+    SemanticDispatcher disp = new MemoryCalculator(pctx);
     TaskGraphWalker ogw = new TaskGraphWalker(disp);
 
     // get all the tasks nodes from root task
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java
index 1de0fb8..5ec9762 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java
@@ -30,12 +30,12 @@ import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
@@ -105,7 +105,7 @@ public class MetadataOnlyOptimizer implements PhysicalPlanResolver {
 
   }
 
-  static private class TableScanProcessor implements NodeProcessor {
+  static private class TableScanProcessor implements SemanticNodeProcessor {
     public TableScanProcessor() {
     }
 
@@ -127,7 +127,7 @@ public class MetadataOnlyOptimizer implements PhysicalPlanResolver {
     }
   }
 
-  static private class FileSinkProcessor implements NodeProcessor {
+  static private class FileSinkProcessor implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -156,14 +156,14 @@ public class MetadataOnlyOptimizer implements PhysicalPlanResolver {
 
   @Override
   public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", TableScanOperator.getOperatorName() + "%"),
       new TableScanProcessor());
     opRules.put(new RuleRegExp("R2",
       GroupByOperator.getOperatorName() + "%.*" + FileSinkOperator.getOperatorName() + "%"),
       new FileSinkProcessor());
-    Dispatcher disp = new NullScanTaskDispatcher(pctx, opRules);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new NullScanTaskDispatcher(pctx, opRules);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getRootTasks());
     ogw.startWalking(topNodes, null);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanOptimizer.java
index 282805d..202b7ca 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanOptimizer.java
@@ -33,12 +33,12 @@ import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.LimitOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.physical.MetadataOnlyOptimizer.WalkerCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -61,14 +61,14 @@ public class NullScanOptimizer implements PhysicalPlanResolver {
   @Override
   public PhysicalContext resolve(PhysicalContext pctx)
       throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<>();
     opRules.put(
         new RuleRegExp("R1",
             TableScanOperator.getOperatorName() + "%.*"
                 + FilterOperator.getOperatorName() + "%"),
         new WhereFalseProcessor());
-    Dispatcher disp = new NullScanTaskDispatcher(pctx, opRules);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new NullScanTaskDispatcher(pctx, opRules);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     List<Node> topNodes = new ArrayList<>(pctx.getRootTasks());
     ogw.startWalking(topNodes, null);
 
@@ -109,7 +109,7 @@ public class NullScanOptimizer implements PhysicalPlanResolver {
     return true;
   }
 
-  private static class WhereFalseProcessor implements NodeProcessor {
+  private static class WhereFalseProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -139,7 +139,7 @@ public class NullScanOptimizer implements PhysicalPlanResolver {
     }
   }
 
-  private static class Limit0Processor implements NodeProcessor {
+  private static class Limit0Processor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -165,7 +165,7 @@ public class NullScanOptimizer implements PhysicalPlanResolver {
     }
   }
 
-  private static class TSMarker implements NodeProcessor {
+  private static class TSMarker implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
index 2fb666d..9b5a099 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanTaskDispatcher.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -45,12 +44,12 @@ import org.apache.hadoop.hive.ql.io.NullScanFileSystem;
 import org.apache.hadoop.hive.ql.io.OneNullRowInputFormat;
 import org.apache.hadoop.hive.ql.io.ZeroRowsInputFormat;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.PreOrderOnceWalker;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.optimizer.physical.MetadataOnlyOptimizer.WalkerCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -66,16 +65,16 @@ import org.slf4j.LoggerFactory;
  * Iterate over all tasks one by one and removes all input paths from task if
  * conditions as defined in rules match.
  */
-public class NullScanTaskDispatcher implements Dispatcher {
+public class NullScanTaskDispatcher implements SemanticDispatcher {
 
   static final Logger LOG =
       LoggerFactory.getLogger(NullScanTaskDispatcher.class);
 
   private final PhysicalContext physicalContext;
-  private final Map<Rule, NodeProcessor> rules;
+  private final Map<SemanticRule, SemanticNodeProcessor> rules;
 
   public NullScanTaskDispatcher(PhysicalContext context,
-      Map<Rule, NodeProcessor> rules) {
+      Map<SemanticRule, SemanticNodeProcessor> rules) {
     super();
     this.physicalContext = context;
     this.rules = rules;
@@ -195,8 +194,8 @@ public class NullScanTaskDispatcher implements Dispatcher {
 
       // The dispatcher fires the processor corresponding to the closest
       // matching rule and passes the context along
-      Dispatcher disp = new DefaultRuleDispatcher(null, rules, walkerCtx);
-      GraphWalker ogw = new PreOrderOnceWalker(disp);
+      SemanticDispatcher disp = new DefaultRuleDispatcher(null, rules, walkerCtx);
+      SemanticGraphWalker ogw = new PreOrderOnceWalker(disp);
 
       // Create a list of topOp nodes
       ArrayList<Node> topNodes = new ArrayList<>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SerializeFilter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SerializeFilter.java
index c15d050..273196f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SerializeFilter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SerializeFilter.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
@@ -31,12 +30,12 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.tez.TezTask;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -56,7 +55,7 @@ public class SerializeFilter implements PhysicalPlanResolver {
 
   protected static transient final Logger LOG = LoggerFactory.getLogger(SerializeFilter.class);
 
-  public class Serializer implements Dispatcher {
+  public class Serializer implements SemanticDispatcher {
 
     private final PhysicalContext pctx;
 
@@ -110,12 +109,12 @@ public class SerializeFilter implements PhysicalPlanResolver {
 
     private void evaluateOperators(BaseWork w, PhysicalContext pctx) throws SemanticException {
 
-      Dispatcher disp = null;
+      SemanticDispatcher disp = null;
       final Set<TableScanOperator> tableScans = new LinkedHashSet<TableScanOperator>();
 
-      LinkedHashMap<Rule, NodeProcessor> rules = new LinkedHashMap<Rule, NodeProcessor>();
+      LinkedHashMap<SemanticRule, SemanticNodeProcessor> rules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       rules.put(new RuleRegExp("TS finder",
-              TableScanOperator.getOperatorName() + "%"), new NodeProcessor() {
+              TableScanOperator.getOperatorName() + "%"), new SemanticNodeProcessor() {
           @Override
           public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
               Object... nodeOutputs) {
@@ -125,7 +124,7 @@ public class SerializeFilter implements PhysicalPlanResolver {
         });
       disp = new DefaultRuleDispatcher(null, rules, null);
 
-      GraphWalker ogw = new DefaultGraphWalker(disp);
+      SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
       ArrayList<Node> topNodes = new ArrayList<Node>();
       topNodes.addAll(w.getAllRootOperators());
@@ -153,7 +152,7 @@ public class SerializeFilter implements PhysicalPlanResolver {
       }
     }
 
-    public class DefaultRule implements NodeProcessor {
+    public class DefaultRule implements SemanticNodeProcessor {
 
       @Override
       public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -169,7 +168,7 @@ public class SerializeFilter implements PhysicalPlanResolver {
     pctx.getConf();
 
     // create dispatcher and graph walker
-    Dispatcher disp = new Serializer(pctx);
+    SemanticDispatcher disp = new Serializer(pctx);
     TaskGraphWalker ogw = new TaskGraphWalker(disp);
 
     // get all the tasks nodes from root task
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinProcFactory.java
index 093226c..63a326e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinProcFactory.java
@@ -18,13 +18,12 @@
 
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
-import java.io.Serializable;
 import java.util.Stack;
 
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.physical.SkewJoinResolver.SkewJoinProcCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -35,11 +34,11 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
  */
 public final class SkewJoinProcFactory {
 
-  public static NodeProcessor getDefaultProc() {
+  public static SemanticNodeProcessor getDefaultProc() {
     return new SkewJoinDefaultProcessor();
   }
 
-  public static NodeProcessor getJoinProc() {
+  public static SemanticNodeProcessor getJoinProc() {
     return new SkewJoinJoinProcessor();
   }
 
@@ -47,7 +46,7 @@ public final class SkewJoinProcFactory {
    * SkewJoinJoinProcessor.
    *
    */
-  public static class SkewJoinJoinProcessor implements NodeProcessor {
+  public static class SkewJoinJoinProcessor implements SemanticNodeProcessor {
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
       SkewJoinProcCtx context = (SkewJoinProcCtx) ctx;
@@ -66,7 +65,7 @@ public final class SkewJoinProcFactory {
    * SkewJoinDefaultProcessor.
    *
    */
-  public static class SkewJoinDefaultProcessor implements NodeProcessor {
+  public static class SkewJoinDefaultProcessor implements SemanticNodeProcessor {
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx,
         Object... nodeOutputs) throws SemanticException {
       return null;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinResolver.java
index 3cabd38..bd32aaa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinResolver.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -29,12 +28,12 @@ import org.apache.hadoop.hive.ql.exec.ConditionalTask;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -53,8 +52,8 @@ public class SkewJoinResolver implements PhysicalPlanResolver {
 
   @Override
   public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException {
-    Dispatcher disp = new SkewJoinTaskDispatcher(pctx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new SkewJoinTaskDispatcher(pctx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getRootTasks());
     ogw.startWalking(topNodes, null);
@@ -64,7 +63,7 @@ public class SkewJoinResolver implements PhysicalPlanResolver {
   /**
    * Iterator a task with a rule dispatcher for its reducer operator tree.
    */
-  class SkewJoinTaskDispatcher implements Dispatcher {
+  class SkewJoinTaskDispatcher implements SemanticDispatcher {
 
     private PhysicalContext physicalContext;
 
@@ -107,16 +106,16 @@ public class SkewJoinResolver implements PhysicalPlanResolver {
 
       SkewJoinProcCtx skewJoinProcContext = new SkewJoinProcCtx(task, pc);
 
-      Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+      Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
       opRules.put(new RuleRegExp("R1",
         CommonJoinOperator.getOperatorName() + "%"),
         SkewJoinProcFactory.getJoinProc());
 
       // The dispatcher fires the processor corresponding to the closest
       // matching rule and passes the context along
-      Dispatcher disp = new DefaultRuleDispatcher(SkewJoinProcFactory
+      SemanticDispatcher disp = new DefaultRuleDispatcher(SkewJoinProcFactory
           .getDefaultProc(), opRules, skewJoinProcContext);
-      GraphWalker ogw = new DefaultGraphWalker(disp);
+      SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
       // iterator the reducer operator tree
       ArrayList<Node> topNodes = new ArrayList<Node>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinResolver.java
index c839b87..7334b32 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinResolver.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer.physical;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -43,7 +43,7 @@ public class SortMergeJoinResolver implements PhysicalPlanResolver {
   public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException {
 
     // create dispatcher and graph walker
-    Dispatcher disp = new SortMergeJoinTaskDispatcher(pctx);
+    SemanticDispatcher disp = new SortMergeJoinTaskDispatcher(pctx);
     TaskGraphWalker ogw = new TaskGraphWalker(disp);
 
     // get all the tasks nodes from root task
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java
index fbf6852..575e9c2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.mr.MapRedTask;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
 import org.apache.hadoop.hive.ql.optimizer.MapJoinProcessor;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -62,7 +62,7 @@ import org.apache.hadoop.hive.ql.plan.SMBJoinDesc;
  * The conditional task will first try all mapjoin possibilities, and go the the smb join if the
  * mapjoin fails. The smb join will be a backup task for all the mapjoin tasks.
  */
-public class SortMergeJoinTaskDispatcher extends AbstractJoinTaskDispatcher implements Dispatcher {
+public class SortMergeJoinTaskDispatcher extends AbstractJoinTaskDispatcher implements SemanticDispatcher {
 
   public SortMergeJoinTaskDispatcher(PhysicalContext context) {
     super(context);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java
index 80248d6..3debfdc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -34,7 +34,6 @@ import org.apache.hadoop.hive.ql.plan.ReduceWork;
 import org.apache.hadoop.hive.ql.plan.SparkWork;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -53,7 +52,7 @@ import java.util.TreeMap;
  * If the keys expr list on the mapJoin Desc is an empty list for any input,
  * this implies a cross product.
  */
-public class SparkCrossProductCheck implements PhysicalPlanResolver, Dispatcher {
+public class SparkCrossProductCheck implements PhysicalPlanResolver, SemanticDispatcher {
 
   @Override
   public Object dispatch(Node nd, Stack<Node> stack, Object... nodeOutputs)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java
index 98d9c6a..496bf04 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java
@@ -18,13 +18,11 @@
 
 package org.apache.hadoop.hive.ql.optimizer.physical;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 import java.util.Stack;
 
-import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -32,7 +30,7 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc;
@@ -77,7 +75,7 @@ public class SparkDynamicPartitionPruningResolver implements PhysicalPlanResolve
     return pctx;
   }
 
-  private class SparkDynamicPartitionPruningDispatcher implements Dispatcher {
+  private class SparkDynamicPartitionPruningDispatcher implements SemanticDispatcher {
 
     @Override
     public Object dispatch(Node nd, Stack<Node> stack, Object... nodeOutputs) throws SemanticException {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
index 97cb4a4..bfc5002 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -66,7 +66,7 @@ public class SparkMapJoinResolver implements PhysicalPlanResolver {
   @Override
   public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException {
 
-    Dispatcher dispatcher = new SparkMapJoinTaskDispatcher(pctx);
+    SemanticDispatcher dispatcher = new SparkMapJoinTaskDispatcher(pctx);
     TaskGraphWalker graphWalker = new TaskGraphWalker(dispatcher);
 
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -92,7 +92,7 @@ public class SparkMapJoinResolver implements PhysicalPlanResolver {
   }
 
   @SuppressWarnings("unchecked")
-  class SparkMapJoinTaskDispatcher implements Dispatcher {
+  class SparkMapJoinTaskDispatcher implements SemanticDispatcher {
 
     private final PhysicalContext physicalContext;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 8fc834e..27ff0c2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hive.ql.optimizer.physical;
 import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.UNIFORM;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.lang.annotation.Annotation;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -103,7 +102,7 @@ import org.apache.hadoop.hive.ql.io.OneNullRowInputFormat;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.io.ZeroRowsInputFormat;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -985,7 +984,7 @@ public class Vectorizer implements PhysicalPlanResolver {
     return vectorChild;
   }
 
-  class VectorizationDispatcher implements Dispatcher {
+  class VectorizationDispatcher implements SemanticDispatcher {
 
     @Override
     public Object dispatch(Node nd, Stack<Node> stack, Object... nodeOutputs)
@@ -2512,7 +2511,7 @@ public class Vectorizer implements PhysicalPlanResolver {
             HiveConf.ConfVars.HIVE_TEST_VECTORIZATION_SUPPRESS_EXPLAIN_EXECUTION_MODE);
 
     // create dispatcher and graph walker
-    Dispatcher disp = new VectorizationDispatcher();
+    SemanticDispatcher disp = new VectorizationDispatcher();
     TaskGraphWalker ogw = new TaskGraphWalker(disp);
 
     // get all the tasks nodes from root task
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/ExprProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/ExprProcFactory.java
index e20d178..df659d0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/ExprProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/ExprProcFactory.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.ql.optimizer.ppr;
 import java.util.Map;
 
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.PrunerExpressionOperatorFactory;
 import org.apache.hadoop.hive.ql.optimizer.PrunerUtils;
@@ -74,7 +74,7 @@ public final class ExprProcFactory extends PrunerExpressionOperatorFactory {
    *
    * @return
    */
-  public static NodeProcessor getColumnProcessor() {
+  public static SemanticNodeProcessor getColumnProcessor() {
     return new PPRColumnExprProcessor();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/OpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/OpProcFactory.java
index 3c1a335..af22cd4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/OpProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/OpProcFactory.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.ql.optimizer.ppr;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.PrunerOperatorFactory;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -52,7 +52,7 @@ public final class OpProcFactory extends PrunerOperatorFactory {
 
     @Override
     protected void generatePredicate(NodeProcessorCtx procCtx, FilterOperator fop,
-        TableScanOperator top) throws SemanticException, UDFArgumentException {
+                                     TableScanOperator top) throws SemanticException, UDFArgumentException {
       OpWalkerCtx owc = (OpWalkerCtx) procCtx;
       // Otherwise this is not a sampling predicate and we need to
       ExprNodeDesc predicate = fop.getConf().getPredicate();
@@ -67,7 +67,7 @@ public final class OpProcFactory extends PrunerOperatorFactory {
 
   }
 
-  public static NodeProcessor getFilterProc() {
+  public static SemanticNodeProcessor getFilterProc() {
     return new FilterPPR();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/CombineEquivalentWorkResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/CombineEquivalentWorkResolver.java
index c681c74..ee41194 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/CombineEquivalentWorkResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/CombineEquivalentWorkResolver.java
@@ -34,7 +34,7 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
 import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext;
 import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalPlanResolver;
@@ -72,13 +72,13 @@ public class CombineEquivalentWorkResolver implements PhysicalPlanResolver {
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getRootTasks());
     // use a pre-order walker so that DPP sink works are visited (and combined) first
-    GraphWalker taskWalker = new PreOrderWalker(new EquivalentWorkMatcher());
+    SemanticGraphWalker taskWalker = new PreOrderWalker(new EquivalentWorkMatcher());
     HashMap<Node, Object> nodeOutput = Maps.newHashMap();
     taskWalker.startWalking(topNodes, nodeOutput);
     return pctx;
   }
 
-  class EquivalentWorkMatcher implements Dispatcher {
+  class EquivalentWorkMatcher implements SemanticDispatcher {
     private Comparator<BaseWork> baseWorkComparator = new Comparator<BaseWork>() {
       @Override
       public int compare(BaseWork o1, BaseWork o2) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
index 341422f..d406574 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession;
 import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManager;
 import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -59,7 +59,7 @@ import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.UNIFOR
  * SetSparkReducerParallelism determines how many reducers should
  * be run for a given reduce sink, clone from SetReducerParallelism.
  */
-public class SetSparkReducerParallelism implements NodeProcessor {
+public class SetSparkReducerParallelism implements SemanticNodeProcessor {
 
   private static final Logger LOG = LoggerFactory.getLogger(SetSparkReducerParallelism.class.getName());
 
@@ -76,7 +76,7 @@ public class SetSparkReducerParallelism implements NodeProcessor {
 
   @Override
   public Object process(Node nd, Stack<Node> stack,
-      NodeProcessorCtx procContext, Object... nodeOutputs)
+                        NodeProcessorCtx procContext, Object... nodeOutputs)
       throws SemanticException {
 
     OptimizeSparkProcContext context = (OptimizeSparkProcContext) procContext;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java
index 884e693..821f8e2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java
@@ -22,7 +22,7 @@ import java.util.Stack;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.BucketJoinProcCtx;
 import org.apache.hadoop.hive.ql.optimizer.BucketMapjoinProc;
@@ -39,7 +39,7 @@ import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext;
  * The idea is eventually to process even hinted Mapjoin hints here,
  * but due to code complexity in refactoring, that is still in Optimizer.
  */
-public class SparkJoinHintOptimizer implements NodeProcessor {
+public class SparkJoinHintOptimizer implements SemanticNodeProcessor {
 
   private BucketMapjoinProc bucketMapJoinOptimizer;
   private SparkSMBJoinHintOptimizer smbMapJoinOptimizer;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java
index c52acea..ca4f3e8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer.spark;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -32,7 +32,7 @@ import java.util.Stack;
  * Converts a join to a more optimized join for the Spark path.
  * Delegates to a more specialized join processor.
  */
-public class SparkJoinOptimizer implements NodeProcessor {
+public class SparkJoinOptimizer implements SemanticNodeProcessor {
 
   private SparkSortMergeJoinOptimizer smbJoinOptimizer;
   private SparkMapJoinOptimizer mapJoinOptimizer;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
index ad6db21..5dcd49b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
@@ -43,7 +42,7 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.BucketMapjoinProc;
 import org.apache.hadoop.hive.ql.optimizer.MapJoinProcessor;
@@ -64,7 +63,7 @@ import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
  * converted (e.g.: full outer joins cannot be handled as map joins) as well
  * as memory restrictions (one side of the join has to fit into memory).
  */
-public class SparkMapJoinOptimizer implements NodeProcessor {
+public class SparkMapJoinOptimizer implements SemanticNodeProcessor {
 
   private static final Logger LOG = LoggerFactory.getLogger(SparkMapJoinOptimizer.class.getName());
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
index f53e3ce..2cc378d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
@@ -36,12 +36,12 @@ import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.SparkHashTableSinkOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.spark.GenSparkProcContext;
@@ -61,11 +61,11 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 
-public class SparkReduceSinkMapJoinProc implements NodeProcessor {
+public class SparkReduceSinkMapJoinProc implements SemanticNodeProcessor {
 
   public static final Logger LOG = LoggerFactory.getLogger(SparkReduceSinkMapJoinProc.class.getName());
 
-  public static class SparkMapJoinFollowedByGroupByProcessor implements NodeProcessor {
+  public static class SparkMapJoinFollowedByGroupByProcessor implements SemanticNodeProcessor {
     private boolean hasGroupBy = false;
 
     @Override
@@ -88,11 +88,11 @@ public class SparkReduceSinkMapJoinProc implements NodeProcessor {
   private boolean hasGroupBy(Operator<? extends OperatorDesc> mapjoinOp,
                              GenSparkProcContext context) throws SemanticException {
     List<Operator<? extends OperatorDesc>> childOps = mapjoinOp.getChildOperators();
-    Map<Rule, NodeProcessor> rules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> rules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     SparkMapJoinFollowedByGroupByProcessor processor = new SparkMapJoinFollowedByGroupByProcessor();
     rules.put(new RuleRegExp("GBY", GroupByOperator.getOperatorName() + "%"), processor);
-    Dispatcher disp = new DefaultRuleDispatcher(null, rules, context);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, rules, context);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(childOps);
     ogw.startWalking(topNodes, null);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java
index e54e517..816be35 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.AbstractSMBJoinProc;
 import org.apache.hadoop.hive.ql.optimizer.SortBucketJoinProcCtx;
@@ -42,7 +42,7 @@ import com.clearspring.analytics.util.Preconditions;
 /**
  * Converts from a bucket-mapjoin created from hints to SMB mapjoin.
  */
-public class SparkSMBJoinHintOptimizer extends AbstractSMBJoinProc implements NodeProcessor {
+public class SparkSMBJoinHintOptimizer extends AbstractSMBJoinProc implements SemanticNodeProcessor {
 
   public SparkSMBJoinHintOptimizer(ParseContext pctx) {
     super(pctx);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
index 54b2550..9acd268 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
 import org.apache.hadoop.hive.ql.optimizer.physical.GenMRSkewJoinProcessor;
@@ -51,7 +51,6 @@ import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty;
 import org.apache.hadoop.hive.ql.plan.SparkWork;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 
-import java.io.Serializable;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -68,15 +67,15 @@ public class SparkSkewJoinProcFactory {
     // prevent instantiation
   }
 
-  public static NodeProcessor getDefaultProc() {
+  public static SemanticNodeProcessor getDefaultProc() {
     return SkewJoinProcFactory.getDefaultProc();
   }
 
-  public static NodeProcessor getJoinProc() {
+  public static SemanticNodeProcessor getJoinProc() {
     return new SparkSkewJoinJoinProcessor();
   }
 
-  public static class SparkSkewJoinJoinProcessor implements NodeProcessor {
+  public static class SparkSkewJoinJoinProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java
index fbd53e0..7192e04 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.optimizer.spark;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -33,12 +32,12 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext;
 import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalPlanResolver;
@@ -55,16 +54,16 @@ public class SparkSkewJoinResolver implements PhysicalPlanResolver {
   @Override
   public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException {
     SparkSkewJoinProcFactory.getVisitedJoinOp().clear();
-    Dispatcher disp = new SparkSkewJoinTaskDispatcher(pctx);
+    SemanticDispatcher disp = new SparkSkewJoinTaskDispatcher(pctx);
     // since we may split current task, use a pre-order walker
-    GraphWalker ogw = new PreOrderWalker(disp);
+    SemanticGraphWalker ogw = new PreOrderWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pctx.getRootTasks());
     ogw.startWalking(topNodes, null);
     return pctx;
   }
 
-  class SparkSkewJoinTaskDispatcher implements Dispatcher {
+  class SparkSkewJoinTaskDispatcher implements SemanticDispatcher {
     private PhysicalContext physicalContext;
 
     public SparkSkewJoinTaskDispatcher(PhysicalContext context) {
@@ -82,12 +81,12 @@ public class SparkSkewJoinResolver implements PhysicalPlanResolver {
         SparkWork sparkWork = ((SparkTask) task).getWork();
         SparkSkewJoinProcCtx skewJoinProcCtx =
             new SparkSkewJoinProcCtx(task, physicalContext.getParseContext());
-        Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+        Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
         opRules.put(new RuleRegExp("R1", CommonJoinOperator.getOperatorName() + "%"),
             SparkSkewJoinProcFactory.getJoinProc());
-        Dispatcher disp = new DefaultRuleDispatcher(
+        SemanticDispatcher disp = new DefaultRuleDispatcher(
             SparkSkewJoinProcFactory.getDefaultProc(), opRules, skewJoinProcCtx);
-        GraphWalker ogw = new DefaultGraphWalker(disp);
+        SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
         ArrayList<Node> topNodes = new ArrayList<Node>();
         // since we may need to split the task, let's walk the graph bottom-up
         List<ReduceWork> reduceWorkList = sparkWork.getAllReduceWork();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java
index 97b2989..75c9bc3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.AbstractSMBJoinProc;
 import org.apache.hadoop.hive.ql.optimizer.SortBucketJoinProcCtx;
@@ -38,7 +38,7 @@ import java.util.Stack;
 /**
  * Converts a common join operator to an SMB join if eligible.  Handles auto SMB conversion.
  */
-public class SparkSortMergeJoinOptimizer extends AbstractSMBJoinProc implements NodeProcessor {
+public class SparkSortMergeJoinOptimizer extends AbstractSMBJoinProc implements SemanticNodeProcessor {
 
   public SparkSortMergeJoinOptimizer(ParseContext pctx) {
       super(pctx);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateWithStatistics.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateWithStatistics.java
index cfcb355..eb3e5dc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateWithStatistics.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/AnnotateWithStatistics.java
@@ -32,12 +32,12 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDTFOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -51,7 +51,7 @@ public class AnnotateWithStatistics extends Transform {
 
     // create a walker which walks the tree in a BFS manner while maintaining the
     // operator stack. The dispatcher generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("TS", TableScanOperator.getOperatorName() + "%"),
         StatsRulesProcFactory.getTableScanRule());
     opRules.put(new RuleRegExp("SEL", SelectOperator.getOperatorName() + "%"),
@@ -71,9 +71,9 @@ public class AnnotateWithStatistics extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(StatsRulesProcFactory.getDefaultRule(), opRules,
+    SemanticDispatcher disp = new DefaultRuleDispatcher(StatsRulesProcFactory.getDefaultRule(), opRules,
         aspCtx);
-    GraphWalker ogw = new LevelOrderWalker(disp, 0);
+    SemanticGraphWalker ogw = new LevelOrderWalker(disp, 0);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
index f5c9cbd..0ada066 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDTFOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -135,7 +135,7 @@ public class StatsRulesProcFactory {
    * available then number of rows will be estimated from file size and average row size (computed
    * from schema).
    */
-  public static class TableScanStatsRule extends DefaultStatsRule implements NodeProcessor {
+  public static class TableScanStatsRule extends DefaultStatsRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -181,7 +181,7 @@ public class StatsRulesProcFactory {
    * "Database Systems: The Complete Book" by Garcia-Molina et. al.</i>
    * </p>
    */
-  public static class SelectStatsRule extends DefaultStatsRule implements NodeProcessor {
+  public static class SelectStatsRule extends DefaultStatsRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -260,7 +260,7 @@ public class StatsRulesProcFactory {
    * "Database Systems: The Complete Book" by Garcia-Molina et. al.</i>
    * <br>
    */
-  public static class FilterStatsRule extends DefaultStatsRule implements NodeProcessor {
+  public static class FilterStatsRule extends DefaultStatsRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -1353,7 +1353,7 @@ public class StatsRulesProcFactory {
    * "Database Systems: The Complete Book" by Garcia-Molina et. al.</i>
    * </p>
    */
-  public static class GroupByStatsRule extends DefaultStatsRule implements NodeProcessor {
+  public static class GroupByStatsRule extends DefaultStatsRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -1851,7 +1851,7 @@ public class StatsRulesProcFactory {
    * "Database Systems: The Complete Book" by Garcia-Molina et. al.</i>
    * </p>
    */
-  public static class JoinStatsRule extends FilterStatsRule implements NodeProcessor {
+  public static class JoinStatsRule extends FilterStatsRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -2759,7 +2759,7 @@ public class StatsRulesProcFactory {
   /**
    * LIMIT operator changes the number of rows and thereby the data size.
    */
-  public static class LimitStatsRule extends DefaultStatsRule implements NodeProcessor {
+  public static class LimitStatsRule extends DefaultStatsRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -2814,7 +2814,7 @@ public class StatsRulesProcFactory {
    * from the default stats which just aggregates and passes along the statistics
    * without actually renaming based on output schema of the operator.
    */
-  public static class ReduceSinkStatsRule extends DefaultStatsRule implements NodeProcessor {
+  public static class ReduceSinkStatsRule extends DefaultStatsRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
@@ -2872,7 +2872,7 @@ public class StatsRulesProcFactory {
   /**
    * UDTF operator changes the number of rows and thereby the data size.
    */
-  public static class UDTFStatsRule extends DefaultStatsRule implements NodeProcessor {
+  public static class UDTFStatsRule extends DefaultStatsRule implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
                           Object... nodeOutputs) throws SemanticException {
@@ -2915,7 +2915,7 @@ public class StatsRulesProcFactory {
   /**
    * Default rule is to aggregate the statistics from all its parent operators.
    */
-  public static class DefaultStatsRule implements NodeProcessor {
+  public static class DefaultStatsRule implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -2971,39 +2971,39 @@ public class StatsRulesProcFactory {
 
   }
 
-  public static NodeProcessor getTableScanRule() {
+  public static SemanticNodeProcessor getTableScanRule() {
     return new TableScanStatsRule();
   }
 
-  public static NodeProcessor getSelectRule() {
+  public static SemanticNodeProcessor getSelectRule() {
     return new SelectStatsRule();
   }
 
-  public static NodeProcessor getFilterRule() {
+  public static SemanticNodeProcessor getFilterRule() {
     return new FilterStatsRule();
   }
 
-  public static NodeProcessor getGroupByRule() {
+  public static SemanticNodeProcessor getGroupByRule() {
     return new GroupByStatsRule();
   }
 
-  public static NodeProcessor getJoinRule() {
+  public static SemanticNodeProcessor getJoinRule() {
     return new JoinStatsRule();
   }
 
-  public static NodeProcessor getLimitRule() {
+  public static SemanticNodeProcessor getLimitRule() {
     return new LimitStatsRule();
   }
 
-  public static NodeProcessor getReduceSinkRule() {
+  public static SemanticNodeProcessor getReduceSinkRule() {
     return new ReduceSinkStatsRule();
   }
 
-  public static NodeProcessor getUDTFRule() {
+  public static SemanticNodeProcessor getUDTFRule() {
     return new UDTFStatsRule();
   }
 
-  public static NodeProcessor getDefaultRule() {
+  public static SemanticNodeProcessor getDefaultRule() {
     return new DefaultStatsRule();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyProcessor.java
index 8ad52d0..3869ffa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyProcessor.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.TopNKeyOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -41,7 +41,7 @@ import java.util.Stack;
  * TopNKeyProcessor is a processor for TopNKeyOperator.
  * A TopNKeyOperator will be placed before any ReduceSinkOperator which has a topN property >= 0.
  */
-public class TopNKeyProcessor implements NodeProcessor {
+public class TopNKeyProcessor implements SemanticNodeProcessor {
   private static final Logger LOG = LoggerFactory.getLogger(TopNKeyProcessor.class);
   private final int maxTopNAllowed;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyPushdownProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyPushdownProcessor.java
index d2ae723..8cef337 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyPushdownProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/topnkey/TopNKeyPushdownProcessor.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TopNKeyOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -48,7 +48,7 @@ import static org.apache.hadoop.hive.ql.optimizer.topnkey.TopNKeyProcessor.copyD
 /**
  * Implementation of TopNKey operator pushdown.
  */
-public class TopNKeyPushdownProcessor implements NodeProcessor {
+public class TopNKeyPushdownProcessor implements SemanticNodeProcessor {
   private static final Logger LOG = LoggerFactory.getLogger(TopNKeyPushdownProcessor.class);
 
   @Override
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java
index f753a90..cc6b0ff 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.unionproc.UnionProcContext.UnionParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -65,7 +65,7 @@ public final class UnionProcFactory {
   /**
    * MapRed subquery followed by Union.
    */
-  public static class MapRedUnion implements NodeProcessor {
+  public static class MapRedUnion implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -90,7 +90,7 @@ public final class UnionProcFactory {
   /**
    * Map-only subquery followed by Union.
    */
-  public static class MapUnion implements NodeProcessor {
+  public static class MapUnion implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -115,7 +115,7 @@ public final class UnionProcFactory {
   /**
    * Union subquery followed by Union.
    */
-  public static class UnknownUnion implements NodeProcessor {
+  public static class UnknownUnion implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -173,7 +173,7 @@ public final class UnionProcFactory {
    *   select * from (subq1 where filter union all subq2 where filter ...)x;
    * and then optimized.
    */
-  public static class UnionNoProcessFile implements NodeProcessor {
+  public static class UnionNoProcessFile implements SemanticNodeProcessor {
 
     private void pushOperatorsAboveUnion(UnionOperator union,
       Stack<Node> stack, int pos) throws SemanticException {
@@ -310,7 +310,7 @@ public final class UnionProcFactory {
   /**
    * Default processor.
    */
-  public static class NoUnion implements NodeProcessor {
+  public static class NoUnion implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -319,23 +319,23 @@ public final class UnionProcFactory {
     }
   }
 
-  public static NodeProcessor getMapRedUnion() {
+  public static SemanticNodeProcessor getMapRedUnion() {
     return new MapRedUnion();
   }
 
-  public static NodeProcessor getMapUnion() {
+  public static SemanticNodeProcessor getMapUnion() {
     return new MapUnion();
   }
 
-  public static NodeProcessor getUnknownUnion() {
+  public static SemanticNodeProcessor getUnknownUnion() {
     return new UnknownUnion();
   }
 
-  public static NodeProcessor getNoUnion() {
+  public static SemanticNodeProcessor getNoUnion() {
     return new NoUnion();
   }
 
-  public static NodeProcessor getUnionNoProcessFile() {
+  public static SemanticNodeProcessor getUnionNoProcessFile() {
     return new UnionNoProcessFile();
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java
index 2ecb230..d68037f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java
@@ -23,18 +23,16 @@ import java.util.LinkedHashMap;
 import java.util.Map;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -68,7 +66,7 @@ public class UnionProcessor extends Transform {
   public ParseContext transform(ParseContext pCtx) throws SemanticException {
     // create a walker which walks the tree in a BFS manner while maintaining
     // the operator stack.
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1",
       ReduceSinkOperator.getOperatorName() + "%.*" + UnionOperator.getOperatorName() + "%"),
       UnionProcFactory.getMapRedUnion());
@@ -83,7 +81,7 @@ public class UnionProcessor extends Transform {
     // context along
     UnionProcContext uCtx = new UnionProcContext();
     uCtx.setParseContext(pCtx);
-    Dispatcher disp = new DefaultRuleDispatcher(UnionProcFactory.getNoUnion(),
+    SemanticDispatcher disp = new DefaultRuleDispatcher(UnionProcFactory.getNoUnion(),
         opRules, uCtx);
     LevelOrderWalker ogw = new LevelOrderWalker(disp);
     ogw.setNodeTypes(UnionOperator.class);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java
index 1659383..6291d1a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AppMasterEventProcessor.java
@@ -26,7 +26,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
 
@@ -35,7 +35,7 @@ import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
  * processing.
  *
  */
-public class AppMasterEventProcessor implements NodeProcessor {
+public class AppMasterEventProcessor implements SemanticNodeProcessor {
 
   static final private Logger LOG = LoggerFactory.getLogger(AppMasterEventProcessor.class.getName());
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index ee64dbf..f7ac6d3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -371,20 +371,20 @@ public abstract class BaseSemanticAnalyzer {
 
     if (tableNameNode.getType() != HiveParser.TOK_TABNAME ||
         (tableNameNode.getChildCount() != 1 && tableNameNode.getChildCount() != 2)) {
-      throw new SemanticException(ErrorMsg.INVALID_TABLE_NAME.getMsg(tableNameNode));
+      throw new SemanticException(ASTErrorUtils.getMsg(ErrorMsg.INVALID_TABLE_NAME.getMsg(), tableNameNode));
     }
 
     if (tableNameNode.getChildCount() == 2) {
       String dbName = unescapeIdentifier(tableNameNode.getChild(0).getText());
       String tableName = unescapeIdentifier(tableNameNode.getChild(1).getText());
       if (dbName.contains(".") || tableName.contains(".")) {
-        throw new SemanticException(ErrorMsg.OBJECTNAME_CONTAINS_DOT.getMsg(tableNameNode));
+        throw new SemanticException(ASTErrorUtils.getMsg(ErrorMsg.OBJECTNAME_CONTAINS_DOT.getMsg(), tableNameNode));
       }
       return Pair.of(dbName, tableName);
     } else {
       String tableName = unescapeIdentifier(tableNameNode.getChild(0).getText());
       if (tableName.contains(".")) {
-        throw new SemanticException(ErrorMsg.OBJECTNAME_CONTAINS_DOT.getMsg(tableNameNode));
+        throw new SemanticException(ASTErrorUtils.getMsg(ErrorMsg.OBJECTNAME_CONTAINS_DOT.getMsg(), tableNameNode));
       }
       return Pair.of(null,tableName);
     }
@@ -427,19 +427,22 @@ public abstract class BaseSemanticAnalyzer {
   public static TableName getQualifiedTableName(ASTNode tabNameNode, String catalogName) throws SemanticException {
     if (tabNameNode.getType() != HiveParser.TOK_TABNAME || (tabNameNode.getChildCount() != 1
         && tabNameNode.getChildCount() != 2)) {
-      throw new SemanticException(ErrorMsg.INVALID_TABLE_NAME.getMsg(tabNameNode));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_TABLE_NAME.getMsg(), tabNameNode));
     }
     if (tabNameNode.getChildCount() == 2) {
       final String dbName = unescapeIdentifier(tabNameNode.getChild(0).getText());
       final String tableName = unescapeIdentifier(tabNameNode.getChild(1).getText());
       if (dbName.contains(".") || tableName.contains(".")) {
-        throw new SemanticException(ErrorMsg.OBJECTNAME_CONTAINS_DOT.getMsg(tabNameNode));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.OBJECTNAME_CONTAINS_DOT.getMsg(), tabNameNode));
       }
       return HiveTableName.ofNullable(tableName, dbName);
     }
     final String tableName = unescapeIdentifier(tabNameNode.getChild(0).getText());
     if (tableName.contains(".")) {
-      throw new SemanticException(ErrorMsg.OBJECTNAME_CONTAINS_DOT.getMsg(tabNameNode));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.OBJECTNAME_CONTAINS_DOT.getMsg(), tabNameNode));
     }
     return HiveTableName.ofNullable(tableName);
   }
@@ -1014,11 +1017,12 @@ public abstract class BaseSemanticAnalyzer {
           tableHandle = db.getTable(tableName);
         }
       } catch (InvalidTableException ite) {
-        throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(ast
-            .getChild(0)), ite);
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_TABLE.getMsg(), ast.getChild(0)), ite);
       } catch (HiveException e) {
-        throw new SemanticException(ErrorMsg.CANNOT_RETRIEVE_TABLE_METADATA.getMsg(ast
-            .getChild(childIndex), e.getMessage()), e);
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.CANNOT_RETRIEVE_TABLE_METADATA.getMsg(),
+            ast.getChild(childIndex), e.getMessage()), e);
       }
 
       // get partition metadata if partition specified
@@ -1080,8 +1084,8 @@ public abstract class BaseSemanticAnalyzer {
           for (FieldSchema fs: parts) {
             if (partSpec.get(fs.getName().toLowerCase()) == null) {
               if (numStaPart > 0) { // found a DP, but there exists ST as subpartition
-                throw new SemanticException(
-                    ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg(ast.getChild(childIndex)));
+                throw new SemanticException(ASTErrorUtils.getMsg(
+                    ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg(), ast.getChild(childIndex)));
               }
               break;
             } else {
@@ -1106,8 +1110,8 @@ public abstract class BaseSemanticAnalyzer {
               }
             }
           } catch (HiveException e) {
-            throw new SemanticException(
-                ErrorMsg.INVALID_PARTITION.getMsg(ast.getChild(childIndex)), e);
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.INVALID_PARTITION.getMsg(), ast.getChild(childIndex)), e);
           }
           specType = SpecType.STATIC_PARTITION;
         }
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 7680ede..8986315 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
@@ -3275,7 +3275,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
           inputRel = genLateralViewPlans(next, aliasToRel);
           break;
         default:
-          throw new SemanticException(ErrorMsg.LATERAL_VIEW_INVALID_CHILD.getMsg(lateralView));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.LATERAL_VIEW_INVALID_CHILD.getMsg(), lateralView));
       }
       // Input row resolver
       RowResolver inputRR = this.relToHiveRR.get(inputRel);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 8fb161f..fd76ada 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -631,7 +631,8 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       }
     }
     if (tbl.isNonNative() && !AlterTableType.NON_NATIVE_TABLE_ALLOWED.contains(op)) {
-      throw new SemanticException(ErrorMsg.ALTER_TABLE_NON_NATIVE.getMsg(tbl.getTableName()));
+      throw new SemanticException(ErrorMsg.ALTER_TABLE_NON_NATIVE.format(
+          AlterTableType.NON_NATIVE_TABLE_ALLOWED.toString(), tbl.getTableName()));
     }
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExportSemanticAnalyzer.java
index 83c0d2b..8c82379 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExportSemanticAnalyzer.java
@@ -104,7 +104,8 @@ public class ExportSemanticAnalyzer extends BaseSemanticAnalyzer {
     String tmpPath = stripQuotes(toTree.getText());
     // All parsing is done, we're now good to start the export process
     TableExport.Paths exportPaths =
-        new TableExport.Paths(ErrorMsg.INVALID_PATH.getMsg(ast), tmpPath, conf, false);
+        new TableExport.Paths(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_PATH.getMsg(), ast), tmpPath, conf, false);
     // Note: this tableExport is actually never used other than for auth, and another one is
     //       created when the task is executed. So, we don't care about the correct MM state here.
     TableExport.AuthEntities authEntities = new TableExport(
@@ -118,7 +119,7 @@ public class ExportSemanticAnalyzer extends BaseSemanticAnalyzer {
         ts == null ? null : ts.getTableName(), mmCtx);
       // Configure export work
     ExportWork exportWork = new ExportWork(exportRootDirName, ts, replicationSpec,
-        ErrorMsg.INVALID_PATH.getMsg(ast), acidTableName, mmCtx);
+        ASTErrorUtils.getMsg(ErrorMsg.INVALID_PATH.getMsg(), ast), acidTableName, mmCtx);
     // Create an export task and add it as a root task
     return TaskFactory.get(exportWork);
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/FileSinkProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/FileSinkProcessor.java
index 71565bb..e706369 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/FileSinkProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/FileSinkProcessor.java
@@ -24,7 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 
 /**
@@ -32,13 +32,13 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
  * processing.
  *
  */
-public class FileSinkProcessor implements NodeProcessor {
+public class FileSinkProcessor implements SemanticNodeProcessor {
 
   static final private Logger LOG = LoggerFactory.getLogger(FileSinkProcessor.class.getName());
 
   @Override
   public Object process(Node nd, Stack<Node> stack,
-      NodeProcessorCtx procCtx, Object... nodeOutputs)
+                        NodeProcessorCtx procCtx, Object... nodeOutputs)
       throws SemanticException {
 
     GenTezProcContext context = (GenTezProcContext) procCtx;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java
index 553b2b3..5f39e7e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.ql.parse;
 import java.util.List;
 
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 
 /**
@@ -35,7 +35,7 @@ public class GenMapRedWalker extends DefaultGraphWalker {
    * @param disp
    *          the dispatcher to be called for each node visited
    */
-  public GenMapRedWalker(Dispatcher disp) {
+  public GenMapRedWalker(SemanticDispatcher disp) {
     super(disp);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
index 2c0d21a..6042c09 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.hive.ql.plan.TezEdgeProperty;
 import org.apache.hadoop.hive.ql.plan.TezWork;
 import org.apache.hadoop.hive.ql.plan.UnionWork;
 
-import java.io.Serializable;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
@@ -61,7 +60,7 @@ import java.util.concurrent.atomic.AtomicInteger;
  * to break them into TezTasks.
  *
  */
-public class GenTezProcContext implements NodeProcessorCtx{
+public class GenTezProcContext implements NodeProcessorCtx {
 
   public final ParseContext parseContext;
   public final HiveConf conf;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
index 9a7d48d..78be42e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
@@ -751,7 +751,7 @@ public class GenTezUtils {
     HashMap<ExprNodeDesc, ExprNodeDesc> childParentMapping = new HashMap<ExprNodeDesc, ExprNodeDesc>();
   }
 
-  private static class DynamicValuePredicateProc implements NodeProcessor {
+  private static class DynamicValuePredicateProc implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -777,10 +777,10 @@ public class GenTezUtils {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack. The dispatcher
     // generates the plan from the operator tree
-    Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> exprRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     exprRules.put(new RuleRegExp("R1", ExprNodeDynamicValueDesc.class.getName() + "%"), new DynamicValuePredicateProc());
-    Dispatcher disp = new DefaultRuleDispatcher(null, exprRules, ctx);
-    GraphWalker egw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, exprRules, ctx);
+    SemanticGraphWalker egw = new DefaultGraphWalker(disp);
     List<Node> startNodes = new ArrayList<Node>();
     startNodes.add(pred);
 
@@ -826,7 +826,7 @@ public class GenTezUtils {
     }
   }
 
-  public static class DynamicPartitionPrunerProc implements NodeProcessor {
+  public static class DynamicPartitionPrunerProc implements SemanticNodeProcessor {
 
     /**
      * process simply remembers all the dynamic partition pruning expressions
@@ -855,14 +855,14 @@ public class GenTezUtils {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack. The dispatcher
     // generates the plan from the operator tree
-    Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> exprRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     exprRules.put(new RuleRegExp("R1", ExprNodeDynamicListDesc.class.getName() + "%"),
         new DynamicPartitionPrunerProc());
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, exprRules, ctx);
-    GraphWalker egw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, exprRules, ctx);
+    SemanticGraphWalker egw = new DefaultGraphWalker(disp);
 
     List<Node> startNodes = new ArrayList<Node>();
     startNodes.add(pred);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
index 002dafa..0b4cb31 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
 import org.apache.hadoop.hive.ql.optimizer.ReduceSinkMapJoinProc;
@@ -58,7 +58,7 @@ import org.slf4j.LoggerFactory;
  * a new execution unit.) and break the operators into work
  * and tasks along the way.
  */
-public class GenTezWork implements NodeProcessor {
+public class GenTezWork implements SemanticNodeProcessor {
 
   private static final Logger LOG = LoggerFactory.getLogger(GenTezWork.class.getName());
 
@@ -70,7 +70,7 @@ public class GenTezWork implements NodeProcessor {
 
   @Override
   public Object process(Node nd, Stack<Node> stack,
-      NodeProcessorCtx procContext, Object... nodeOutputs)
+                        NodeProcessorCtx procContext, Object... nodeOutputs)
       throws SemanticException {
 
     GenTezProcContext context = (GenTezProcContext) procContext;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWorkWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWorkWalker.java
index 8d796e4..ac23710 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWorkWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWorkWalker.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -45,7 +45,7 @@ public class GenTezWorkWalker extends DefaultGraphWalker {
    * @param ctx the context where we'll set the current root operator
    *
    */
-  public GenTezWorkWalker(Dispatcher disp, GenTezProcContext ctx) {
+  public GenTezWorkWalker(SemanticDispatcher disp, GenTezProcContext ctx) {
     super(disp);
     this.ctx = ctx;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
index 94923e3..58b2615 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
@@ -164,7 +164,8 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
     // local mode implies that scheme should be "file"
     // we can change this going forward
     if (isLocal && !fromURI.getScheme().equals("file")) {
-      throw new SemanticException(ErrorMsg.ILLEGAL_PATH.getMsg(fromTree,
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.ILLEGAL_PATH.getMsg(), fromTree,
           "Source file system should be \"file\" if \"local\" is specified"));
     }
 
@@ -172,7 +173,8 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
       FileSystem fileSystem = FileSystem.get(fromURI, conf);
       srcs = matchFilesOrDir(fileSystem, new Path(fromURI));
       if (srcs == null || srcs.length == 0) {
-        throw new SemanticException(ErrorMsg.INVALID_PATH.getMsg(fromTree,
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_PATH.getMsg(), fromTree,
             "No files matching path " + fromURI));
       }
 
@@ -212,8 +214,9 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
       }
     } catch (IOException e) {
       // Has to use full name to make sure it does not conflict with
-      // org.apache.commons.lang3.StringUtils
-      throw new SemanticException(ErrorMsg.INVALID_PATH.getMsg(fromTree), e);
+      // org.apache.commons.lang.StringUtils
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_PATH.getMsg(), fromTree), e);
     }
 
     return Lists.newArrayList(srcs);
@@ -280,8 +283,8 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
       String fromPath = stripQuotes(fromTree.getText());
       fromURI = initializeFromURI(fromPath, isLocal);
     } catch (IOException | URISyntaxException e) {
-      throw new SemanticException(ErrorMsg.INVALID_PATH.getMsg(fromTree, e
-          .getMessage()), e);
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_PATH.getMsg(), fromTree, e.getMessage()), e);
     }
 
     // initialize destination table/partition
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java
index bcba4d7..c674e04 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.parse;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -48,11 +47,11 @@ import org.apache.hadoop.hive.ql.exec.mr.MapRedTask;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.optimizer.GenMRFileSink1;
@@ -307,7 +306,7 @@ public class MapReduceCompiler extends TaskCompiler {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack.
     // The dispatcher generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp(new String("R1"),
         TableScanOperator.getOperatorName() + "%"),
         new GenMRTableScan1());
@@ -332,10 +331,10 @@ public class MapReduceCompiler extends TaskCompiler {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(new GenMROperator(), opRules,
+    SemanticDispatcher disp = new DefaultRuleDispatcher(new GenMROperator(), opRules,
         procCtx);
 
-    GraphWalker ogw = new GenMapRedWalker(disp);
+    SemanticGraphWalker ogw = new GenMapRedWalker(disp);
     ArrayList<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pCtx.getTopOps().values());
     ogw.startWalking(topNodes, null);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java
index 13f9243..3ffdcec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MergeSemanticAnalyzer.java
@@ -521,7 +521,7 @@ public class MergeSemanticAnalyzer extends RewriteSemanticAnalyzer {
   }
 
   private static String addParseInfo(ASTNode n) {
-    return " at " + ErrorMsg.renderPosition(n);
+    return " at " + ASTErrorUtils.renderPosition(n);
   }
 
   private boolean isAliased(ASTNode n) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java
index ee28b1d..7aebbec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java
@@ -41,7 +41,7 @@ import com.google.common.collect.Multimap;
  * to do some additional optimizations on it.
  *
  */
-public class OptimizeTezProcContext implements NodeProcessorCtx{
+public class OptimizeTezProcContext implements NodeProcessorCtx {
 
   public final ParseContext parseContext;
   public final HiveConf conf;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
index 31c5c4b..ab28d78 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
@@ -22,16 +22,13 @@ import java.util.List;
 import java.util.Set;
 import java.util.Stack;
 
-import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
-import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -48,7 +45,7 @@ import org.apache.hadoop.mapred.InputFormat;
  * (normal, no scan.) The plan at this point will be a single
  * table scan operator.
  */
-public class ProcessAnalyzeTable implements NodeProcessor {
+public class ProcessAnalyzeTable implements SemanticNodeProcessor {
 
   private static final Logger LOG = LoggerFactory.getLogger(ProcessAnalyzeTable.class.getName());
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java
index d596abc..9132b89 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSubquerySemanticException;
 import org.apache.hadoop.hive.ql.parse.SubQueryDiagnostic.QBSubQueryRewrite;
 import org.apache.hadoop.hive.ql.parse.SubQueryUtils.ISubQueryJoinInfo;
@@ -258,7 +258,7 @@ public class QBSubQuery implements ISubQueryJoinInfo {
     RowResolver parentQueryRR;
     boolean forHavingClause;
     String parentQueryNewAlias;
-    NodeProcessor defaultExprProcessor;
+    SemanticNodeProcessor defaultExprProcessor;
     Stack<Node> stack;
 
     ConjunctAnalyzer(RowResolver parentQueryRR,
@@ -549,8 +549,9 @@ public class QBSubQuery implements ISubQueryJoinInfo {
             operator.getType() != SubQueryType.NOT_EXISTS &&
             selectClause.getChildCount() - selectExprStart > 1 ) {
       subQueryAST.setOrigin(originalSQASTOrigin);
-      throw new SemanticException(ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(
-              subQueryAST, "SubQuery can contain only 1 item in Select List."));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(),
+          subQueryAST, "SubQuery can contain only 1 item in Select List."));
     }
 
     boolean hasAggreateExprs = false;
@@ -606,8 +607,9 @@ public class QBSubQuery implements ISubQueryJoinInfo {
      * Restriction.14.h :: Correlated Sub Queries cannot contain Windowing clauses.
      */
     if (  hasWindowing && hasCorrelation) {
-      throw new CalciteSubquerySemanticException(ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(
-              subQueryAST, "Correlated Sub Queries cannot contain Windowing clauses."));
+      throw new CalciteSubquerySemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(),
+          subQueryAST, "Correlated Sub Queries cannot contain Windowing clauses."));
     }
 
     /*
@@ -632,7 +634,9 @@ public class QBSubQuery implements ISubQueryJoinInfo {
         if(operator.getType() == SubQueryType.EXISTS
                 || operator.getType() == SubQueryType.NOT_EXISTS) {
           if(hasCorrelation) {
-            throw new CalciteSubquerySemanticException(ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(
+            throw new CalciteSubquerySemanticException(
+                ASTErrorUtils.getMsg(
+                    ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(),
                     subQueryAST,
                     "A predicate on EXISTS/NOT EXISTS SubQuery with implicit Aggregation(no Group By clause) " +
                             "cannot be rewritten."));
@@ -703,7 +707,8 @@ public class QBSubQuery implements ISubQueryJoinInfo {
         operator.getType() != SubQueryType.NOT_EXISTS &&
         selectClause.getChildCount() - selectExprStart > 1 ) {
       subQueryAST.setOrigin(originalSQASTOrigin);
-      throw new SemanticException(ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(),
           subQueryAST, "SubQuery can contain only 1 item in Select List."));
     }
 
@@ -731,7 +736,8 @@ public class QBSubQuery implements ISubQueryJoinInfo {
     if ( operator.getType() == SubQueryType.EXISTS  &&
         containsAggregationExprs &&
         groupbyAddedToSQ ) {
-      throw new SemanticException(ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(),
           subQueryAST,
           "An Exists predicate on SubQuery with implicit Aggregation(no Group By clause) " +
           "cannot be rewritten. (predicate will always return true)."));
@@ -739,7 +745,8 @@ public class QBSubQuery implements ISubQueryJoinInfo {
     if ( operator.getType() == SubQueryType.NOT_EXISTS  &&
         containsAggregationExprs &&
         groupbyAddedToSQ ) {
-      throw new SemanticException(ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(),
           subQueryAST,
           "A Not Exists predicate on SubQuery with implicit Aggregation(no Group By clause) " +
           "cannot be rewritten. (predicate will always return false)."));
@@ -749,7 +756,8 @@ public class QBSubQuery implements ISubQueryJoinInfo {
      * Restriction.14.h :: Correlated Sub Queries cannot contain Windowing clauses.
      */
     if ( containsWindowing && hasCorrelation ) {
-      throw new SemanticException(ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(),
           subQueryAST, "Correlated Sub Queries cannot contain Windowing clauses."));
     }
 
@@ -760,7 +768,8 @@ public class QBSubQuery implements ISubQueryJoinInfo {
     if ( ( operator.getType() == SubQueryType.EXISTS ||
         operator.getType() == SubQueryType.NOT_EXISTS ) &&
         !hasCorrelation ) {
-      throw new SemanticException(ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(),
           subQueryAST, "For Exists/Not Exists operator SubQuery must be Correlated."));
     }
 
@@ -770,7 +779,8 @@ public class QBSubQuery implements ISubQueryJoinInfo {
     ASTNode childAST = (ASTNode) subQueryAST.getFirstChildWithType(type);
     if (childAST == null && errorMsg != null) {
       subQueryAST.setOrigin(originalSQASTOrigin);
-      throw new SemanticException(ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(),
           subQueryAST, errorMsg + " clause is missing in SubQuery."));
     }
     return childAST;
@@ -810,7 +820,8 @@ public class QBSubQuery implements ISubQueryJoinInfo {
                 SubQueryUtils.setQualifiedColumnReferences(parentExpr, tableAlias);
         if (parentExpr == null) {
           subQueryAST.setOrigin(originalSQASTOrigin);
-          throw new SemanticException(ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(),
                   parentQueryExpression,
                   "Correlating expression contains ambiguous column references."));
         }
@@ -935,7 +946,8 @@ public class QBSubQuery implements ISubQueryJoinInfo {
        * Check.12.h :: SubQuery predicates cannot only refer to Outer Query columns.
        */
       if ( conjunct.refersOuterOnly() ) {
-        throw new SemanticException(ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(),
             conjunctAST,
             "SubQuery expression refers to Outer query expressions only."));
       }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
index d68ce1d..276f759 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
@@ -265,7 +265,7 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
               oldReplScope,
               eventFrom,
               eventTo,
-              ErrorMsg.INVALID_PATH.getMsg(ast),
+              ASTErrorUtils.getMsg(ErrorMsg.INVALID_PATH.getMsg(), ast),
               maxEventLimit,
               ctx.getResFile().toUri().toString()
       ), conf);
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 5fcc367..7e389a9 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
@@ -153,8 +153,8 @@ import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 import org.apache.hadoop.hive.ql.io.NullRowsInputFormat;
 import org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
@@ -1015,7 +1015,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     sb.append(" ");
     sb.append(message);
     sb.append(". Error encountered near token '");
-    sb.append(ErrorMsg.getText(ast));
+    sb.append(ASTErrorUtils.getText(ast));
     sb.append("'");
     return sb.toString();
   }
@@ -1097,8 +1097,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     // If the alias is already there then we have a conflict
     if (qb.exists(alias)) {
-      throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(tabref
-          .getChild(aliasIndex)));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(),
+          tabref.getChild(aliasIndex)));
     }
     if (tsampleIndex >= 0) {
       ASTNode sampleClause = (ASTNode) tabref.getChild(tsampleIndex);
@@ -1203,7 +1204,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     // This is a subquery and must have an alias
     if (subq.getChildCount() != 2) {
-      throw new SemanticException(ErrorMsg.NO_SUBQUERY_ALIAS.getMsg(subq));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.NO_SUBQUERY_ALIAS.getMsg(), subq));
     }
     ASTNode subqref = (ASTNode) subq.getChild(0);
     String alias = unescapeIdentifier(subq.getChild(1).getText());
@@ -1215,8 +1217,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     // If the alias is already there then we have a conflict
     if (qb.exists(alias)) {
-      throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(subq
-          .getChild(1)));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(),
+          subq.getChild(1)));
     }
     // Insert this map into the stats
     qb.setSubqAlias(alias, qbexpr);
@@ -1244,7 +1247,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       qName += alias.toLowerCase();
 
       if ( aliasToCTEs.containsKey(qName)) {
-        throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(cte.getChild(1)));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(),
+            cte.getChild(1)));
       }
       aliasToCTEs.put(qName, new CTEClause(qName, cteQry));
     }
@@ -1517,8 +1522,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         // is not supported. Instead, the lateral view must be in a subquery
         // SELECT * FROM (SELECT * FROM src1 LATERAL VIEW udtf() AS myTable) a
         // JOIN src2 ...
-        throw new SemanticException(ErrorMsg.LATERAL_VIEW_WITH_JOIN
-            .getMsg(join));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.LATERAL_VIEW_WITH_JOIN.getMsg(), join));
       } else if (isJoinToken(child)) {
         processJoin(qb, child);
       }
@@ -1559,8 +1564,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       alias = processLateralView(qb, next);
       break;
     default:
-      throw new SemanticException(ErrorMsg.LATERAL_VIEW_INVALID_CHILD
-          .getMsg(lateralView));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.LATERAL_VIEW_INVALID_CHILD.getMsg(), lateralView));
     }
     alias = alias.toLowerCase();
     qb.getParseInfo().addLateralViewForAlias(alias, lateralView);
@@ -1653,7 +1658,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
         // is there a insert in the subquery
         if (qbp.getIsSubQ() && !isTmpFileDest) {
-          throw new SemanticException(ErrorMsg.NO_INSERT_INSUBQUERY.getMsg(ast));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.NO_INSERT_INSUBQUERY.getMsg(), ast));
         }
 
         qbp.setDestForClause(ctx_1.dest, (ASTNode) ast.getChild(0));
@@ -2169,7 +2175,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         }
         ASTNode src = qb.getParseInfo().getSrcForAlias(alias);
         if (null != src) {
-          throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(src));
+          throw new SemanticException(ASTErrorUtils.getMsg(ErrorMsg.INVALID_TABLE.getMsg(), src));
         } else {
           throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(alias));
         }
@@ -2280,8 +2286,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         Class<?> outputFormatClass = ts.tableHandle.getOutputFormatClass();
         if (!ts.tableHandle.isNonNative() &&
             !HiveOutputFormat.class.isAssignableFrom(outputFormatClass)) {
-          throw new SemanticException(ErrorMsg.INVALID_OUTPUT_FORMAT_TYPE
-              .getMsg(ast, "The class is " + outputFormatClass.toString()));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.INVALID_OUTPUT_FORMAT_TYPE.getMsg(),
+              ast, "The class is " + outputFormatClass.toString()));
         }
 
         boolean isTableWrittenTo = qb.getParseInfo().isInsertIntoTable(ts.tableHandle.getDbName(),
@@ -2604,7 +2611,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
             ctx.getViewTokenRewriteStream(viewFullyQualifiedName),
             ctx, db, tabNameToTabObject);
       }
-      Dispatcher nodeOriginDispatcher = new Dispatcher() {
+      SemanticDispatcher nodeOriginDispatcher = new SemanticDispatcher() {
         @Override
         public Object dispatch(Node nd, java.util.Stack<Node> stack,
                                Object... nodeOutputs) {
@@ -2612,7 +2619,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           return null;
         }
       };
-      GraphWalker nodeOriginTagger = new DefaultGraphWalker(
+      SemanticGraphWalker nodeOriginTagger = new DefaultGraphWalker(
           nodeOriginDispatcher);
       nodeOriginTagger.startWalking(java.util.Collections
           .<Node> singleton(viewTree), null);
@@ -2624,7 +2631,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e));
       StringBuilder sb = new StringBuilder();
       sb.append(e.getMessage());
-      ErrorMsg.renderOrigin(sb, viewOrigin);
+      ASTErrorUtils.renderOrigin(sb, viewOrigin);
       throw new SemanticException(sb.toString(), e);
     }
     QBExpr qbexpr = new QBExpr(alias);
@@ -2678,15 +2685,15 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           if (tabAlias == null) {
             tabAlias = opEntry.getKey();
           } else {
-            throw new SemanticException(
-                ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(columnRef.getChild(0)));
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.AMBIGUOUS_TABLE_ALIAS.getMsg(), columnRef.getChild(0)));
           }
         }
       }
     }
     if ( tabAlias == null ) {
-      throw new SemanticException(ErrorMsg.INVALID_TABLE_ALIAS.getMsg(columnRef
-          .getChild(0)));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_TABLE_ALIAS.getMsg(), columnRef.getChild(0)));
     }
     return tabAlias;
   }
@@ -3370,7 +3377,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
        * Restriction.9.m :: disallow nested SubQuery expressions.
        */
       if (qb.getSubQueryPredicateDef() != null  ) {
-        throw new SemanticException(ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(),
             subQueriesInOriginalTree.get(0), "Nested SubQuery expressions are not supported."));
       }
 
@@ -3379,7 +3387,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
        */
       if (subQueriesInOriginalTree.size() > 1 ) {
 
-        throw new SemanticException(ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(),
             subQueriesInOriginalTree.get(1), "Only 1 SubQuery expression is supported."));
       }
 
@@ -3427,7 +3436,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
             subQuery.getOperator().getType() != SubQueryType.NOT_EXISTS &&
             sqRR.getColumnInfos().size() -
                 subQuery.getNumOfCorrelationExprsAddedToSQSelect() > 1 ) {
-          throw new SemanticException(ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.INVALID_SUBQUERY_EXPRESSION.getMsg(),
               subQueryAST, "SubQuery can contain only 1 item in Select List."));
         }
 
@@ -3599,7 +3609,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
     // The table alias should exist
     if (tabAlias != null && !colSrcRR.hasTableAlias(tabAlias)) {
-      throw new SemanticException(ErrorMsg.INVALID_TABLE_ALIAS.getMsg(sel));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_TABLE_ALIAS.getMsg(), sel));
     }
 
     // TODO: Have to put in the support for AS clause
@@ -3607,8 +3618,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     try {
       regex = Pattern.compile(colRegex, Pattern.CASE_INSENSITIVE);
     } catch (PatternSyntaxException e) {
-      throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(sel, e
-          .getMessage()));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_COLUMN.getMsg(), sel, e.getMessage()));
     }
 
     StringBuilder replacementText = new StringBuilder();
@@ -3780,7 +3791,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     if (matched == 0) {
-      throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(sel));
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_COLUMN.getMsg(), sel));
     }
 
     if (unparseTranslator.isEnabled()) {
@@ -4834,7 +4846,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     if (null == result) {
       String reason = "Looking for UDAF Evaluator\"" + aggName
           + "\" with parameters " + originalParameterTypeInfos;
-      throw new SemanticException(ErrorMsg.INVALID_FUNCTION_SIGNATURE.getMsg(
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.INVALID_FUNCTION_SIGNATURE.getMsg(),
           (ASTNode) aggTree.getChild(0), reason));
     }
     return result;
@@ -4979,7 +4992,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       ColumnInfo exprInfo = groupByInputRowResolver.getExpression(grpbyExpr);
 
       if (exprInfo == null) {
-        throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(grpbyExpr));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), grpbyExpr));
       }
 
       groupByKeys.add(new ExprNodeColumnDesc(exprInfo.getType(), exprInfo
@@ -5020,7 +5034,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         ColumnInfo paraExprInfo =
             groupByInputRowResolver.getExpression(paraExpr);
         if (paraExprInfo == null) {
-          throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(paraExpr));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.INVALID_COLUMN.getMsg(), paraExpr));
         }
 
         String paraExpression = paraExprInfo.getInternalName();
@@ -5189,7 +5204,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       ColumnInfo exprInfo = groupByInputRowResolver.getExpression(grpbyExpr);
 
       if (exprInfo == null) {
-        throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(grpbyExpr));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), grpbyExpr));
       }
 
       groupByKeys.add(new ExprNodeColumnDesc(exprInfo));
@@ -5276,8 +5292,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           ColumnInfo paraExprInfo =
               groupByInputRowResolver.getExpression(paraExpr);
           if (paraExprInfo == null) {
-            throw new SemanticException(ErrorMsg.INVALID_COLUMN
-                .getMsg(paraExpr));
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.INVALID_COLUMN.getMsg(),
+                paraExpr));
           }
 
           String paraExpression = paraExprInfo.getInternalName();
@@ -5305,7 +5322,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       } else {
         ColumnInfo paraExprInfo = groupByInputRowResolver.getExpression(value);
         if (paraExprInfo == null) {
-          throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(value));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.INVALID_COLUMN.getMsg(), value));
         }
         String paraExpression = paraExprInfo.getInternalName();
         assert (paraExpression != null);
@@ -5979,7 +5997,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       ASTNode grpbyExpr = grpByExprs.get(i);
       ColumnInfo exprInfo = groupByInputRowResolver2.getExpression(grpbyExpr);
       if (exprInfo == null) {
-        throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(grpbyExpr));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), grpbyExpr));
       }
 
       String expression = exprInfo.getInternalName();
@@ -6013,7 +6032,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       ASTNode value = entry.getValue();
       ColumnInfo paraExprInfo = groupByInputRowResolver2.getExpression(value);
       if (paraExprInfo == null) {
-        throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(value));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), value));
       }
       String paraExpression = paraExprInfo.getInternalName();
       assert (paraExpression != null);
@@ -8401,7 +8421,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     if (!updating(dest) && !deleting(dest) && inColumnCnt != outColumnCnt) {
       String reason = "Table " + dest + " has " + outColumnCnt
           + " columns, but query has " + inColumnCnt + " columns.";
-      throw new SemanticException(ErrorMsg.TARGET_TABLE_COLUMN_MISMATCH.getMsg(
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.TARGET_TABLE_COLUMN_MISMATCH.getMsg(),
           qb.getParseInfo().getDestForClause(dest), reason));
     }
 
@@ -8450,8 +8471,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           if (column == null) {
             String reason = "Cannot convert column " + i + " from "
                 + rowFieldTypeInfo + " to " + tableFieldTypeInfo + ".";
-            throw new SemanticException(ErrorMsg.TARGET_TABLE_COLUMN_MISMATCH
-                .getMsg(qb.getParseInfo().getDestForClause(dest), reason));
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.TARGET_TABLE_COLUMN_MISMATCH.getMsg(),
+                qb.getParseInfo().getDestForClause(dest), reason));
           }
         }
         expressions.add(column);
@@ -8709,8 +8731,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         if (column == null) {
           String reason = "Cannot convert column " + posn + " from "
               + rowFieldTypeInfo + " to " + tableFieldTypeInfo + ".";
-          throw new SemanticException(ErrorMsg.TARGET_TABLE_COLUMN_MISMATCH
-              .getMsg(qb.getParseInfo().getDestForClause(dest), reason));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.TARGET_TABLE_COLUMN_MISMATCH.getMsg(),
+              qb.getParseInfo().getDestForClause(dest), reason));
         }
       }
       expressions.add(column);
@@ -13021,7 +13044,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       ColumnInfo colInfo = input.get(null, tableOrCol);
       String errMsg;
       if (colInfo == null && input.getIsExprResolver()){
-        errMsg = ErrorMsg.NON_KEY_EXPR_IN_GROUPBY.getMsg(expr);
+        errMsg = ASTErrorUtils.getMsg(
+            ErrorMsg.NON_KEY_EXPR_IN_GROUPBY.getMsg(), expr);
       } else {
         errMsg = tcCtx.getError();
       }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SubQueryUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SubQueryUtils.java
index 65bb13e..e7ad914 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SubQueryUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SubQueryUtils.java
@@ -116,8 +116,9 @@ public class SubQueryUtils {
      */
     if (subqueryExprNode.getChildren().size() == 3
         && subqueryExprNode.getChild(2).getType() == HiveParser.TOK_SUBQUERY_EXPR) {
-      throw new CalciteSubquerySemanticException(ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION
-          .getMsg(subqueryExprNode.getChild(2), "SubQuery on left hand side is not supported."));
+      throw new CalciteSubquerySemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(),
+          subqueryExprNode.getChild(2), "SubQuery on left hand side is not supported."));
     }
 
     // avoid subquery restrictions for SOME/ALL for now
@@ -142,9 +143,9 @@ public class SubQueryUtils {
     ASTNode outerQueryExpr = (ASTNode) subqueryExprNode.getChild(2);
 
     if (outerQueryExpr != null && outerQueryExpr.getType() == HiveParser.TOK_SUBQUERY_EXPR) {
-      throw new CalciteSubquerySemanticException(
-          ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(
-              outerQueryExpr, "IN/EXISTS/SOME/ALL subqueries are not allowed in LHS"));
+      throw new CalciteSubquerySemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(),
+          outerQueryExpr, "IN/EXISTS/SOME/ALL subqueries are not allowed in LHS"));
     }
 
     QBSubQuery subQuery = SubQueryUtils.buildSubQuery(qb.getId(), sqIdx, subqueryExprNode,
@@ -259,7 +260,8 @@ public class SubQueryUtils {
       /*
        *  Restriction.7.h :: SubQuery predicates can appear only as top level conjuncts.
        */
-      throw new SemanticException(ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(
+      throw new SemanticException(ASTErrorUtils.getMsg(
+          ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(),
           subQuery, "Only SubQuery expressions that are top level conjuncts are allowed"));
     }
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TableAccessAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TableAccessAnalyzer.java
index f92b350..5ddcd31 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TableAccessAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TableAccessAnalyzer.java
@@ -34,12 +34,12 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -72,7 +72,7 @@ public class TableAccessAnalyzer {
   public TableAccessInfo analyzeTableAccess() throws SemanticException {
 
     // Set up the rules for the graph walker for group by and join operators
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", GroupByOperator.getOperatorName() + "%"),
         new GroupByProcessor(pGraphContext));
     opRules.put(new RuleRegExp("R2", JoinOperator.getOperatorName() + "%"),
@@ -81,8 +81,8 @@ public class TableAccessAnalyzer {
         new JoinProcessor(pGraphContext));
 
     TableAccessCtx tableAccessCtx = new TableAccessCtx();
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, tableAccessCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, tableAccessCtx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes and walk!
     List<Node> topNodes = new ArrayList<Node>();
@@ -92,11 +92,11 @@ public class TableAccessAnalyzer {
     return tableAccessCtx.getTableAccessInfo();
   }
 
-  private NodeProcessor getDefaultProc() {
-    return new NodeProcessor() {
+  private SemanticNodeProcessor getDefaultProc() {
+    return new SemanticNodeProcessor() {
       @Override
       public Object process(Node nd, Stack<Node> stack,
-          NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
+                            NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException {
         return null;
       }
     };
@@ -105,7 +105,7 @@ public class TableAccessAnalyzer {
   /**
    * Processor for GroupBy operator
    */
-  public class GroupByProcessor implements NodeProcessor {
+  public class GroupByProcessor implements SemanticNodeProcessor {
     protected ParseContext pGraphContext;
 
     public GroupByProcessor(ParseContext pGraphContext) {
@@ -152,7 +152,7 @@ public class TableAccessAnalyzer {
   /**
    * Processor for Join operator.
    */
-  public class JoinProcessor implements NodeProcessor {
+  public class JoinProcessor implements SemanticNodeProcessor {
     protected ParseContext pGraphContext;
 
     public JoinProcessor(ParseContext pGraphContext) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
index f1ebc2f..31735c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
@@ -70,14 +70,14 @@ import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.lib.CompositeProcessor;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.ForwardWalker;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.PreOrderOnceWalker;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
 import org.apache.hadoop.hive.ql.metadata.Hive;
@@ -461,7 +461,7 @@ public class TezCompiler extends TaskCompiler {
 
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack.
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("Set parallelism - ReduceSink",
         ReduceSinkOperator.getOperatorName() + "%"),
         new SetReducerParallelism());
@@ -475,10 +475,10 @@ public class TezCompiler extends TaskCompiler {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(procCtx.parseContext.getTopOps().values());
-    GraphWalker ogw = new ForwardWalker(disp);
+    SemanticGraphWalker ogw = new ForwardWalker(disp);
     ogw.startWalking(topNodes, null);
   }
 
@@ -557,7 +557,7 @@ public class TezCompiler extends TaskCompiler {
 
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack.
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(
         new RuleRegExp("Remove dynamic pruning by size",
         AppMasterEventOperator.getOperatorName() + "%"),
@@ -565,10 +565,10 @@ public class TezCompiler extends TaskCompiler {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(procCtx.parseContext.getTopOps().values());
-    GraphWalker ogw = new ForwardWalker(disp);
+    SemanticGraphWalker ogw = new ForwardWalker(disp);
     ogw.startWalking(topNodes, null);
   }
 
@@ -583,17 +583,17 @@ public class TezCompiler extends TaskCompiler {
     Deque<Operator<?>> deque = new LinkedList<Operator<?>>();
     deque.addAll(procCtx.parseContext.getTopOps().values());
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(
         new RuleRegExp(new String("Dynamic Partition Pruning"), FilterOperator.getOperatorName()
             + "%"), new DynamicPartitionPruningOptimization());
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(procCtx.parseContext.getTopOps().values());
-    GraphWalker ogw = new ForwardWalker(disp);
+    SemanticGraphWalker ogw = new ForwardWalker(disp);
     ogw.startWalking(topNodes, null);
   }
 
@@ -614,7 +614,7 @@ public class TezCompiler extends TaskCompiler {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack.
     // The dispatcher generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("Split Work - ReduceSink",
         ReduceSinkOperator.getOperatorName() + "%"),
         genTezWork);
@@ -647,10 +647,10 @@ public class TezCompiler extends TaskCompiler {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pCtx.getTopOps().values());
-    GraphWalker ogw = new GenTezWorkWalker(disp, procCtx);
+    SemanticGraphWalker ogw = new GenTezWorkWalker(disp, procCtx);
     ogw.startWalking(topNodes, null);
 
     // we need to specify the reserved memory for each work that contains Map Join
@@ -819,7 +819,7 @@ public class TezCompiler extends TaskCompiler {
     HashMap<CommonMergeJoinOperator, TableScanOperator> JoinOpToTsOpMap = new HashMap<CommonMergeJoinOperator, TableScanOperator>();
   }
 
-  private static class SMBJoinOpProc implements NodeProcessor {
+  private static class SMBJoinOpProc implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -833,7 +833,7 @@ public class TezCompiler extends TaskCompiler {
 
   private static void removeSemijoinOptimizationFromSMBJoins(
           OptimizeTezProcContext procCtx) throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(
             new RuleRegExp("R1", TableScanOperator.getOperatorName() + "%" +
                     ".*" + TezDummyStoreOperator.getOperatorName() + "%" +
@@ -842,10 +842,10 @@ public class TezCompiler extends TaskCompiler {
 
     SMBJoinOpProcContext ctx = new SMBJoinOpProcContext();
     // The dispatcher finds SMB and if there is semijoin optimization before it, removes it.
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, ctx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, ctx);
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(procCtx.parseContext.getTopOps().values());
-    GraphWalker ogw = new PreOrderOnceWalker(disp);
+    SemanticGraphWalker ogw = new PreOrderOnceWalker(disp);
     ogw.startWalking(topNodes, null);
 
     List<TableScanOperator> tsOps = new ArrayList<>();
@@ -943,7 +943,7 @@ public class TezCompiler extends TaskCompiler {
 
   private void removeSemiJoinIfNoStats(OptimizeTezProcContext procCtx)
           throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(
             new RuleRegExp("R1", GroupByOperator.getOperatorName() + "%" +
                     ReduceSinkOperator.getOperatorName() + "%" +
@@ -952,14 +952,14 @@ public class TezCompiler extends TaskCompiler {
             new SemiJoinRemovalProc(true, false));
     SemiJoinRemovalContext ctx =
         new SemiJoinRemovalContext(procCtx.parseContext);
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, ctx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, ctx);
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(procCtx.parseContext.getTopOps().values());
-    GraphWalker ogw = new PreOrderOnceWalker(disp);
+    SemanticGraphWalker ogw = new PreOrderOnceWalker(disp);
     ogw.startWalking(topNodes, null);
   }
 
-  private static class CollectAll implements NodeProcessor {
+  private static class CollectAll implements SemanticNodeProcessor {
     private PlanMapper planMapper;
 
     @Override
@@ -988,7 +988,7 @@ public class TezCompiler extends TaskCompiler {
     }
   }
 
-  private static class MarkRuntimeStatsAsIncorrect implements NodeProcessor {
+  private static class MarkRuntimeStatsAsIncorrect implements SemanticNodeProcessor {
 
     private PlanMapper planMapper;
 
@@ -1045,7 +1045,7 @@ public class TezCompiler extends TaskCompiler {
   }
 
   private void markOperatorsWithUnstableRuntimeStats(OptimizeTezProcContext procCtx) throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(
         new RuleRegExp("R1",
             ReduceSinkOperator.getOperatorName() + "%"),
@@ -1058,14 +1058,14 @@ public class TezCompiler extends TaskCompiler {
         new RuleRegExp("R3",
             TableScanOperator.getOperatorName() + "%"),
         new MarkRuntimeStatsAsIncorrect());
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(procCtx.parseContext.getTopOps().values());
-    GraphWalker ogw = new PreOrderOnceWalker(disp);
+    SemanticGraphWalker ogw = new PreOrderOnceWalker(disp);
     ogw.startWalking(topNodes, null);
   }
 
-  private class SemiJoinRemovalProc implements NodeProcessor {
+  private class SemiJoinRemovalProc implements SemanticNodeProcessor {
 
     private final boolean removeBasedOnStats;
     private final boolean removeRedundant;
@@ -1188,7 +1188,7 @@ public class TezCompiler extends TaskCompiler {
     return "bloom_filter".equals(agg.getGenericUDAFName());
   }
 
-  private static class DynamicPruningRemovalRedundantProc implements NodeProcessor {
+  private static class DynamicPruningRemovalRedundantProc implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -1233,7 +1233,7 @@ public class TezCompiler extends TaskCompiler {
 
   private void removeRedundantSemijoinAndDpp(OptimizeTezProcContext procCtx)
       throws SemanticException {
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<>();
     opRules.put(
         new RuleRegExp("R1", GroupByOperator.getOperatorName() + "%" +
             ReduceSinkOperator.getOperatorName() + "%" +
@@ -1248,10 +1248,10 @@ public class TezCompiler extends TaskCompiler {
     // Gather
     SemiJoinRemovalContext ctx =
         new SemiJoinRemovalContext(procCtx.parseContext);
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, ctx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, ctx);
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(procCtx.parseContext.getTopOps().values());
-    GraphWalker ogw = new PreOrderOnceWalker(disp);
+    SemanticGraphWalker ogw = new PreOrderOnceWalker(disp);
     ogw.startWalking(topNodes, null);
 
     // Remove
@@ -1286,7 +1286,7 @@ public class TezCompiler extends TaskCompiler {
       return;
     }
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(
         new RuleRegExp("Top n key optimization", ReduceSinkOperator.getOperatorName() + "%"),
         new TopNKeyProcessor(HiveConf.getIntVar(procCtx.conf, HiveConf.ConfVars.HIVE_MAX_TOPN_ALLOWED)));
@@ -1297,10 +1297,10 @@ public class TezCompiler extends TaskCompiler {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(procCtx.parseContext.getTopOps().values());
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
     ogw.startWalking(topNodes, null);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UnionProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UnionProcessor.java
index 095d1d9..3736f42 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UnionProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UnionProcessor.java
@@ -24,7 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 
 /**
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
  * processing.
  *
  */
-public class UnionProcessor implements NodeProcessor {
+public class UnionProcessor implements SemanticNodeProcessor {
 
   static final private Logger LOG = LoggerFactory.getLogger(UnionProcessor.class.getName());
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
index 767e4cb..f6b3ec0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.ReduceWork;
 import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty;
 
-import java.io.Serializable;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
index 4fcc0c2..dbd4ab1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -56,7 +56,7 @@ import com.google.common.base.Preconditions;
  *
  * Cloned from GenTezWork.
  */
-public class GenSparkWork implements NodeProcessor {
+public class GenSparkWork implements SemanticNodeProcessor {
   static final private Logger LOG = LoggerFactory.getLogger(GenSparkWork.class.getName());
 
   // instance of shared utils
@@ -71,7 +71,7 @@ public class GenSparkWork implements NodeProcessor {
 
   @Override
   public Object process(Node nd, Stack<Node> stack,
-      NodeProcessorCtx procContext, Object... nodeOutputs) throws SemanticException {
+                        NodeProcessorCtx procContext, Object... nodeOutputs) throws SemanticException {
     GenSparkProcContext context = (GenSparkProcContext) procContext;
 
     Preconditions.checkArgument(context != null,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java
index ae61463..a1c385c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java
@@ -24,7 +24,7 @@ import java.util.List;
 
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
@@ -45,7 +45,7 @@ public class GenSparkWorkWalker extends DefaultGraphWalker {
    * @param ctx the context where we'll set the current root operator
    *
    */
-  public GenSparkWorkWalker(Dispatcher disp, GenSparkProcContext ctx) {
+  public GenSparkWorkWalker(SemanticDispatcher disp, GenSparkProcContext ctx) {
     super(disp);
     this.ctx = ctx;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
index 24429b4..09fa145 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.parse.spark;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -50,14 +49,14 @@ import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.lib.CompositeProcessor;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.ForwardWalker;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TypeRule;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
@@ -162,7 +161,7 @@ public class SparkCompiler extends TaskCompiler {
 
   private void runRemoveDynamicPruning(OptimizeSparkProcContext procCtx) throws SemanticException {
     ParseContext pCtx = procCtx.getParseContext();
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
     opRules.put(new RuleRegExp("Disabling Dynamic Partition Pruning",
         SparkPartitionPruningSinkOperator.getOperatorName() + "%"),
@@ -170,8 +169,8 @@ public class SparkCompiler extends TaskCompiler {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -297,7 +296,7 @@ public class SparkCompiler extends TaskCompiler {
     }
 
     ParseContext parseContext = procCtx.getParseContext();
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(
         new RuleRegExp(new String("Dynamic Partition Pruning"),
             FilterOperator.getOperatorName() + "%"),
@@ -305,8 +304,8 @@ public class SparkCompiler extends TaskCompiler {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
-    GraphWalker ogw = new ForwardWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticGraphWalker ogw = new ForwardWalker(disp);
 
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(parseContext.getTopOps().values());
@@ -315,15 +314,15 @@ public class SparkCompiler extends TaskCompiler {
 
   private void runSetReducerParallelism(OptimizeSparkProcContext procCtx) throws SemanticException {
     ParseContext pCtx = procCtx.getParseContext();
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("Set parallelism - ReduceSink",
             ReduceSinkOperator.getOperatorName() + "%"),
         new SetSparkReducerParallelism(pCtx.getConf()));
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
-    GraphWalker ogw = new PreOrderWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticGraphWalker ogw = new PreOrderWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -333,7 +332,7 @@ public class SparkCompiler extends TaskCompiler {
 
   private void runJoinOptimizations(OptimizeSparkProcContext procCtx) throws SemanticException {
     ParseContext pCtx = procCtx.getParseContext();
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
 
     opRules.put(new TypeRule(JoinOperator.class), new SparkJoinOptimizer(pCtx));
 
@@ -341,8 +340,8 @@ public class SparkCompiler extends TaskCompiler {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
@@ -376,13 +375,13 @@ public class SparkCompiler extends TaskCompiler {
     // -------------------------------- First Pass ---------------------------------- //
     // Identify SparkPartitionPruningSinkOperators, and break OP tree if necessary
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("Clone OP tree for PartitionPruningSink",
             SparkPartitionPruningSinkOperator.getOperatorName() + "%"),
         new SplitOpTreeForDPP());
 
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
-    GraphWalker ogw = new GenSparkWorkWalker(disp, procCtx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticGraphWalker ogw = new GenSparkWorkWalker(disp, procCtx);
 
     List<Node> topNodes = new ArrayList<Node>();
     topNodes.addAll(pCtx.getTopOps().values());
@@ -440,7 +439,7 @@ public class SparkCompiler extends TaskCompiler {
     throws SemanticException {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack. The dispatcher generates the plan from the operator tree
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     GenSparkWork genSparkWork = new GenSparkWork(GenSparkUtils.getUtils());
 
     opRules.put(new RuleRegExp("Split Work - ReduceSink",
@@ -460,7 +459,7 @@ public class SparkCompiler extends TaskCompiler {
         new SparkProcessAnalyzeTable(GenSparkUtils.getUtils()));
 
     opRules.put(new RuleRegExp("Remember union", UnionOperator.getOperatorName() + "%"),
-        new NodeProcessor() {
+        new SemanticNodeProcessor() {
           @Override
           public Object process(Node n, Stack<Node> s,
                                 NodeProcessorCtx procCtx, Object... os) throws SemanticException {
@@ -489,7 +488,7 @@ public class SparkCompiler extends TaskCompiler {
      * the MapWork later on.
      */
     opRules.put(new TypeRule(SMBMapJoinOperator.class),
-      new NodeProcessor() {
+      new SemanticNodeProcessor() {
         @Override
         public Object process(Node currNode, Stack<Node> stack,
                               NodeProcessorCtx procCtx, Object... os) throws SemanticException {
@@ -517,8 +516,8 @@ public class SparkCompiler extends TaskCompiler {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
-    GraphWalker ogw = new GenSparkWorkWalker(disp, procCtx);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    SemanticGraphWalker ogw = new GenSparkWorkWalker(disp, procCtx);
     ogw.startWalking(topNodes, null);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java
index 4414250..c932e27 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java
@@ -24,7 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
  * FileSinkProcessor handles addition of merge, move and stats tasks for filesinks.
  * Cloned from tez's FileSinkProcessor.
  */
-public class SparkFileSinkProcessor implements NodeProcessor {
+public class SparkFileSinkProcessor implements SemanticNodeProcessor {
   private static final Logger LOGGER = LoggerFactory.getLogger(SparkFileSinkProcessor.class.getName());
 
   /*
@@ -43,7 +43,7 @@ public class SparkFileSinkProcessor implements NodeProcessor {
    */
   @Override
   public Object process(Node nd, Stack<Node> stack,
-      NodeProcessorCtx procCtx, Object... nodeOutputs)
+                        NodeProcessorCtx procCtx, Object... nodeOutputs)
       throws SemanticException {
 
     GenSparkProcContext context = (GenSparkProcContext) procCtx;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
index a9ba302..4a836bc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
@@ -22,17 +22,13 @@ import java.util.List;
 import java.util.Set;
 import java.util.Stack;
 
-import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -56,7 +52,7 @@ import com.google.common.base.Preconditions;
  *
  * Cloned from Tez ProcessAnalyzeTable.
  */
-public class SparkProcessAnalyzeTable implements NodeProcessor {
+public class SparkProcessAnalyzeTable implements SemanticNodeProcessor {
   private static final Logger LOGGER = LoggerFactory.getLogger(SparkProcessAnalyzeTable.class.getName());
 
   // shared plan utils for spark
@@ -72,7 +68,7 @@ public class SparkProcessAnalyzeTable implements NodeProcessor {
   @SuppressWarnings("unchecked")
   @Override
   public Object process(Node nd, Stack<Node> stack,
-      NodeProcessorCtx procContext, Object... nodeOutputs) throws SemanticException {
+                        NodeProcessorCtx procContext, Object... nodeOutputs) throws SemanticException {
     GenSparkProcContext context = (GenSparkProcContext) procContext;
 
     TableScanOperator tableScan = (TableScanOperator) nd;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java
index b9c2c1c..e6499f5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 
@@ -76,7 +76,7 @@ import com.google.common.base.Preconditions;
  * For MapJoinOperator, this optimizer will not do anything - it should be executed within
  * the same SparkTask.
  */
-public class SplitOpTreeForDPP implements NodeProcessor {
+public class SplitOpTreeForDPP implements SemanticNodeProcessor {
 
   @Override
   public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/JoinCondTypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/JoinCondTypeCheckProcFactory.java
index d5652f7..319bae6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/JoinCondTypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/JoinCondTypeCheckProcFactory.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.parse.ASTErrorUtils;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
@@ -71,7 +72,8 @@ public class JoinCondTypeCheckProcFactory<T> extends TypeCheckProcFactory<T> {
       ASTNode parent = stack.size() > 1 ? (ASTNode) stack.get(stack.size() - 2) : null;
 
       if (expr.getType() != HiveParser.TOK_TABLE_OR_COL) {
-        ctx.setError(ErrorMsg.INVALID_COLUMN.getMsg(expr), expr);
+        ctx.setError(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), expr), expr);
         return null;
       }
 
@@ -89,7 +91,8 @@ public class JoinCondTypeCheckProcFactory<T> extends TypeCheckProcFactory<T> {
         return null;
       } else {
         // Qualified column access for which table was not found
-        throw new SemanticException(ErrorMsg.INVALID_TABLE_ALIAS.getMsg(expr));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_TABLE_ALIAS.getMsg(), expr));
       }
     }
 
@@ -102,7 +105,8 @@ public class JoinCondTypeCheckProcFactory<T> extends TypeCheckProcFactory<T> {
       }
 
       if (tblAliasCnt > 1) {
-        throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_OR_COLUMN.getMsg(expr));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.AMBIGUOUS_TABLE_OR_COLUMN.getMsg(), expr));
       }
 
       return (tblAliasCnt == 1) ? true : false;
@@ -117,7 +121,8 @@ public class JoinCondTypeCheckProcFactory<T> extends TypeCheckProcFactory<T> {
         tmp = rr.get(tabName, colAlias);
         if (tmp != null) {
           if (cInfoToRet != null) {
-            throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_OR_COLUMN.getMsg(expr));
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.AMBIGUOUS_TABLE_OR_COLUMN.getMsg(), expr));
           }
           cInfoToRet = tmp;
         }
@@ -163,7 +168,8 @@ public class JoinCondTypeCheckProcFactory<T> extends TypeCheckProcFactory<T> {
           exprFactory.getConstantValue((T) nodeOutputs[1]).toString(), expr);
 
       if (colInfo == null) {
-        ctx.setError(ErrorMsg.INVALID_COLUMN.getMsg(expr.getChild(1)), expr);
+        ctx.setError(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), expr.getChild(1)), expr);
         return null;
       }
       ColumnInfo newColumnInfo = new ColumnInfo(colInfo);
@@ -180,7 +186,8 @@ public class JoinCondTypeCheckProcFactory<T> extends TypeCheckProcFactory<T> {
         tmp = rr.get(tabName, colAlias);
         if (tmp != null) {
           if (cInfoToRet != null) {
-            throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_OR_COLUMN.getMsg(expr));
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.AMBIGUOUS_TABLE_OR_COLUMN.getMsg(), expr));
           }
           cInfoToRet = tmp;
         }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java
index d8c3085..4615fc5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java
@@ -37,15 +37,16 @@ import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.lib.CostLessRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.SubqueryExpressionWalker;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSubquerySemanticException;
+import org.apache.hadoop.hive.ql.parse.ASTErrorUtils;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
@@ -183,7 +184,7 @@ public class TypeCheckProcFactory<T> {
     // the operator stack. The dispatcher
     // generates the plan from the operator tree
 
-    SetMultimap<Integer, NodeProcessor> astNodeToProcessor = HashMultimap.create();
+    SetMultimap<Integer, SemanticNodeProcessor> astNodeToProcessor = HashMultimap.create();
     astNodeToProcessor.put(HiveParser.TOK_NULL, getNullExprProcessor());
 
     astNodeToProcessor.put(HiveParser.Number, getNumExprProcessor());
@@ -226,9 +227,9 @@ public class TypeCheckProcFactory<T> {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new CostLessRuleDispatcher(getDefaultExprProcessor(),
+    SemanticDispatcher disp = new CostLessRuleDispatcher(getDefaultExprProcessor(),
         astNodeToProcessor, tcCtx);
-    GraphWalker ogw = new SubqueryExpressionWalker(disp);
+    SemanticGraphWalker ogw = new SubqueryExpressionWalker(disp);
 
     // Create a list of top nodes
     ArrayList<Node> topNodes = Lists.<Node>newArrayList(expr);
@@ -255,7 +256,7 @@ public class TypeCheckProcFactory<T> {
   /**
    * Processor for processing NULL expression.
    */
-  public class NullExprProcessor implements NodeProcessor {
+  public class NullExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -287,7 +288,7 @@ public class TypeCheckProcFactory<T> {
   /**
    * Processor for processing numeric constants.
    */
-  public class NumExprProcessor implements NodeProcessor {
+  public class NumExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -337,7 +338,8 @@ public class TypeCheckProcFactory<T> {
         // do nothing here, we will throw an exception in the following block
       }
       if (result == null) {
-        throw new SemanticException(ErrorMsg.INVALID_NUMERICAL_CONSTANT.getMsg(expr));
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_NUMERICAL_CONSTANT.getMsg(), expr));
       }
       return result;
     }
@@ -356,7 +358,7 @@ public class TypeCheckProcFactory<T> {
   /**
    * Processor for processing string constants.
    */
-  public class StrExprProcessor implements NodeProcessor {
+  public class StrExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -413,7 +415,7 @@ public class TypeCheckProcFactory<T> {
   /**
    * Processor for boolean constants.
    */
-  public class BoolExprProcessor implements NodeProcessor {
+  public class BoolExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -458,7 +460,7 @@ public class TypeCheckProcFactory<T> {
   /**
    * Processor for date constants.
    */
-  public class DateTimeExprProcessor implements NodeProcessor {
+  public class DateTimeExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -513,7 +515,7 @@ public class TypeCheckProcFactory<T> {
   /**
    * Processor for interval constants.
    */
-  public class IntervalExprProcessor implements NodeProcessor {
+  public class IntervalExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -572,7 +574,7 @@ public class TypeCheckProcFactory<T> {
   /**
    * Processor for table columns.
    */
-  public class ColumnExprProcessor implements NodeProcessor {
+  public class ColumnExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -591,12 +593,14 @@ public class TypeCheckProcFactory<T> {
       ASTNode parent = stack.size() > 1 ? (ASTNode) stack.get(stack.size() - 2) : null;
       RowResolver input = ctx.getInputRR();
       if (input == null) {
-        ctx.setError(ErrorMsg.INVALID_COLUMN.getMsg(expr), expr);
+        ctx.setError(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), expr), expr);
         return null;
       }
 
       if (expr.getType() != HiveParser.TOK_TABLE_OR_COL) {
-        ctx.setError(ErrorMsg.INVALID_COLUMN.getMsg(expr), expr);
+        ctx.setError(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), expr), expr);
         return null;
       }
 
@@ -645,13 +649,15 @@ public class TypeCheckProcFactory<T> {
               }
               stack.push(tmp);
             }
-            ctx.setError(ErrorMsg.NON_KEY_EXPR_IN_GROUPBY.getMsg(exprNode), expr);
+            ctx.setError(ASTErrorUtils.getMsg(
+                ErrorMsg.NON_KEY_EXPR_IN_GROUPBY.getMsg(), exprNode), expr);
             return null;
           } else {
             List<String> possibleColumnNames = input.getReferenceableColumnAliases(tableOrCol, -1);
             String reason = String.format("(possible column names are: %s)",
                 StringUtils.join(possibleColumnNames, ", "));
-            ctx.setError(ErrorMsg.INVALID_TABLE_OR_COLUMN.getMsg(expr.getChild(0), reason),
+            ctx.setError(ASTErrorUtils.getMsg(
+                ErrorMsg.INVALID_TABLE_OR_COLUMN.getMsg(), expr.getChild(0), reason),
                 expr);
             LOG.debug(ErrorMsg.INVALID_TABLE_OR_COLUMN.toString() + ":"
                 + input.toString());
@@ -678,7 +684,7 @@ public class TypeCheckProcFactory<T> {
   /**
    * The default processor for typechecking.
    */
-  public class DefaultExprProcessor implements NodeProcessor {
+  public class DefaultExprProcessor implements SemanticNodeProcessor {
 
     protected boolean isRedundantConversionFunction(ASTNode expr,
         boolean isFunction, List<T> children) {
@@ -770,10 +776,11 @@ public class TypeCheckProcFactory<T> {
       // UDAF in filter condition, group-by caluse, param of funtion, etc.
       if (fi.getGenericUDAFResolver() != null) {
         if (isFunction) {
-          throw new SemanticException(ErrorMsg.UDAF_INVALID_LOCATION.getMsg((ASTNode) expr
-              .getChild(0)));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.UDAF_INVALID_LOCATION.getMsg(), (ASTNode) expr.getChild(0)));
         } else {
-          throw new SemanticException(ErrorMsg.UDAF_INVALID_LOCATION.getMsg(expr));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.UDAF_INVALID_LOCATION.getMsg(), expr));
         }
       }
       if (!ctx.getAllowStatefulFunctions() && (genericUDF != null)) {
@@ -823,7 +830,8 @@ public class TypeCheckProcFactory<T> {
           objectTypeInfo = ((ListTypeInfo) objectTypeInfo).getListElementTypeInfo();
         }
         if (objectTypeInfo.getCategory() != Category.STRUCT) {
-          throw new SemanticException(ErrorMsg.INVALID_DOT.getMsg(expr));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.INVALID_DOT.getMsg(), expr));
         }
         TypeInfo t = ((StructTypeInfo) objectTypeInfo).getStructFieldTypeInfo(fieldNameString);
         if (isList) {
@@ -834,7 +842,8 @@ public class TypeCheckProcFactory<T> {
       } else if (funcText.equals("[")) {
         // "[]" : LSQUARE/INDEX Expression
         if (!ctx.getallowIndexExpr()) {
-          throw new SemanticException(ErrorMsg.INVALID_FUNCTION.getMsg(expr));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.INVALID_FUNCTION.getMsg(), expr));
         }
 
         assert (children.size() == 2);
@@ -856,14 +865,15 @@ public class TypeCheckProcFactory<T> {
         } else if (myt.getCategory() == Category.MAP) {
           if (!TypeInfoUtils.implicitConvertible(exprFactory.getTypeInfo(children.get(1)),
               ((MapTypeInfo) myt).getMapKeyTypeInfo())) {
-            throw new SemanticException(ErrorMsg.INVALID_MAPINDEX_TYPE
-                .getMsg(expr));
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.INVALID_MAPINDEX_TYPE.getMsg(), expr));
           }
           // Calculate TypeInfo
           TypeInfo t = ((MapTypeInfo) myt).getMapValueTypeInfo();
           desc = exprFactory.createFuncCallExpr(t, FunctionRegistry.getGenericUDFForIndex(), children);
         } else {
-          throw new SemanticException(ErrorMsg.NON_COLLECTION_TYPE.getMsg(expr, myt.getTypeName()));
+          throw new SemanticException(ASTErrorUtils.getMsg(
+              ErrorMsg.NON_COLLECTION_TYPE.getMsg(), expr, myt.getTypeName()));
         }
       } else {
         // other operators or functions
@@ -871,10 +881,11 @@ public class TypeCheckProcFactory<T> {
 
         if (fi == null) {
           if (isFunction) {
-            throw new SemanticException(ErrorMsg.INVALID_FUNCTION
-                .getMsg((ASTNode) expr.getChild(0)));
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.INVALID_FUNCTION.getMsg(), (ASTNode) expr.getChild(0)));
           } else {
-            throw new SemanticException(ErrorMsg.INVALID_FUNCTION.getMsg(expr));
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.INVALID_FUNCTION.getMsg(), expr));
           }
         }
 
@@ -1192,7 +1203,8 @@ public class TypeCheckProcFactory<T> {
       }
 
       if (colInfo == null) {
-        ctx.setError(ErrorMsg.INVALID_COLUMN.getMsg(expr.getChild(1)), expr);
+        ctx.setError(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), expr.getChild(1)), expr);
         return null;
       }
       return exprFactory.toExpr(colInfo);
@@ -1282,7 +1294,8 @@ public class TypeCheckProcFactory<T> {
           String tableAlias = BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText());
           Map<String, ColumnInfo> columns = input.getFieldMap(tableAlias);
           if (columns == null) {
-            throw new SemanticException(ErrorMsg.INVALID_TABLE_ALIAS.getMsg(child));
+            throw new SemanticException(ASTErrorUtils.getMsg(
+                ErrorMsg.INVALID_TABLE_ALIAS.getMsg(), child));
           }
           for (Map.Entry<String, ColumnInfo> colMap : columns.entrySet()) {
             ColumnInfo colInfo = colMap.getValue();
@@ -1359,7 +1372,8 @@ public class TypeCheckProcFactory<T> {
         List<String> possibleColumnNames = getReferenceableColumnAliases(ctx);
         String reason = String.format("(possible column names are: %s)",
             StringUtils.join(possibleColumnNames, ", "));
-        ctx.setError(ErrorMsg.INVALID_COLUMN.getMsg(expr.getChild(0), reason),
+        ctx.setError(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_COLUMN.getMsg(), expr.getChild(0), reason),
             expr);
         return null;
       }
@@ -1368,14 +1382,17 @@ public class TypeCheckProcFactory<T> {
       try {
         return getXpathOrFuncExprNodeDesc(expr, isFunction, children, ctx);
       } catch (UDFArgumentTypeException e) {
-        throw new SemanticException(ErrorMsg.INVALID_ARGUMENT_TYPE.getMsg(expr
-            .getChild(childrenBegin + e.getArgumentId()), e.getMessage()), e);
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_ARGUMENT_TYPE.getMsg(),
+            expr.getChild(childrenBegin + e.getArgumentId()), e.getMessage()), e);
       } catch (UDFArgumentLengthException e) {
-        throw new SemanticException(ErrorMsg.INVALID_ARGUMENT_LENGTH.getMsg(
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_ARGUMENT_LENGTH.getMsg(),
             expr, e.getMessage()), e);
       } catch (UDFArgumentException e) {
-        throw new SemanticException(ErrorMsg.INVALID_ARGUMENT.getMsg(expr, e
-            .getMessage()), e);
+        throw new SemanticException(ASTErrorUtils.getMsg(
+            ErrorMsg.INVALID_ARGUMENT.getMsg(),
+            expr, e.getMessage()), e);
       }
     }
 
@@ -1396,7 +1413,7 @@ public class TypeCheckProcFactory<T> {
   /**
    * Processor for subquery expressions..
    */
-  public class SubQueryExprProcessor implements NodeProcessor {
+  public class SubQueryExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -1447,7 +1464,8 @@ public class TypeCheckProcFactory<T> {
         /*
          * Restriction.1.h :: SubQueries only supported in the SQL Where Clause.
          */
-        ctx.setError(ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(sqNode,
+        ctx.setError(ASTErrorUtils.getMsg(
+            ErrorMsg.UNSUPPORTED_SUBQUERY_EXPRESSION.getMsg(), sqNode,
             "Currently only IN & EXISTS SubQuery expressions are allowed"),
             sqNode);
       }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/AuxOpTreeSignature.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/AuxOpTreeSignature.java
index 1815a52..5ed1fd5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/AuxOpTreeSignature.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/AuxOpTreeSignature.java
@@ -26,10 +26,10 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -60,7 +60,7 @@ public final class AuxOpTreeSignature {
     return sig.toString();
   }
 
-  static class AuxSignatureLinker implements NodeProcessor {
+  static class AuxSignatureLinker implements SemanticNodeProcessor {
 
     private PlanMapper pm;
 
@@ -86,8 +86,8 @@ public final class AuxOpTreeSignature {
 
     PlanMapper pm = pctx.getContext().getPlanMapper();
     pm.clearSignatureCache();
-    Dispatcher disp = new DefaultRuleDispatcher(new AuxSignatureLinker(pm), new HashMap(), null);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(new AuxSignatureLinker(pm), new HashMap(), null);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     ogw.startWalking(topNodes, null);
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
index f8b4ace..efcadab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
@@ -30,13 +30,13 @@ import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
 import org.apache.hadoop.hive.ql.lib.ExpressionWalker;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.TypeRule;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -61,7 +61,7 @@ public final class ExprWalkerProcFactory {
    * ColumnExprProcessor.
    *
    */
-  public static class ColumnExprProcessor implements NodeProcessor {
+  public static class ColumnExprProcessor implements SemanticNodeProcessor {
 
     /**
      * Converts the reference from child row resolver to current row resolver.
@@ -167,7 +167,7 @@ public final class ExprWalkerProcFactory {
    * FieldExprProcessor.
    *
    */
-  public static class FieldExprProcessor implements NodeProcessor {
+  public static class FieldExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -217,7 +217,7 @@ public final class ExprWalkerProcFactory {
    * expr is a candidate else it is not a candidate but its children could be
    * final candidates.
    */
-  public static class GenericFuncExprProcessor implements NodeProcessor {
+  public static class GenericFuncExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -281,7 +281,7 @@ public final class ExprWalkerProcFactory {
   /**
    * For constants and null expressions.
    */
-  public static class DefaultExprProcessor implements NodeProcessor {
+  public static class DefaultExprProcessor implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -293,19 +293,19 @@ public final class ExprWalkerProcFactory {
     }
   }
 
-  public static NodeProcessor getDefaultExprProcessor() {
+  public static SemanticNodeProcessor getDefaultExprProcessor() {
     return new DefaultExprProcessor();
   }
 
-  public static NodeProcessor getGenericFuncProcessor() {
+  public static SemanticNodeProcessor getGenericFuncProcessor() {
     return new GenericFuncExprProcessor();
   }
 
-  public static NodeProcessor getColumnProcessor() {
+  public static SemanticNodeProcessor getColumnProcessor() {
     return new ColumnExprProcessor();
   }
 
-  private static NodeProcessor getFieldProcessor() {
+  private static SemanticNodeProcessor getFieldProcessor() {
     return new FieldExprProcessor();
   }
 
@@ -337,16 +337,16 @@ public final class ExprWalkerProcFactory {
     // create a walker which walks the tree in a DFS manner while maintaining
     // the operator stack. The dispatcher
     // generates the plan from the operator tree
-    Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> exprRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     exprRules.put(new TypeRule(ExprNodeColumnDesc.class), getColumnProcessor());
     exprRules.put(new TypeRule(ExprNodeFieldDesc.class), getFieldProcessor());
     exprRules.put(new TypeRule(ExprNodeGenericFuncDesc.class), getGenericFuncProcessor());
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(getDefaultExprProcessor(),
+    SemanticDispatcher disp = new DefaultRuleDispatcher(getDefaultExprProcessor(),
         exprRules, exprContext);
-    GraphWalker egw = new ExpressionWalker(disp);
+    SemanticGraphWalker egw = new ExpressionWalker(disp);
 
     List<Node> startNodes = new ArrayList<Node>();
     List<ExprNodeDesc> clonedPreds = new ArrayList<ExprNodeDesc>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
index 38f66f9..b01f74d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
 import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler;
@@ -156,7 +156,7 @@ public final class OpProcFactory {
   /**
    * Processor for Script Operator Prevents any predicates being pushed.
    */
-  public static class ScriptPPD extends DefaultPPD implements NodeProcessor {
+  public static class ScriptPPD extends DefaultPPD implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -364,7 +364,7 @@ public final class OpProcFactory {
     }
   }
 
-  public static class UDTFPPD extends DefaultPPD implements NodeProcessor {
+  public static class UDTFPPD extends DefaultPPD implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -382,7 +382,7 @@ public final class OpProcFactory {
 
   }
 
-  public static class LateralViewForwardPPD extends DefaultPPD implements NodeProcessor {
+  public static class LateralViewForwardPPD extends DefaultPPD implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -409,7 +409,7 @@ public final class OpProcFactory {
    * Combines predicates of its child into a single expression and adds a filter
    * op as new child.
    */
-  public static class TableScanPPD extends DefaultPPD implements NodeProcessor {
+  public static class TableScanPPD extends DefaultPPD implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -437,7 +437,7 @@ public final class OpProcFactory {
    * Determines the push down predicates in its where expression and then
    * combines it with the push down predicates that are passed from its children.
    */
-  public static class FilterPPD extends DefaultPPD implements NodeProcessor {
+  public static class FilterPPD extends DefaultPPD implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -496,7 +496,7 @@ public final class OpProcFactory {
     }
   }
 
-  public static class SimpleFilterPPD extends FilterPPD implements NodeProcessor {
+  public static class SimpleFilterPPD extends FilterPPD implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -532,7 +532,7 @@ public final class OpProcFactory {
    * Determines predicates for which alias can be pushed to it's parents. See
    * the comments for getQualifiedAliases function.
    */
-  public static class JoinerPPD extends DefaultPPD implements NodeProcessor {
+  public static class JoinerPPD extends DefaultPPD implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
@@ -652,7 +652,7 @@ public final class OpProcFactory {
     }
   }
 
-  public static class ReduceSinkPPD extends DefaultPPD implements NodeProcessor {
+  public static class ReduceSinkPPD extends DefaultPPD implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
                           Object... nodeOutputs) throws SemanticException {
@@ -736,7 +736,7 @@ public final class OpProcFactory {
   /**
    * Default processor which just merges its children.
    */
-  public static class DefaultPPD implements NodeProcessor {
+  public static class DefaultPPD implements SemanticNodeProcessor {
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
@@ -1045,51 +1045,51 @@ public final class OpProcFactory {
     return decomposed.residualPredicate;
   }
 
-  public static NodeProcessor getFilterProc() {
+  public static SemanticNodeProcessor getFilterProc() {
     return new FilterPPD();
   }
 
-  public static NodeProcessor getFilterSyntheticJoinPredicateProc() {
+  public static SemanticNodeProcessor getFilterSyntheticJoinPredicateProc() {
     return new SimpleFilterPPD();
   }
 
-  public static NodeProcessor getJoinProc() {
+  public static SemanticNodeProcessor getJoinProc() {
     return new JoinPPD();
   }
 
-  public static NodeProcessor getTSProc() {
+  public static SemanticNodeProcessor getTSProc() {
     return new TableScanPPD();
   }
 
-  public static NodeProcessor getDefaultProc() {
+  public static SemanticNodeProcessor getDefaultProc() {
     return new DefaultPPD();
   }
 
-  public static NodeProcessor getPTFProc() {
+  public static SemanticNodeProcessor getPTFProc() {
     return new PTFPPD();
   }
 
-  public static NodeProcessor getSCRProc() {
+  public static SemanticNodeProcessor getSCRProc() {
     return new ScriptPPD();
   }
 
-  public static NodeProcessor getLIMProc() {
+  public static SemanticNodeProcessor getLIMProc() {
     return new ScriptPPD();
   }
 
-  public static NodeProcessor getLVFProc() {
+  public static SemanticNodeProcessor getLVFProc() {
     return new LateralViewForwardPPD();
   }
 
-  public static NodeProcessor getUDTFProc() {
+  public static SemanticNodeProcessor getUDTFProc() {
     return new UDTFPPD();
   }
 
-  public static NodeProcessor getLVJProc() {
+  public static SemanticNodeProcessor getLVJProc() {
     return new JoinerPPD();
   }
 
-  public static NodeProcessor getRSProc() {
+  public static SemanticNodeProcessor getRSProc() {
     return new ReduceSinkPPD();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicatePushDown.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicatePushDown.java
index b3d59e3..4cf86bb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicatePushDown.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicatePushDown.java
@@ -36,11 +36,11 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDTFOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -91,7 +91,7 @@ public class PredicatePushDown extends Transform {
     // create a the context for walking operators
     OpWalkerInfo opWalkerInfo = new OpWalkerInfo(pGraphContext);
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1",
       FilterOperator.getOperatorName() + "%"),
       OpProcFactory.getFilterProc());
@@ -125,9 +125,9 @@ public class PredicatePushDown extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(OpProcFactory.getDefaultProc(),
+    SemanticDispatcher disp = new DefaultRuleDispatcher(OpProcFactory.getDefaultProc(),
         opRules, opWalkerInfo);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
     ArrayList<Node> topNodes = new ArrayList<Node>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
index 35f9c7b..9666dd7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
@@ -35,13 +35,13 @@ import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.LevelOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -64,7 +64,7 @@ public class PredicateTransitivePropagate extends Transform {
   public ParseContext transform(ParseContext pctx) throws SemanticException {
     pGraphContext = pctx;
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1", "(" +
         FilterOperator.getOperatorName() + "%" +
         ReduceSinkOperator.getOperatorName() + "%" +
@@ -73,8 +73,8 @@ public class PredicateTransitivePropagate extends Transform {
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
     TransitiveContext context = new TransitiveContext();
-    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, context);
-    GraphWalker ogw = new LevelOrderWalker(disp, 2);
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, context);
+    SemanticGraphWalker ogw = new LevelOrderWalker(disp, 2);
 
     // Create a list of topop nodes
     List<Node> topNodes = new ArrayList<Node>();
@@ -134,7 +134,7 @@ public class PredicateTransitivePropagate extends Transform {
     }
   }
 
-  private static class JoinTransitive implements NodeProcessor {
+  private static class JoinTransitive implements SemanticNodeProcessor {
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SimplePredicatePushDown.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SimplePredicatePushDown.java
index a75ad21..6f43e09 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SimplePredicatePushDown.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SimplePredicatePushDown.java
@@ -34,11 +34,11 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDTFOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
-import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
-import org.apache.hadoop.hive.ql.lib.NodeProcessor;
-import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.optimizer.Transform;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -58,7 +58,7 @@ public class SimplePredicatePushDown extends Transform {
     // create a the context for walking operators
     OpWalkerInfo opWalkerInfo = new OpWalkerInfo(pGraphContext);
 
-    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
     opRules.put(new RuleRegExp("R1",
       FilterOperator.getOperatorName() + "%"),
       OpProcFactory.getFilterSyntheticJoinPredicateProc());
@@ -92,9 +92,9 @@ public class SimplePredicatePushDown extends Transform {
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
-    Dispatcher disp = new DefaultRuleDispatcher(OpProcFactory.getDefaultProc(),
+    SemanticDispatcher disp = new DefaultRuleDispatcher(OpProcFactory.getDefaultProc(),
         opRules, opWalkerInfo);
-    GraphWalker ogw = new DefaultGraphWalker(disp);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
 
     // Create a list of topop nodes
... 534 lines suppressed ...