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/09 06:46:23 UTC

svn commit: r724613 - in /hadoop/hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/parse/

Author: zshao
Date: Mon Dec  8 21:46:23 2008
New Revision: 724613

URL: http://svn.apache.org/viewvc?rev=724613&view=rev
Log:
HIVE-140. Event Based Infrastructure for Syntax Trees in the compiler.
(Ashish Thusoo through zshao)

Added:
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEvent.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventDispatcher.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventProcessor.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTProcessor.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTEventDispatcher.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTProcessor.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
Modified:
    hadoop/hive/trunk/CHANGES.txt

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=724613&r1=724612&r2=724613&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Mon Dec  8 21:46:23 2008
@@ -6,6 +6,9 @@
 
   NEW FEATURES
 
+    HIVE-140. Event Based Infrastructure for Syntax Trees in the compiler.
+    (Ashish Thusoo through zshao)
+
     HIVE-98. Dependency management with hadoop core using ivy.
     (Ashish Thusoo through zshao)
 

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEvent.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEvent.java?rev=724613&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEvent.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEvent.java Mon Dec  8 21:46:23 2008
@@ -0,0 +1,108 @@
+/**
+ * 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;
+
+/**
+ * Enumeration that encapsulates the various event types that are seen
+ * while processing the parse tree (in an implementation of the ParseTreeProcessor).
+ * These event types are used to register the different event processors with
+ * the parse tree processor.
+ *
+ */
+public enum ASTEvent {
+	
+	/**
+	 * Query event
+	 */
+	QUERY("QUERY"),
+	
+	/**
+	 * Union
+	 */
+	UNION("UNION"),
+	
+	/**
+	 * Source Table (table in the from clause)
+	 */
+	SRC_TABLE("SRC_TABLE"),
+
+	/**
+	 * Any type of Destination (this fires for hdfs directory, local directory and table)
+	 */
+	DESTINATION("DESTINATION"),
+	
+	/**
+	 * Select clause
+	 */
+	SELECT_CLAUSE("SELECT_CLAUSE"),
+	
+	/**
+	 * Join clause
+	 */
+	JOIN_CLAUSE("JOIN_CLAUSE"),
+	
+	/**
+	 * Where clause
+	 */
+	WHERE_CLAUSE("WHERE_CLAUSE"),
+	
+	/**
+	 * CLusterby clause
+	 */
+	CLUSTERBY_CLAUSE("CLUSTERBY_CLAUSE"),
+	
+	/**
+	 * Group by clause
+	 */
+	GROUPBY_CLAUSE("GROUPBY_CLAUSE"),
+	
+	/**
+	 * Limit clause
+	 */
+	LIMIT_CLAUSE("LIMIT_CLAUSE"),
+	
+	/**
+	 * Subquery
+	 */
+	SUBQUERY("SUBQUERY");
+	
+	/**
+	 * The name of the event (string representation of the event)
+	 */
+	private final String name;
+	
+	/**
+	 * Constructs the event
+	 * 
+	 * @param name The name(String representation of the event)
+	 */
+	ASTEvent(String name) {
+      this.name = name;
+	}
+	
+	/**
+	 * String representation of the event
+	 * 
+	 * @return String
+	 */
+	@Override
+	public String toString() {
+		return name;
+	}
+}

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventDispatcher.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventDispatcher.java?rev=724613&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventDispatcher.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventDispatcher.java Mon Dec  8 21:46:23 2008
@@ -0,0 +1,43 @@
+/**
+ * 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;
+
+/**
+ * Dispatches ParseTreeEvent to the appropriate ParseTreeEventProcessor
+ */
+public interface ASTEventDispatcher {
+	
+	/**
+	 * Registers the event processor with the event
+	 * 
+	 * @param evt The parse tree event
+	 * @param evt_p The associated parse tree event processor
+	 */
+	void register(ASTEvent evt, ASTEventProcessor evt_p);
+	
+	/**
+	 * Dispatches the parse tree event to a registered event processor
+	 * 
+	 * @param evt The parse tree event to dispatch
+	 * @param pt The parse subtree to dispatch to the event processor
+	 */
+	void dispatch(ASTEvent evt, CommonTree pt);
+}

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventProcessor.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventProcessor.java?rev=724613&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventProcessor.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTEventProcessor.java Mon Dec  8 21:46:23 2008
@@ -0,0 +1,37 @@
+/**
+ * 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;
+
+/**
+ * Interface that a parse tree event processor needs to implement. Classes implementing
+ * this interface and registered with the ParseTreeProcessor are called by the later
+ * when associated ParseTreeEvents are processed
+ *
+ */
+public interface ASTEventProcessor {
+
+  /**
+   * Processes the parse subtree corresponding to the event
+   * 
+   * @param pt The parse subtree to process
+   */
+  public void process(CommonTree pt);
+}

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTProcessor.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTProcessor.java?rev=724613&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTProcessor.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ASTProcessor.java Mon Dec  8 21:46:23 2008
@@ -0,0 +1,43 @@
+/**
+ * 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;
+
+/**
+ * Interface that a parse tree processor needs to implement
+ */
+public interface ASTProcessor {
+  
+	/**
+	 * Sets the event dispatcher for the processors
+	 * 
+	 * @param dispatcher The parse tree event dispatcher
+	 */
+	void setDispatcher(ASTEventDispatcher dispatcher);
+	
+	/**
+	 * Processes the parse tree and calls the registered event processors
+	 * for the associated parse tree events
+	 * 
+	 * @param pt The parse tree to process
+	 */
+	void process(CommonTree pt);
+}
+

Added: 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=724613&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTEventDispatcher.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTEventDispatcher.java Mon Dec  8 21:46:23 2008
@@ -0,0 +1,84 @@
+/**
+ * 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
+	 */
+	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);
+	}
+}

Added: 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=724613&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTProcessor.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DefaultASTProcessor.java Mon Dec  8 21:46:23 2008
@@ -0,0 +1,108 @@
+/**
+ * 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;
+	}
+
+}

Added: 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=724613&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java Mon Dec  8 21:46:23 2008
@@ -0,0 +1,44 @@
+/**
+ * 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;
+
+/**
+ * Library of utility functions used in the parse code
+ *
+ */
+public class ParseUtils {
+	
+	/**
+	 * Tests whether the parse tree node is a join token
+	 * 
+	 * @param node The parse tree node
+	 * @return boolean
+	 */
+	public static boolean isJoinToken(CommonTree node) {
+		if ((node.getToken().getType() == HiveParser.TOK_JOIN)
+				|| (node.getToken().getType() == HiveParser.TOK_LEFTOUTERJOIN)
+				|| (node.getToken().getType() == HiveParser.TOK_RIGHTOUTERJOIN)
+				|| (node.getToken().getType() == HiveParser.TOK_FULLOUTERJOIN))
+			return true;
+
+		return false;
+	}
+}