You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2008/12/23 00:23:33 UTC

svn commit: r728823 [2/3] - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/lib/ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ ql/src/java/org/apache/...

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Mon Dec 22 15:23:32 2008
@@ -25,8 +25,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.antlr.runtime.tree.CommonTree;
-import org.antlr.runtime.tree.Tree;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -74,7 +72,7 @@
   }
 
   @Override
-  public void analyzeInternal(CommonTree ast, Context ctx) throws SemanticException {
+  public void analyzeInternal(ASTNode ast, Context ctx) throws SemanticException {
     this.ctx = ctx;
     if (ast.getToken().getType() == HiveParser.TOK_CREATETABLE)
        analyzeCreateTable(ast, false);
@@ -116,7 +114,7 @@
     }
   }
 
-  private void analyzeCreateTable(CommonTree ast, boolean isExt) 
+  private void analyzeCreateTable(ASTNode ast, boolean isExt) 
     throws SemanticException {
     String            tableName     = unescapeIdentifier(ast.getChild(0).getText());
     List<FieldSchema> cols          = null;
@@ -139,7 +137,7 @@
     int numCh = ast.getChildCount();
     for (int num = 1; num < numCh; num++)
     {
-      CommonTree child = (CommonTree)ast.getChild(num);
+      ASTNode child = (ASTNode)ast.getChild(num);
       switch (child.getToken().getType()) {
         case HiveParser.TOK_TABCOLLIST:
           cols = getColumns(child);
@@ -148,15 +146,15 @@
           comment = unescapeSQLString(child.getChild(0).getText());
           break;
         case HiveParser.TOK_TABLEPARTCOLS:
-          partCols = getColumns((CommonTree)child.getChild(0));
+          partCols = getColumns((ASTNode)child.getChild(0));
           break;
         case HiveParser.TOK_TABLEBUCKETS:
-          bucketCols = getColumnNames((CommonTree)child.getChild(0));
+          bucketCols = getColumnNames((ASTNode)child.getChild(0));
           if (child.getChildCount() == 2)
             numBuckets = (Integer.valueOf(child.getChild(1).getText())).intValue();
           else
           {
-            sortCols = getColumnNamesOrder((CommonTree)child.getChild(1));
+            sortCols = getColumnNamesOrder((ASTNode)child.getChild(1));
             numBuckets = (Integer.valueOf(child.getChild(2).getText())).intValue();
           }
           break;
@@ -164,7 +162,7 @@
           int numChildRowFormat = child.getChildCount();
           for (int numC = 0; numC < numChildRowFormat; numC++)
           {
-            CommonTree rowChild = (CommonTree)child.getChild(numC);
+            ASTNode rowChild = (ASTNode)child.getChild(numC);
             switch (rowChild.getToken().getType()) {
               case HiveParser.TOK_TABLEROWFORMATFIELD:
                 fieldDelim = unescapeSQLString(rowChild.getChild(0).getText());
@@ -186,7 +184,7 @@
           serde = unescapeSQLString(child.getChild(0).getText());
           if (child.getChildCount() == 2) {
             mapProp = new HashMap<String, String>();
-            CommonTree prop = (CommonTree)((CommonTree)child.getChild(1)).getChild(0);
+            ASTNode prop = (ASTNode)((ASTNode)child.getChild(1)).getChild(0);
             for (int propChild = 0; propChild < prop.getChildCount(); propChild++) {
               String key = unescapeSQLString(prop.getChild(propChild).getChild(0).getText());
               String value = unescapeSQLString(prop.getChild(propChild).getChild(1).getText());
@@ -300,37 +298,37 @@
     }
   }
   
-  private void analyzeDropTable(CommonTree ast) 
+  private void analyzeDropTable(ASTNode ast) 
     throws SemanticException {
     String tableName = unescapeIdentifier(ast.getChild(0).getText());    
     dropTableDesc dropTblDesc = new dropTableDesc(tableName);
     rootTasks.add(TaskFactory.get(new DDLWork(dropTblDesc), conf));
   }
 
-  private void analyzeAlterTableProps(CommonTree ast) throws SemanticException { 
+  private void analyzeAlterTableProps(ASTNode ast) throws SemanticException { 
     String tableName = unescapeIdentifier(ast.getChild(0).getText());    
-    HashMap<String, String> mapProp = getProps((CommonTree)(ast.getChild(1)).getChild(0));
+    HashMap<String, String> mapProp = getProps((ASTNode)(ast.getChild(1)).getChild(0));
     alterTableDesc alterTblDesc = new alterTableDesc(alterTableTypes.ADDPROPS);
     alterTblDesc.setProps(mapProp);
     alterTblDesc.setOldName(tableName);
     rootTasks.add(TaskFactory.get(new DDLWork(alterTblDesc), conf));
   }
 
-  private void analyzeAlterTableSerdeProps(CommonTree ast) throws SemanticException { 
+  private void analyzeAlterTableSerdeProps(ASTNode ast) throws SemanticException { 
     String tableName = unescapeIdentifier(ast.getChild(0).getText());    
-    HashMap<String, String> mapProp = getProps((CommonTree)(ast.getChild(1)).getChild(0));
+    HashMap<String, String> mapProp = getProps((ASTNode)(ast.getChild(1)).getChild(0));
     alterTableDesc alterTblDesc = new alterTableDesc(alterTableTypes.ADDSERDEPROPS);
     alterTblDesc.setProps(mapProp);
     alterTblDesc.setOldName(tableName);
     rootTasks.add(TaskFactory.get(new DDLWork(alterTblDesc), conf));
   }
 
-  private void analyzeAlterTableSerde(CommonTree ast) throws SemanticException { 
+  private void analyzeAlterTableSerde(ASTNode ast) throws SemanticException { 
     String tableName = unescapeIdentifier(ast.getChild(0).getText());    
     String serdeName = unescapeSQLString(ast.getChild(1).getText());
     alterTableDesc alterTblDesc = new alterTableDesc(alterTableTypes.ADDSERDE);
     if(ast.getChildCount() > 2) {
-      HashMap<String, String> mapProp = getProps((CommonTree)(ast.getChild(2)).getChild(0));
+      HashMap<String, String> mapProp = getProps((ASTNode)(ast.getChild(2)).getChild(0));
       alterTblDesc.setProps(mapProp);
     }
     alterTblDesc.setOldName(tableName);
@@ -338,7 +336,7 @@
     rootTasks.add(TaskFactory.get(new DDLWork(alterTblDesc), conf));
   }
 
-  private HashMap<String, String> getProps(CommonTree prop) {
+  private HashMap<String, String> getProps(ASTNode prop) {
     HashMap<String, String> mapProp = new HashMap<String, String>();
     for (int propChild = 0; propChild < prop.getChildCount(); propChild++) {
       String key = unescapeSQLString(prop.getChild(propChild).getChild(0).getText());
@@ -348,24 +346,24 @@
     return mapProp;
   }
 
-  private List<FieldSchema> getColumns(CommonTree ast)
+  private List<FieldSchema> getColumns(ASTNode ast)
   {
     List<FieldSchema> colList = new ArrayList<FieldSchema>();
     int numCh = ast.getChildCount();
     for (int i = 0; i < numCh; i++) {
       FieldSchema col = new FieldSchema();
-      CommonTree child = (CommonTree)ast.getChild(i);
+      ASTNode child = (ASTNode)ast.getChild(i);
       col.setName(unescapeIdentifier(child.getChild(0).getText()));
-      CommonTree typeChild = (CommonTree)(child.getChild(1));
+      ASTNode typeChild = (ASTNode)(child.getChild(1));
       if (typeChild.getToken().getType() == HiveParser.TOK_LIST)
       {
-        CommonTree typName = (CommonTree)typeChild.getChild(0);
+        ASTNode typName = (ASTNode)typeChild.getChild(0);
         col.setType(MetaStoreUtils.getListType(getTypeName(typName.getToken().getType())));
       }
       else if (typeChild.getToken().getType() == HiveParser.TOK_MAP)
       {
-        CommonTree ltypName = (CommonTree)typeChild.getChild(0);
-        CommonTree rtypName = (CommonTree)typeChild.getChild(1);
+        ASTNode ltypName = (ASTNode)typeChild.getChild(0);
+        ASTNode rtypName = (ASTNode)typeChild.getChild(1);
         col.setType(MetaStoreUtils.getMapType(getTypeName(ltypName.getToken().getType()), getTypeName(rtypName.getToken().getType())));
       }
       else                                // primitive type
@@ -378,23 +376,23 @@
     return colList;
   }
   
-  private List<String> getColumnNames(CommonTree ast)
+  private List<String> getColumnNames(ASTNode ast)
   {
     List<String> colList = new ArrayList<String>();
     int numCh = ast.getChildCount();
     for (int i = 0; i < numCh; i++) {
-      CommonTree child = (CommonTree)ast.getChild(i);
+      ASTNode child = (ASTNode)ast.getChild(i);
       colList.add(unescapeIdentifier(child.getText()));
     }
     return colList;
   }
 
-  private List<Order> getColumnNamesOrder(CommonTree ast)
+  private List<Order> getColumnNamesOrder(ASTNode ast)
   {
     List<Order> colList = new ArrayList<Order>();
     int numCh = ast.getChildCount();
     for (int i = 0; i < numCh; i++) {
-      CommonTree child = (CommonTree)ast.getChild(i);
+      ASTNode child = (ASTNode)ast.getChild(i);
       if (child.getToken().getType() == HiveParser.TOK_TABSORTCOLNAMEASC)
         colList.add(new Order(unescapeIdentifier(child.getChild(0).getText()), 1));
       else
@@ -410,29 +408,27 @@
    * @param ast The AST from which the qualified name has to be extracted
    * @return String
    */
-  private String getFullyQualifiedName(CommonTree ast) {
+  private String getFullyQualifiedName(ASTNode ast) {
     if (ast.getChildCount() == 0) {
       return ast.getText();
     }
 
-    return getFullyQualifiedName((CommonTree)ast.getChild(0)) + "." +
-           getFullyQualifiedName((CommonTree)ast.getChild(1));
+    return getFullyQualifiedName((ASTNode)ast.getChild(0)) + "." +
+           getFullyQualifiedName((ASTNode)ast.getChild(1));
   }
 
-  private void analyzeDescribeTable(CommonTree ast) 
+  private void analyzeDescribeTable(ASTNode ast) 
   throws SemanticException {
-    CommonTree tableTypeExpr = (CommonTree)ast.getChild(0);
-    // Walk the tree and generate a list of components
-    ArrayList<String> comp_list = new ArrayList<String>();
-    String tableName = getFullyQualifiedName((CommonTree)tableTypeExpr.getChild(0));
+    ASTNode tableTypeExpr = (ASTNode)ast.getChild(0);
+    String tableName = getFullyQualifiedName((ASTNode)tableTypeExpr.getChild(0));
 
     HashMap<String, String> partSpec = null;
     // get partition metadata if partition specified
     if (tableTypeExpr.getChildCount() == 2) {
-      CommonTree partspec = (CommonTree) tableTypeExpr.getChild(1);
+      ASTNode partspec = (ASTNode) tableTypeExpr.getChild(1);
       partSpec = new LinkedHashMap<String, String>();
       for (int i = 0; i < partspec.getChildCount(); ++i) {
-        CommonTree partspec_val = (CommonTree) partspec.getChild(i);
+        ASTNode partspec_val = (ASTNode) partspec.getChild(i);
         String val = stripQuotes(partspec_val.getChild(1).getText());
         partSpec.put(partspec_val.getChild(0).getText(), val);
       }
@@ -444,7 +440,7 @@
     LOG.info("analyzeDescribeTable done");
   }
   
-  private void analyzeShowPartitions(CommonTree ast) 
+  private void analyzeShowPartitions(ASTNode ast) 
   throws SemanticException {
     showPartitionsDesc showPartsDesc;
     String tableName = unescapeIdentifier(ast.getChild(0).getText());
@@ -452,7 +448,7 @@
     rootTasks.add(TaskFactory.get(new DDLWork(showPartsDesc), conf));
   }
   
-  private void analyzeShowTables(CommonTree ast) 
+  private void analyzeShowTables(ASTNode ast) 
   throws SemanticException {
     showTablesDesc showTblsDesc;
     if (ast.getChildCount() == 1)
@@ -465,7 +461,7 @@
     rootTasks.add(TaskFactory.get(new DDLWork(showTblsDesc), conf));
   }
 
-  private void analyzeAlterTableRename(CommonTree ast) 
+  private void analyzeAlterTableRename(ASTNode ast) 
   throws SemanticException {
     alterTableDesc alterTblDesc = new alterTableDesc(
         unescapeIdentifier(ast.getChild(0).getText()),
@@ -473,15 +469,15 @@
     rootTasks.add(TaskFactory.get(new DDLWork(alterTblDesc), conf));
   }
 
-  private void analyzeAlterTableModifyCols(CommonTree ast, alterTableTypes alterType) 
+  private void analyzeAlterTableModifyCols(ASTNode ast, alterTableTypes alterType) 
   throws SemanticException {
     String tblName = unescapeIdentifier(ast.getChild(0).getText());
-    List<FieldSchema> newCols = getColumns((CommonTree)ast.getChild(1));
+    List<FieldSchema> newCols = getColumns((ASTNode)ast.getChild(1));
     alterTableDesc alterTblDesc = new alterTableDesc(tblName, newCols, alterType);
     rootTasks.add(TaskFactory.get(new DDLWork(alterTblDesc), conf));
   }
 
-  private void analyzeAlterTableDropParts(CommonTree ast) throws SemanticException {
+  private void analyzeAlterTableDropParts(ASTNode ast) throws SemanticException {
     String tblName = null;
     List<HashMap<String, String>> partSpecs = new ArrayList<HashMap<String, String>>();
     int childIndex = 0;
@@ -489,10 +485,10 @@
     tblName = unescapeIdentifier(ast.getChild(0).getText());
     // get partition metadata if partition specified
     for( childIndex = 1; childIndex < ast.getChildCount(); childIndex++) {
-      CommonTree partspec = (CommonTree) ast.getChild(childIndex);
+      ASTNode partspec = (ASTNode) ast.getChild(childIndex);
       HashMap<String, String> partSpec = new LinkedHashMap<String, String>();
       for (int i = 0; i < partspec.getChildCount(); ++i) {
-        CommonTree partspec_val = (CommonTree) partspec.getChild(i);
+        ASTNode partspec_val = (ASTNode) partspec.getChild(i);
         String val = stripQuotes(partspec_val.getChild(1).getText());
         partSpec.put(partspec_val.getChild(0).getText(), val);
       }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTEventDispatcher.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTEventDispatcher.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTEventDispatcher.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTEventDispatcher.java Mon Dec 22 15:23:32 2008
@@ -1,84 +0,0 @@
-/**
- * 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 java.util.HashMap;
-import java.util.ArrayList;
-
-import org.antlr.runtime.tree.CommonTree;
-
-/**
- * Implementation of a default ParseTreeEventDispatcher. This dispatcher calls
- * the associated ParseTreeEventProcessors in the order in which they were
- * registered for the event
- *
- */
-public class DefaultASTEventDispatcher implements
-		ASTEventDispatcher {
-	
-	/**
-	 * Stores the mapping from the ParseTreeEvent to the list of ParseTreeEventProcessors.
-	 * The later are stored in the order that they were registered.
-	 */
-	private HashMap<ASTEvent, ArrayList<ASTEventProcessor>> dispatchMap;
-
-	/**
-	 * Constructs the default event dispatcher
-	 */
-	public  DefaultASTEventDispatcher() {
-		dispatchMap = new HashMap<ASTEvent, ArrayList<ASTEventProcessor>>();
-	}
-	
-	/**
-	 * Dispatches the parse subtree to all the event processors registered for the
-	 * event in the order that they were registered.
-	 * 
-	 * @see org.apache.hadoop.hive.ql.parse.ASTEventDispatcher#dispatch(org.apache.hadoop.hive.ql.parse.ASTEvent, org.antlr.runtime.tree.CommonTree)
-	 */
-	@Override
-	public void dispatch(ASTEvent evt, CommonTree pt) {
-		
-		ArrayList<ASTEventProcessor> evtp_l = dispatchMap.get(evt);
-		if (evtp_l == null) {
-			return;
-		}
-
-		for(ASTEventProcessor evt_p: evtp_l) {
-			// Do the actual dispatch
-			evt_p.process(pt);
-		}
-	}
-
-	/**
-	 * Registers the event processor for the event.
-	 * 
-	 * @see org.apache.hadoop.hive.ql.parse.ASTEventDispatcher#register(org.apache.hadoop.hive.ql.parse.ASTEvent, org.apache.hadoop.hive.ql.parse.ASTEventProcessor)
-	 */
-	@Override
-	public void register(ASTEvent evt, ASTEventProcessor evt_p) {
-
-		ArrayList<ASTEventProcessor> evtp_l = dispatchMap.get(evt);
-		if (evtp_l == null) {
-			evtp_l = new ArrayList<ASTEventProcessor>();
-			dispatchMap.put(evt, evtp_l);
-		}
-
-		evtp_l.add(evt_p);
-	}
-}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTProcessor.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTProcessor.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTProcessor.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTProcessor.java Mon Dec 22 15:23:32 2008
@@ -1,108 +0,0 @@
-/**
- * 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.CommonTree;
-
-/**
- * Implementation of a parse tree processor. This processor does a depth first walk
- * of the parse tree and calls the associated event processors
- */
-public class DefaultASTProcessor implements ASTProcessor {
-
-	/**
-	 * The dispatcher used to dispatch ParseTreeEvents to the ParseTreeEventProcessors
-	 */
-	private ASTEventDispatcher dispatcher;
-	
-	/**
-	 * Processes the parse tree
-	 * 
-	 * @see org.apache.hadoop.hive.ql.parse.ASTProcessor#process(org.antlr.runtime.tree.CommonTree)
-	 */
-	@Override
-	public void process(CommonTree ast) {
-
-		// Base case
-		if (ast.getToken() == null) {
-			return;
-		}
-
-		switch (ast.getToken().getType()) {
-		case HiveParser.TOK_SELECTDI:
-		case HiveParser.TOK_SELECT:
-			dispatcher.dispatch(ASTEvent.SELECT_CLAUSE, ast);
-			break;
-
-		case HiveParser.TOK_WHERE:
-			dispatcher.dispatch(ASTEvent.WHERE_CLAUSE, ast);
-			break;
-
-		case HiveParser.TOK_DESTINATION:
-			dispatcher.dispatch(ASTEvent.DESTINATION, ast);
-			break;
-
-		case HiveParser.TOK_FROM:
-
-			// Check if this is a subquery
-			CommonTree frm = (CommonTree) ast.getChild(0);
-			if (frm.getToken().getType() == HiveParser.TOK_TABREF) {
-				dispatcher.dispatch(ASTEvent.SRC_TABLE, ast);
-			} else if (frm.getToken().getType() == HiveParser.TOK_SUBQUERY) {
-				dispatcher.dispatch(ASTEvent.SUBQUERY, ast);
-			} else if (ParseUtils.isJoinToken(frm)) {
-				dispatcher.dispatch(ASTEvent.JOIN_CLAUSE, ast);
-			}
-			break;
-
-		case HiveParser.TOK_CLUSTERBY:
-			dispatcher.dispatch(ASTEvent.CLUSTERBY_CLAUSE, ast);
-			break;
-
-		case HiveParser.TOK_GROUPBY:
-			dispatcher.dispatch(ASTEvent.GROUPBY_CLAUSE, ast);
-			break;
-
-		case HiveParser.TOK_LIMIT:
-			dispatcher.dispatch(ASTEvent.LIMIT_CLAUSE, ast);
-			break;
-		default:
-			break;
-		}
-
-		// Iterate over the rest of the children
-		int child_count = ast.getChildCount();
-		for (int child_pos = 0; child_pos < child_count; ++child_pos) {
-			// Recurse
-			process((CommonTree) ast.getChild(child_pos));
-		}
-	}
-
-	/**
-	 * Sets the dispatcher for the parse tree processor
-	 * 
-	 * @see org.apache.hadoop.hive.ql.parse.ASTProcessor#register(org.apache.hadoop.hive.ql.parse.ASTEvent, org.apache.hadoop.hive.ql.parse.ParseTreeEventProcessor)
-	 */
-	@Override
-	public void setDispatcher(ASTEventDispatcher dispatcher) {
-		
-		this.dispatcher = dispatcher;
-	}
-
-}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultDispatcher.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultDispatcher.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultDispatcher.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultDispatcher.java Mon Dec 22 15:23:32 2008
@@ -1,103 +0,0 @@
-/**
- * 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 java.io.Serializable;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.List;
-import java.util.Stack;
-import java.lang.ClassNotFoundException;
-
-import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.OperatorFactory;
-import org.apache.hadoop.hive.ql.exec.OperatorFactory.opTuple;
-
-/**
- * Dispatches calls to relevant method in processor 
- */
-public class DefaultDispatcher implements Dispatcher {
-  
-  private OperatorProcessor opProcessor;
-  
-  /**
-   * constructor
-   * @param opp operator processor that handles actual processing of the node
-   */
-  public DefaultDispatcher(OperatorProcessor opp) {
-    this.opProcessor = opp;
-  }
-
-  /**
-   * dispatcher function
-   * @param op operator to process
-   * @param opStack the operators encountered so far
-   * @throws SemanticException
-   */
-  public void dispatch(Operator<? extends Serializable> op, Stack<Operator<? extends Serializable>> opStack) 
-    throws SemanticException {
-
-    // If the processor has registered a process method for the particular operator, invoke it.
-    // Otherwise implement the generic function, which would definitely be implemented
-    for(opTuple opt : OperatorFactory.opvec) {
-      if(opt.opClass.isInstance(op)) {
-        Method pcall;
-        try {
-          pcall = opProcessor.getClass().getMethod("process", opt.opClass, 
-                                                   Class.forName("org.apache.hadoop.hive.ql.optimizer.OperatorProcessorContext"));
-          pcall.invoke(opProcessor, op, null);
-          return;
-        } catch (SecurityException e) {
-          assert false;
-        } catch (NoSuchMethodException e) {
-          assert false;
-        } catch (IllegalArgumentException e) {
-          assert false;
-        } catch (IllegalAccessException e) {
-          assert false;
-        } catch (InvocationTargetException e) {
-          throw new SemanticException(e.getTargetException());
-        } catch (ClassNotFoundException e) {
-          assert false;
-        }
-      }
-    }
-
-    try {
-      // no method found - invoke the generic function
-      Method pcall = opProcessor.getClass().getMethod("process", Class.forName("org.apache.hadoop.hive.ql.exec.Operator"), 
-                                                      Class.forName("org.apache.hadoop.hive.ql.optimizer.OperatorProcessorContext"));
-      
-      pcall.invoke(opProcessor, ((Operator<? extends Serializable>)op), null);
-      return;
-    } catch (SecurityException e) {
-      assert false;
-    } catch (NoSuchMethodException e) {
-      assert false;
-    } catch (IllegalArgumentException e) {
-      assert false;
-    } catch (IllegalAccessException e) {
-      assert false;
-    } catch (InvocationTargetException e) {
-      throw new SemanticException(e.getTargetException());
-    } catch (ClassNotFoundException e) {
-      assert false;
-    }
-  }
-}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ErrorMsg.java Mon Dec 22 15:23:32 2008
@@ -74,45 +74,45 @@
     this.mesg = mesg;
   }
 
-  private int getLine(CommonTree tree) {
+  private int getLine(ASTNode tree) {
     if (tree.getChildCount() == 0) {
       return tree.getToken().getLine();
     }
 
-    return getLine((CommonTree)tree.getChild(0));
+    return getLine((ASTNode)tree.getChild(0));
   }
 
-  private int getCharPositionInLine(CommonTree tree) {
+  private int getCharPositionInLine(ASTNode tree) {
     if (tree.getChildCount() == 0) {
       return tree.getToken().getCharPositionInLine();
     }
 
-    return getCharPositionInLine((CommonTree)tree.getChild(0));
+    return getCharPositionInLine((ASTNode)tree.getChild(0));
   }
 
   // Dirty hack as this will throw away spaces and other things - find a better way!
-  private String getText(CommonTree tree) {
+  private String getText(ASTNode tree) {
     if (tree.getChildCount() == 0) {
       return tree.getText();
     }
 
-    return getText((CommonTree)tree.getChild(tree.getChildCount() - 1));
+    return getText((ASTNode)tree.getChild(tree.getChildCount() - 1));
   }
 
-  String getMsg(CommonTree tree) {
+  String getMsg(ASTNode tree) {
     return "line " + getLine(tree) + ":" + getCharPositionInLine(tree) + " " + mesg + " " + getText(tree);
   }
 
   String getMsg(Tree tree) {
-    return getMsg((CommonTree)tree);
+    return getMsg((ASTNode)tree);
   }
 
-  String getMsg(CommonTree tree, String reason) {
+  String getMsg(ASTNode tree, String reason) {
     return "line " + getLine(tree) + ":" + getCharPositionInLine(tree) + " " + mesg + " " + getText(tree) + ": " + reason;
   }
 
   String getMsg(Tree tree, String reason) {
-    return getMsg((CommonTree)tree, reason);
+    return getMsg((ASTNode)tree, reason);
   }
 
   String getMsg() {

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java Mon Dec 22 15:23:32 2008
@@ -22,7 +22,6 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.antlr.runtime.tree.CommonTree;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Context;
@@ -37,11 +36,11 @@
     super(conf);
   }
 
-  public void analyzeInternal(CommonTree ast, Context ctx) throws SemanticException {
+  public void analyzeInternal(ASTNode ast, Context ctx) throws SemanticException {
     
     // Create a semantic analyzer for the query
-    BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, (CommonTree)ast.getChild(0));
-    sem.analyze((CommonTree)ast.getChild(0), ctx);
+    BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, (ASTNode)ast.getChild(0));
+    sem.analyze((ASTNode)ast.getChild(0), ctx);
     
     boolean extended = false;
     if (ast.getChildCount() > 1) {
@@ -61,7 +60,7 @@
     	tasks.add(fetchTask); 
     		
     rootTasks.add(TaskFactory.get(new explainWork(ctx.getResFile(), tasks,
-                                                  ((CommonTree)ast.getChild(0)).toStringTree(),
+                                                  ((ASTNode)ast.getChild(0)).toStringTree(),
                                                   extended), this.conf));
   }
 }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java Mon Dec 22 15:23:32 2008
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import org.antlr.runtime.tree.CommonTree;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -36,7 +34,7 @@
     super(conf);
   }
   
-  public void analyzeInternal(CommonTree ast, Context ctx) throws SemanticException {
+  public void analyzeInternal(ASTNode ast, Context ctx) throws SemanticException {
     String functionName = ast.getChild(0).getText();
     String className = unescapeSQLString(ast.getChild(1).getText());
     createFunctionDesc desc = new createFunctionDesc(functionName, className);

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenMapRedWalker.java Mon Dec 22 15:23:32 2008
@@ -18,21 +18,17 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.io.Serializable;
-import java.util.List;
 import java.util.Stack;
-import java.util.regex.Pattern;
-import java.util.regex.Matcher;
-
-import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import java.util.Vector;
 import org.apache.hadoop.hive.ql.exec.*;
+import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
+import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.Node;
 
 /**
  * Walks the operator tree in pre order fashion
  */
-public class GenMapRedWalker extends DefaultOpGraphWalker {
-  private Stack<Operator<? extends Serializable>> opStack;
+public class GenMapRedWalker extends DefaultGraphWalker {
 
   /**
    * constructor of the walker - the dispatcher is passed
@@ -40,30 +36,29 @@
    */
   public GenMapRedWalker(Dispatcher disp) {
     super(disp);
-    opStack = new Stack<Operator<? extends Serializable>>();
   }
   
   /**
    * Walk the given operator
-   * @param op operator being walked
+   * @param nd operator being walked
    */
   @Override
-  public void walk(Operator<? extends Serializable> op) throws SemanticException {
-    List<Operator<? extends Serializable>> children = op.getChildOperators();
+  public void walk(Node nd) throws SemanticException {
+    Vector<Node> children = nd.getChildren();
     
     // maintain the stack of operators encountered
-    opStack.push(op);
-    dispatch(op, opStack);
+    opStack.push(nd);
+    dispatch(nd, opStack);
 
     // kids of reduce sink operator need not be traversed again
     if ((children == null) ||
-        ((op instanceof ReduceSinkOperator) && (getDispatchedList().containsAll(children)))) {
+        ((nd instanceof ReduceSinkOperator) && (getDispatchedList().containsAll(children)))) {
       opStack.pop();
       return;
     }
 
     // move all the children to the front of queue
-    for (Operator<? extends Serializable> ch : children)
+    for (Node ch : children)
       walk(ch);
 
     // done with this operator

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java Mon Dec 22 15:23:32 2008
@@ -26,7 +26,6 @@
 import java.util.HashMap;
 import java.util.List;
 
-import org.antlr.runtime.tree.CommonTree;
 import org.antlr.runtime.tree.Tree;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FileStatus;
@@ -156,7 +155,7 @@
   }
 
   @Override
-  public void analyzeInternal(CommonTree ast, Context ctx) throws SemanticException {
+  public void analyzeInternal(ASTNode ast, Context ctx) throws SemanticException {
     isLocal = isOverWrite = false;
     Tree from_t = ast.getChild(0);
     Tree table_t = ast.getChild(1);
@@ -185,7 +184,7 @@
     }
 
     // initialize destination table/partition
-    tableSpec ts = new tableSpec(db, (CommonTree) table_t, true);
+    tableSpec ts = new tableSpec(db, (ASTNode) table_t, true);
     URI toURI = (ts.partHandle != null) ? ts.partHandle.getDataLocation() : ts.tableHandle.getDataLocation();
 
     // make sure the arguments make sense

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java Mon Dec 22 15:23:32 2008
@@ -25,7 +25,6 @@
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.plan.loadFileDesc;
 import org.apache.hadoop.hive.ql.plan.loadTableDesc;
-import org.antlr.runtime.tree.CommonTree;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.conf.HiveConf;
 
@@ -41,7 +40,7 @@
 
 public class ParseContext {
   private QB qb;
-  private CommonTree ast;
+  private ASTNode ast;
   private HashMap<String, PartitionPruner> aliasToPruner;
   private HashMap<String, SamplePruner> aliasToSamplePruner;
   private HashMap<String, Operator<? extends Serializable>> topOps;
@@ -73,7 +72,7 @@
    * @param topSelOps
    *          list of operators for the selects introduced for column pruning
    */
-  public ParseContext(HiveConf conf, QB qb, CommonTree ast,
+  public ParseContext(HiveConf conf, QB qb, ASTNode ast,
       HashMap<String, PartitionPruner> aliasToPruner,
       HashMap<String, SamplePruner> aliasToSamplePruner,
       HashMap<String, Operator<? extends Serializable>> topOps,
@@ -142,7 +141,7 @@
   /**
    * @return the ast
    */
-  public CommonTree getParseTree() {
+  public ASTNode getParseTree() {
     return ast;
   }
 
@@ -150,7 +149,7 @@
    * @param ast
    *          the parsetree to set
    */
-  public void setParseTree(CommonTree ast) {
+  public void setParseTree(ASTNode ast) {
     this.ast = ast;
   }
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseDriver.java Mon Dec 22 15:23:32 2008
@@ -245,12 +245,32 @@
 
   }
 
-  public CommonTree parse(String command) throws ParseException {
+  /**
+   * Tree adaptor for making antlr return ASTNodes instead of CommonTree nodes
+   * so that the graph walking algorithms and the rules framework defined in
+   * ql.lib can be used with the AST Nodes.
+   */
+  static final TreeAdaptor adaptor = new CommonTreeAdaptor() {
+    /**
+     * Creates an ASTNode for the given token. The ASTNode is a wrapper around antlr's
+     * CommonTree class that implements the Node interface.
+     * 
+     * @param payload The token.
+     * @return Object (which is actually an ASTNode) for the token.
+     */
+    @Override
+    public Object create(Token payload) {
+      return new ASTNode(payload);
+    }
+  };
+  
+  public ASTNode parse(String command) throws ParseException {
     LOG.info("Parsing command: " + command);
       
     HiveLexerX lexer = new HiveLexerX(new ANTLRNoCaseStringStream(command));
     TokenStream tokens = new TokenRewriteStream(lexer);
     HiveParserX parser = new HiveParserX(tokens);
+    parser.setTreeAdaptor(adaptor);
     HiveParser.statement_return r = null;
     try {
       r = parser.statement();
@@ -268,7 +288,7 @@
       throw new ParseException(parser.getErrors());
     }
       
-    return (CommonTree)r.getTree();
+    return (ASTNode)r.getTree();
   }
 }
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java Mon Dec 22 15:23:32 2008
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import org.antlr.runtime.tree.CommonTree;
-
 /**
  * Library of utility functions used in the parse code
  *
@@ -32,7 +30,7 @@
 	 * @param node The parse tree node
 	 * @return boolean
 	 */
-	public static boolean isJoinToken(CommonTree node) {
+	public static boolean isJoinToken(ASTNode node) {
 		if ((node.getToken().getType() == HiveParser.TOK_JOIN)
 				|| (node.getToken().getType() == HiveParser.TOK_LEFTOUTERJOIN)
 				|| (node.getToken().getType() == HiveParser.TOK_RIGHTOUTERJOIN)

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/PartitionPruner.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/PartitionPruner.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/PartitionPruner.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/PartitionPruner.java Mon Dec 22 15:23:32 2008
@@ -20,8 +20,6 @@
 
 import java.util.*;
 
-import org.antlr.runtime.tree.*;
-
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
 import org.apache.hadoop.hive.ql.metadata.*;
@@ -89,7 +87,7 @@
    * @throws SemanticException
    */
   @SuppressWarnings("nls")
-  private exprNodeDesc genExprNodeDesc(CommonTree expr)
+  private exprNodeDesc genExprNodeDesc(ASTNode expr)
   throws SemanticException {
     //  We recursively create the exprNodeDesc.  Base cases:  when we encounter 
     //  a column ref, we convert that into an exprNodeColumnDesc;  when we encounter 
@@ -117,7 +115,7 @@
         }
         else {
           colName = BaseSemanticAnalyzer.unescapeIdentifier(expr.getChild(0).getText());
-          tabAlias = SemanticAnalyzer.getTabAliasForCol(this.metaData, colName, (CommonTree)expr.getChild(0));
+          tabAlias = SemanticAnalyzer.getTabAliasForCol(this.metaData, colName, (ASTNode)expr.getChild(0));
         }
 
         // Set value to null if it's not partition column
@@ -150,7 +148,7 @@
         int childrenBegin = (isFunction ? 1 : 0);
         ArrayList<exprNodeDesc> children = new ArrayList<exprNodeDesc>(expr.getChildCount() - childrenBegin);
         for (int ci=childrenBegin; ci<expr.getChildCount(); ci++) {
-          exprNodeDesc child = genExprNodeDesc((CommonTree)expr.getChild(ci));
+          exprNodeDesc child = genExprNodeDesc((ASTNode)expr.getChild(ci));
           assert(child.getTypeInfo() != null);
           children.add(child);
         }
@@ -205,7 +203,7 @@
     return false;
   }
   
-  public boolean hasPartitionPredicate(CommonTree expr) {
+  public boolean hasPartitionPredicate(ASTNode expr) {
 
     int tokType = expr.getType();
     boolean hasPPred = false;
@@ -227,7 +225,7 @@
         // Create all children
         int childrenBegin = (isFunction ? 1 : 0);
         for (int ci=childrenBegin; ci<expr.getChildCount(); ci++) {
-          hasPPred = (hasPPred || hasPartitionPredicate((CommonTree)expr.getChild(ci)));
+          hasPPred = (hasPPred || hasPartitionPredicate((ASTNode)expr.getChild(ci)));
         }
         break;
       }
@@ -238,7 +236,7 @@
 
   /** Add an expression */
   @SuppressWarnings("nls")
-  public void addExpression(CommonTree expr) throws SemanticException {
+  public void addExpression(ASTNode expr) throws SemanticException {
     LOG.trace("adding pruning Tree = " + expr.toStringTree());
     exprNodeDesc desc = genExprNodeDesc(expr);
     // Ignore null constant expressions
@@ -257,7 +255,7 @@
    * condition.
    */
   @SuppressWarnings("nls")
-  public void addJoinOnExpression(CommonTree expr) throws SemanticException {
+  public void addJoinOnExpression(ASTNode expr) throws SemanticException {
     LOG.trace("adding pruning Tree = " + expr.toStringTree());
     exprNodeDesc desc = genExprNodeDesc(expr);
     // Ignore null constant expressions

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/PrintOpTreeProcessor.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/PrintOpTreeProcessor.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/PrintOpTreeProcessor.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/PrintOpTreeProcessor.java Mon Dec 22 15:23:32 2008
@@ -21,12 +21,12 @@
 import java.io.PrintStream;
 import java.io.Serializable;
 import java.util.HashMap;
-import java.util.Stack;
-
 import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.optimizer.OperatorProcessorContext;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 
-public class PrintOpTreeProcessor implements OperatorProcessor {
+public class PrintOpTreeProcessor implements NodeProcessor {
   
   private PrintStream out;
   private HashMap<Operator<? extends Serializable>, Integer> opMap = new HashMap<Operator<? extends Serializable>, Integer>();
@@ -70,7 +70,8 @@
     return ret.toString();
   }
   
-  public void process(Operator<? extends Serializable> op, OperatorProcessorContext ctx) throws SemanticException {
+  public void process(Node nd, NodeProcessorCtx ctx) throws SemanticException {
+    Operator<? extends Serializable> op = (Operator<? extends Serializable>)nd;
     if (opMap.get(op) == null) {
       opMap.put(op, curNum++);
     }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java Mon Dec 22 15:23:32 2008
@@ -20,8 +20,6 @@
 
 import java.util.Vector;
 
-import org.antlr.runtime.tree.CommonTree;
-
 /**
  * Internal representation of the join tree
  *
@@ -38,10 +36,10 @@
   private boolean       noOuterJoin;
   
   // join conditions
-  private Vector<Vector<CommonTree>> expressions;
+  private Vector<Vector<ASTNode>> expressions;
 
   // filters
-  private Vector<Vector<CommonTree>> filters;
+  private Vector<Vector<ASTNode>> filters;
   
   /**
    * constructor 
@@ -80,11 +78,11 @@
     this.leftAliases = leftAliases;
   }
 
-  public Vector<Vector<CommonTree>> getExpressions() {
+  public Vector<Vector<ASTNode>> getExpressions() {
     return expressions;
   }
 
-  public void setExpressions(Vector<Vector<CommonTree>> expressions) {
+  public void setExpressions(Vector<Vector<ASTNode>> expressions) {
     this.expressions = expressions;
   }
 
@@ -131,14 +129,14 @@
 	/**
 	 * @return the filters
 	 */
-	public Vector<Vector<CommonTree>> getFilters() {
+	public Vector<Vector<ASTNode>> getFilters() {
 		return filters;
 	}
 
 	/**
 	 * @param filters the filters to set
 	 */
-	public void setFilters(Vector<Vector<CommonTree>> filters) {
+	public void setFilters(Vector<Vector<ASTNode>> filters) {
 		this.filters = filters;
 	}
 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java?rev=728823&r1=728822&r2=728823&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/QBParseInfo.java Mon Dec 22 15:23:32 2008
@@ -33,86 +33,86 @@
 
   private boolean isSubQ;
   private String alias;
-  private CommonTree joinExpr;
-  private HashMap<String, CommonTree> aliasToSrc;
-  private HashMap<String, CommonTree> nameToDest;
+  private ASTNode joinExpr;
+  private HashMap<String, ASTNode> aliasToSrc;
+  private HashMap<String, ASTNode> nameToDest;
   private HashMap<String, TableSample> nameToSample;
-  private HashMap<String, CommonTree> destToSelExpr;
-  private HashMap<String, CommonTree> destToWhereExpr;
-  private HashMap<String, CommonTree> destToGroupby;
+  private HashMap<String, ASTNode> destToSelExpr;
+  private HashMap<String, ASTNode> destToWhereExpr;
+  private HashMap<String, ASTNode> destToGroupby;
   /**
    * ClusterBy is a short name for both DistributeBy and SortBy.  
    */
-  private HashMap<String, CommonTree> destToClusterby;
+  private HashMap<String, ASTNode> destToClusterby;
   /**
    * DistributeBy controls the hashcode of the row, which determines which reducer
    * the rows will go to. 
    */
-  private HashMap<String, CommonTree> destToDistributeby;
+  private HashMap<String, ASTNode> destToDistributeby;
   /**
    * SortBy controls the reduce keys, which affects the order of rows 
    * that the reducer receives. 
    */
-  private HashMap<String, CommonTree> destToSortby;
+  private HashMap<String, ASTNode> destToSortby;
   private HashMap<String, Integer>    destToLimit;
   private int outerQueryLimit;
 
   // used by GroupBy
-  private HashMap<String, HashMap<String, CommonTree> > destToAggregationExprs;
-  private HashMap<String, CommonTree> destToDistinctFuncExpr;
+  private HashMap<String, HashMap<String, ASTNode> > destToAggregationExprs;
+  private HashMap<String, ASTNode> destToDistinctFuncExpr;
 
   @SuppressWarnings("unused")
   private static final Log LOG = LogFactory.getLog(QBParseInfo.class.getName());
   
   public QBParseInfo(String alias, boolean isSubQ) {
-    this.aliasToSrc = new HashMap<String, CommonTree>();
-    this.nameToDest = new HashMap<String, CommonTree>();
+    this.aliasToSrc = new HashMap<String, ASTNode>();
+    this.nameToDest = new HashMap<String, ASTNode>();
     this.nameToSample = new HashMap<String, TableSample>();
-    this.destToSelExpr = new HashMap<String, CommonTree>();
-    this.destToWhereExpr = new HashMap<String, CommonTree>();
-    this.destToGroupby = new HashMap<String, CommonTree>();
-    this.destToClusterby = new HashMap<String, CommonTree>();
-    this.destToDistributeby = new HashMap<String, CommonTree>();
-    this.destToSortby = new HashMap<String, CommonTree>();
+    this.destToSelExpr = new HashMap<String, ASTNode>();
+    this.destToWhereExpr = new HashMap<String, ASTNode>();
+    this.destToGroupby = new HashMap<String, ASTNode>();
+    this.destToClusterby = new HashMap<String, ASTNode>();
+    this.destToDistributeby = new HashMap<String, ASTNode>();
+    this.destToSortby = new HashMap<String, ASTNode>();
     this.destToLimit = new HashMap<String, Integer>();
     
-    this.destToAggregationExprs = new HashMap<String, HashMap<String, CommonTree> >();
-    this.destToDistinctFuncExpr = new HashMap<String, CommonTree>();
+    this.destToAggregationExprs = new HashMap<String, HashMap<String, ASTNode> >();
+    this.destToDistinctFuncExpr = new HashMap<String, ASTNode>();
     
     this.alias = alias;
     this.isSubQ = isSubQ;
     this.outerQueryLimit = -1;
   }
 
-  public void setAggregationExprsForClause(String clause, HashMap<String, CommonTree> aggregationTrees) {
+  public void setAggregationExprsForClause(String clause, HashMap<String, ASTNode> aggregationTrees) {
     this.destToAggregationExprs.put(clause, aggregationTrees);
   }
 
-  public HashMap<String, CommonTree> getAggregationExprsForClause(String clause) {
+  public HashMap<String, ASTNode> getAggregationExprsForClause(String clause) {
     return this.destToAggregationExprs.get(clause);
   }
 
-  public void setDistinctFuncExprForClause(String clause, CommonTree ast) {
+  public void setDistinctFuncExprForClause(String clause, ASTNode ast) {
     this.destToDistinctFuncExpr.put(clause, ast);
   }
   
-  public CommonTree getDistinctFuncExprForClause(String clause) {
+  public ASTNode getDistinctFuncExprForClause(String clause) {
     return this.destToDistinctFuncExpr.get(clause);
   }
   
-  public void setSelExprForClause(String clause, CommonTree ast) {
+  public void setSelExprForClause(String clause, ASTNode ast) {
     this.destToSelExpr.put(clause, ast);
   }
 
-  public void setWhrExprForClause(String clause, CommonTree ast) {
+  public void setWhrExprForClause(String clause, ASTNode ast) {
     this.destToWhereExpr.put(clause, ast);
   }
 
-  public void setGroupByExprForClause(String clause, CommonTree ast) {
+  public void setGroupByExprForClause(String clause, ASTNode ast) {
     this.destToGroupby.put(clause, ast);
   }
 
-  public void setDestForClause(String clause, CommonTree ast) {
+  public void setDestForClause(String clause, ASTNode ast) {
     this.nameToDest.put(clause, ast);
   }
 
@@ -121,7 +121,7 @@
    * @param clause the name of the clause
    * @param ast the abstract syntax tree
    */
-  public void setClusterByExprForClause(String clause, CommonTree ast) {
+  public void setClusterByExprForClause(String clause, ASTNode ast) {
     this.destToClusterby.put(clause, ast);
   }
 
@@ -130,7 +130,7 @@
    * @param clause the name of the clause
    * @param ast the abstract syntax tree
    */
-  public void setDistributeByExprForClause(String clause, CommonTree ast) {
+  public void setDistributeByExprForClause(String clause, ASTNode ast) {
     this.destToDistributeby.put(clause, ast);
   }
 
@@ -139,11 +139,11 @@
    * @param clause the name of the clause
    * @param ast the abstract syntax tree
    */
-  public void setSortByExprForClause(String clause, CommonTree ast) {
+  public void setSortByExprForClause(String clause, ASTNode ast) {
     this.destToSortby.put(clause, ast);
   }
 
-  public void setSrcForAlias(String alias, CommonTree ast) {
+  public void setSrcForAlias(String alias, ASTNode ast) {
     this.aliasToSrc.put(alias.toLowerCase(), ast);
   }
 
@@ -155,23 +155,23 @@
     return this.nameToDest.keySet();
   }
 
-  public CommonTree getDestForClause(String clause) {
+  public ASTNode getDestForClause(String clause) {
     return this.nameToDest.get(clause);
   }
 
-  public CommonTree getWhrForClause(String clause) {
+  public ASTNode getWhrForClause(String clause) {
     return this.destToWhereExpr.get(clause);
   }
 
-  public HashMap<String, CommonTree> getDestToWhereExpr() {
+  public HashMap<String, ASTNode> getDestToWhereExpr() {
     return destToWhereExpr;
   }
 
-  public CommonTree getGroupByForClause(String clause) {
+  public ASTNode getGroupByForClause(String clause) {
     return this.destToGroupby.get(clause);
   }
 
-  public CommonTree getSelForClause(String clause) {
+  public ASTNode getSelForClause(String clause) {
     return this.destToSelExpr.get(clause);
   }
 
@@ -180,7 +180,7 @@
    * @param clause the name of the clause
    * @return the abstract syntax tree
    */
-  public CommonTree getClusterByForClause(String clause) {
+  public ASTNode getClusterByForClause(String clause) {
     return this.destToClusterby.get(clause);
   }
 
@@ -189,7 +189,7 @@
    * @param clause the name of the clause
    * @return the abstract syntax tree
    */
-  public CommonTree getDistributeByForClause(String clause) {
+  public ASTNode getDistributeByForClause(String clause) {
     return this.destToDistributeby.get(clause);
   }
 
@@ -198,11 +198,11 @@
    * @param clause the name of the clause
    * @return the abstract syntax tree
    */
-  public CommonTree getSortByForClause(String clause) {
+  public ASTNode getSortByForClause(String clause) {
     return this.destToSortby.get(clause);
   }
 
-  public CommonTree getSrcForAlias(String alias) {
+  public ASTNode getSrcForAlias(String alias) {
     return this.aliasToSrc.get(alias.toLowerCase());
   }
 
@@ -214,11 +214,11 @@
     return this.isSubQ;
   }
 
-  public CommonTree getJoinExpr() {
+  public ASTNode getJoinExpr() {
     return this.joinExpr;
   }
 
-  public void setJoinExpr(CommonTree joinExpr) {
+  public void setJoinExpr(ASTNode joinExpr) {
     this.joinExpr = joinExpr;
   }
 
@@ -260,40 +260,40 @@
        (!destToClusterby.isEmpty()))
       return false;
     
-    Iterator<Map.Entry<String, HashMap<String, CommonTree>>> aggrIter = destToAggregationExprs.entrySet().iterator();
+    Iterator<Map.Entry<String, HashMap<String, ASTNode>>> aggrIter = destToAggregationExprs.entrySet().iterator();
     while (aggrIter.hasNext()) {
-      HashMap<String, CommonTree> h = aggrIter.next().getValue();
+      HashMap<String, ASTNode> h = aggrIter.next().getValue();
       if ((h != null) && (!h.isEmpty()))
         return false;
     }
       	
     if (!destToDistinctFuncExpr.isEmpty()) {
-      Iterator<Map.Entry<String, CommonTree>> distn = destToDistinctFuncExpr.entrySet().iterator();
+      Iterator<Map.Entry<String, ASTNode>> distn = destToDistinctFuncExpr.entrySet().iterator();
       while (distn.hasNext()) {
-        CommonTree ct = distn.next().getValue();
+        ASTNode ct = distn.next().getValue();
         if (ct != null) 
           return false;
       }
     }
         
-    Iterator<Map.Entry<String, CommonTree>> iter = nameToDest.entrySet().iterator();
+    Iterator<Map.Entry<String, ASTNode>> iter = nameToDest.entrySet().iterator();
     while (iter.hasNext()) {
-      Map.Entry<String, CommonTree> entry = iter.next();
-      CommonTree v = entry.getValue();
-      if (!(((CommonTree)v.getChild(0)).getToken().getType() == HiveParser.TOK_TMP_FILE))
+      Map.Entry<String, ASTNode> entry = iter.next();
+      ASTNode v = entry.getValue();
+      if (!(((ASTNode)v.getChild(0)).getToken().getType() == HiveParser.TOK_TMP_FILE))
         return false;
     }
       	
     iter = destToSelExpr.entrySet().iterator();
     while (iter.hasNext()) {
-      Map.Entry<String, CommonTree> entry = iter.next();
-      CommonTree selExprList = entry.getValue();
+      Map.Entry<String, ASTNode> entry = iter.next();
+      ASTNode selExprList = entry.getValue();
       // Iterate over the selects
       for (int i = 0; i < selExprList.getChildCount(); ++i) {
         
         // list of the columns
-        CommonTree selExpr = (CommonTree) selExprList.getChild(i);
-        CommonTree sel = (CommonTree)selExpr.getChild(0);
+        ASTNode selExpr = (ASTNode) selExprList.getChild(i);
+        ASTNode sel = (ASTNode)selExpr.getChild(0);
         
         if (sel.getToken().getType() != HiveParser.TOK_ALLCOLREF)
           return false;