You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ga...@apache.org on 2008/05/15 21:49:36 UTC

svn commit: r656811 - in /incubator/pig/branches/types: ./ src/org/apache/pig/impl/physicalLayer/plans/ src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ test/org/apache/pig/test/

Author: gates
Date: Thu May 15 12:49:36 2008
New Revision: 656811

URL: http://svn.apache.org/viewvc?rev=656811&view=rev
Log:
PIG-161 Checking in POMap parts of Shubham's CastAndMapLookupPatch.  See PIG-161 for comments on cast part.


Added:
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestPOMapLookUp.java
Modified:
    incubator/pig/branches/types/build.xml
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java

Modified: incubator/pig/branches/types/build.xml
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=656811&r1=656810&r2=656811&view=diff
==============================================================================
--- incubator/pig/branches/types/build.xml (original)
+++ incubator/pig/branches/types/build.xml Thu May 15 12:49:36 2008
@@ -146,7 +146,7 @@
         		**/test/TestPODistinct.java, **/test/TestPOSort.java,
         		**/test/TestSchema.java, **/test/TestLogicalPlanBuilder.java,**/test/TestUnion.java, **/test/TestMRCompiler.java,
                 **/test/FakeFSInputStream.java, **/test/Util.java, **/test/TestJobSubmission.java,
-        		**/test/TestLocalJobSubmission.java,
+        		**/test/TestLocalJobSubmission.java, **/test/TestPOMapLookUp.java,
                 **/logicalLayer/*.java, **/logicalLayer/parser/NodeIdGenerator.java,
                 **/logicalLayer/schema/*.java, **/physicalLayer/topLevelOperators/*.java,
                 **/physicalLayer/topLevelOperators/**/*.java, **/physicalLayer/plans/*.java,
@@ -276,6 +276,7 @@
                 	<include name="**/TestSchema.java" /> 
                 	<include name="**/TestLogicalPlanBuilder.java" />
                 	<include name="**/TestLocalJobSubmission.java" />
+                	<include name="**/TestPOMapLookUp.java" />
                 	
                     <!--
                     <include name="**/*Test*.java" />

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java?rev=656811&r1=656810&r2=656811&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java Thu May 15 12:49:36 2008
@@ -22,6 +22,8 @@
 import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ConstantExpression;
 import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ExpressionOperator;
 import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POBinCond;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POCast;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POMapLookUp;
 import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POProject;
 //import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.EqualToExpr;
 //import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.GTOrEqualToExpr;
@@ -118,4 +120,14 @@
         
     }
 
+	public void visitMapLookUp(POMapLookUp mapLookUp) {
+		// TODO Auto-generated method stub
+		
+	}
+
+	public void visitCast(POCast cast) {
+		// TODO Auto-generated method stub
+		
+	}
+
 }

Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java?rev=656811&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POMapLookUp.java Thu May 15 12:49:36 2008
@@ -0,0 +1,150 @@
+/*
+ * 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.pig.impl.physicalLayer.topLevelOperators.expressionOperators;
+
+import java.util.Map;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.OperatorKey;
+import org.apache.pig.impl.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
+
+public class POMapLookUp extends ExpressionOperator {
+	
+	private Object key;
+
+	public POMapLookUp(OperatorKey k) {
+		super(k);
+	}
+	
+	public POMapLookUp(OperatorKey k, int rp) {
+		super(k, rp);
+	}
+	
+	public POMapLookUp(OperatorKey k, int rp, Object key) {
+		super(k, rp);
+		this.key = key;
+	}
+	
+	public void setKey(Object key) {
+		this.key = key;
+	}
+
+	@Override
+	public void visit(ExprPlanVisitor v) throws VisitorException {
+		v.visitMapLookUp(this);
+
+	}
+
+	@Override
+	public String name() {
+		// TODO Auto-generated method stub
+		return "POMapLookUp - " + mKey.toString();
+	}
+
+	@Override
+	public boolean supportsMultipleInputs() {
+		// TODO Auto-generated method stub
+		return false;
+	}
+	
+	@Override
+	public Result processInput() throws ExecException {
+        Result res = new Result();
+        Map<Object, Object> inpValue = null;
+        if (input == null && (inputs == null || inputs.size()==0)) {
+//            log.warn("No inputs found. Signaling End of Processing.");
+            res.returnStatus = POStatus.STATUS_EOP;
+            return res;
+        }
+        if (!isInputAttached())
+            return inputs.get(0).getNext(inpValue);
+        else {
+            res.result = input;
+            res.returnStatus = POStatus.STATUS_OK;
+            detachInput();
+            return res;
+        }
+    }
+	
+	private Result getNext() throws ExecException {
+		Result res = processInput();
+		if(res.result != null && res.returnStatus == POStatus.STATUS_OK) {
+			res.result = ((Map)res.result).get(key);
+		}
+		return res;
+	}
+
+	@Override
+	public Result getNext(Boolean b) throws ExecException {
+		return getNext();
+	}
+
+	@Override
+	public Result getNext(DataBag db) throws ExecException {
+		return getNext();
+	}
+
+	@Override
+	public Result getNext(DataByteArray ba) throws ExecException {
+		return getNext();
+	}
+
+	@Override
+	public Result getNext(Double d) throws ExecException {
+		return getNext();
+	}
+
+	@Override
+	public Result getNext(Float f) throws ExecException {
+		return getNext();
+	}
+
+	@Override
+	public Result getNext(Integer i) throws ExecException {
+		return getNext();
+	}
+
+	@Override
+	public Result getNext(Long l) throws ExecException {
+		return getNext();
+	}
+
+	@Override
+	public Result getNext(Map m) throws ExecException {
+		return getNext();
+	}
+
+	@Override
+	public Result getNext(String s) throws ExecException {
+		return getNext();
+	}
+
+	@Override
+	public Result getNext(Tuple t) throws ExecException {
+		return getNext();
+	}
+	
+	
+
+}

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestPOMapLookUp.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPOMapLookUp.java?rev=656811&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPOMapLookUp.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPOMapLookUp.java Thu May 15 12:49:36 2008
@@ -0,0 +1,68 @@
+/*
+ * 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.pig.test;
+
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.impl.logicalLayer.OperatorKey;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.plans.ExprPlan;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POMapLookUp;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.POProject;
+import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.test.utils.GenRandomData;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestPOMapLookUp extends TestCase {
+	
+	Random r = new Random();
+	Map<Integer, String> map;// = GenRandomData.genRandMap(r, 10);
+	
+	@Test
+	public void testMapLookUp() throws PlanException, ExecException {
+		
+		POProject prj = new POProject(new OperatorKey("", r.nextLong()), -1, 0);
+		POMapLookUp op = new POMapLookUp(new OperatorKey("", r.nextLong()), -1);
+		ExprPlan plan = new ExprPlan();
+		plan.add(op);
+		plan.add(prj);
+		plan.connect(prj, op);
+		
+		for(int i = 0 ; i < 10; i++) {
+			map = GenRandomData.genRandMap(r, 10);
+			Tuple t = TupleFactory.getInstance().newTuple();
+			t.append(map);
+			for(Map.Entry<Integer, String> e : map.entrySet()) {
+				op.setKey(e.getKey());
+				plan.attachInput(t);
+				Result res = op.getNext(map);
+				//System.out.println(e.getValue() + " : " + res.result);
+				assertEquals(e.getValue(), res.result);
+			}
+			
+			
+		}
+		
+	}
+}