You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by sm...@apache.org on 2009/07/24 02:59:42 UTC
svn commit: r797290 - in /hadoop/pig/trunk: CHANGES.txt
src/org/apache/pig/data/DataType.java
src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt
test/org/apache/pig/test/TestLogicalPlanBuilder.java
Author: sms
Date: Fri Jul 24 00:59:41 2009
New Revision: 797290
URL: http://svn.apache.org/viewvc?rev=797290&view=rev
Log:
PIG-773: Empty complex constants (empty bag, empty tuple and empty map) should be supported (ashutoshc via sms)
Modified:
hadoop/pig/trunk/CHANGES.txt
hadoop/pig/trunk/src/org/apache/pig/data/DataType.java
hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt
hadoop/pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java
Modified: hadoop/pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=797290&r1=797289&r2=797290&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Fri Jul 24 00:59:41 2009
@@ -40,6 +40,9 @@
BUG FIXES
+ PIG-773: Empty complex constants (empty bag, empty tuple and empty map)
+ should be supported (ashutoshc via sms)
+
PIG-695: Pig should not fail when error logs cannot be created (sms)
PIG-878: Pig is returning too many blocks in the input split. (arunc via gates).
Modified: hadoop/pig/trunk/src/org/apache/pig/data/DataType.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DataType.java?rev=797290&r1=797289&r2=797290&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/data/DataType.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/data/DataType.java Fri Jul 24 00:59:41 2009
@@ -993,6 +993,7 @@
DataBag b = (DataBag)o;
long bagSize = b.size();
Schema schema = null;
+ Schema bagSchema = null;
if(bagSize != 0) {
Iterator<Tuple> it = b.iterator();
@@ -1003,8 +1004,9 @@
schema = schemas.get(0);
if(null == schema) {
Schema.FieldSchema tupleFs = new Schema.FieldSchema(null, null, TUPLE);
- Schema bagSchema = new Schema(tupleFs);
- return new Schema.FieldSchema(null, null, BAG);
+ bagSchema = new Schema(tupleFs);
+ bagSchema.setTwoLevelAccessRequired(true);
+ return new Schema.FieldSchema(null, bagSchema, BAG);
}
int schemaSize = schema.size();
@@ -1012,19 +1014,19 @@
Schema currSchema = schemas.get(i);
if((null == currSchema) || (currSchema.size() != schemaSize)) {
Schema.FieldSchema tupleFs = new Schema.FieldSchema(null, null, TUPLE);
- Schema bagSchema = new Schema(tupleFs);
+ bagSchema = new Schema(tupleFs);
bagSchema.setTwoLevelAccessRequired(true);
return new Schema.FieldSchema(null, bagSchema, BAG);
}
schema = Schema.mergeSchema(schema, currSchema, false, false, false);
}
+ Schema.FieldSchema tupleFs = new Schema.FieldSchema(null, schema, TUPLE);
+ bagSchema = new Schema(tupleFs);
+ // since this schema has tuple field schema which internally
+ // has a list of field schemas for the actual items in the bag
+ // an access to any field in the bag is a two level access
+ bagSchema.setTwoLevelAccessRequired(true);
}
- Schema.FieldSchema tupleFs = new Schema.FieldSchema(null, schema, TUPLE);
- Schema bagSchema = new Schema(tupleFs);
- // since this schema has tuple field schema which internally
- // has a list of field schemas for the actual items in the bag
- // an access to any field in the bag is a two level access
- bagSchema.setTwoLevelAccessRequired(true);
return new Schema.FieldSchema(null, bagSchema, BAG);
}
default: {
Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt?rev=797290&r1=797289&r2=797290&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt (original)
+++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Fri Jul 24 00:59:41 2009
@@ -3430,7 +3430,9 @@
log.trace("Entering bag");
}
{
- ("(" t = Tuple() {bag.add(t);} ")" ("," "(" t = Tuple() {bag.add(t);} ")" )* )
+ (("(" t = Tuple() {bag.add(t);} ")" ("," "(" t = Tuple() {bag.add(t);} ")" )* )
+ |{ } // Match empty content in bag.
+ )
{
log.trace("Exiting bag");
return bag;
@@ -3443,19 +3445,19 @@
Object obj = null;
TupleFactory tupleFactory = TupleFactory.getInstance();
ArrayList<Object> objList = new ArrayList<Object>();
+ Tuple tuple;
log.trace("Entering Tuple");
}
{
(
- (
(
obj = Datum() {log.debug("Adding " + obj + " to the list: " + objList); objList.add(obj);}
( LOOKAHEAD(2) "," obj = Datum() {log.debug("Adding " + obj + " to the list: " + objList); objList.add(obj);})*
)
- )
+ { tuple = tupleFactory.newTuple(objList); }
+ |{ tuple = tupleFactory.newTuple(0); } // Match empty content in tuple.
)
{
- Tuple tuple = tupleFactory.newTuple(objList);
log.trace("Exiting Tuple");
return tuple;
}
@@ -3468,7 +3470,9 @@
}
{
- ( KeyValuePair(keyValues) ("," KeyValuePair(keyValues))* )
+ (( KeyValuePair(keyValues) ("," KeyValuePair(keyValues))* )
+ |{ } // Match empty content in map.
+ )
{
log.trace("Exiting Map");
return keyValues;
Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java?rev=797290&r1=797289&r2=797290&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestLogicalPlanBuilder.java Fri Jul 24 00:59:41 2009
@@ -41,6 +41,7 @@
import org.apache.pig.builtin.PigStorage;
import org.apache.pig.data.DataBag;
import org.apache.pig.data.Tuple;
+import org.apache.pig.data.BagFactory;
import org.apache.pig.data.TupleFactory;
import org.apache.pig.impl.PigContext;
import org.apache.pig.ExecType;
@@ -1927,6 +1928,137 @@
assertTrue(Schema.equals(foreach.getSchema(), new Schema(bagFs), false, true));
}
+ @Test
+ public void testEmptyTupleConst() throws FrontendException{
+
+ LogicalPlan lp = buildPlan("a = foreach (load 'b') generate ();");
+ LOForEach foreach = (LOForEach) lp.getLeaves().get(0);
+ LogicalOperator logOp = foreach.getForEachPlans().get(0).getLeaves().get(0);
+ assertTrue( logOp instanceof LOConst);
+
+ LOConst loConst = (LOConst)logOp;
+ assertTrue(loConst.getType() == DataType.TUPLE);
+ assertTrue(loConst.getValue() instanceof Tuple);
+ assertTrue(loConst.getValue().equals(TupleFactory.getInstance().newTuple()));
+
+ Schema.FieldSchema tupleFs = new Schema.FieldSchema(null, null, DataType.TUPLE);
+ Schema expectedSchema = new Schema(tupleFs);
+
+ assertTrue(Schema.equals(foreach.getSchema(), expectedSchema, false, true));
+ }
+
+ @Test
+ public void testEmptyMapConst() throws FrontendException{
+
+ LogicalPlan lp = buildPlan("a = foreach (load 'b') generate [];");
+ LOForEach foreach = (LOForEach) lp.getLeaves().get(0);
+ LogicalOperator logOp = foreach.getForEachPlans().get(0).getLeaves().get(0);
+ assertTrue( logOp instanceof LOConst);
+
+ LOConst loConst = (LOConst)logOp;
+ assertTrue(loConst.getType() == DataType.MAP);
+ assertTrue(loConst.getValue() instanceof Map);
+ assertTrue(loConst.getValue().equals(new HashMap<String,Object>()));
+
+ Schema.FieldSchema mapFs = new Schema.FieldSchema(null, null, DataType.MAP);
+ Schema expectedSchema = new Schema(mapFs);
+
+ assertTrue(Schema.equals(foreach.getSchema(), expectedSchema, false, true));
+ }
+
+ @Test
+ public void testEmptyBagConst() throws FrontendException{
+
+ LogicalPlan lp = buildPlan("a = foreach (load 'b') generate {};");
+ LOForEach foreach = (LOForEach) lp.getLeaves().get(0);
+ LogicalOperator logOp = foreach.getForEachPlans().get(0).getLeaves().get(0);
+ assertTrue( logOp instanceof LOConst);
+
+ LOConst loConst = (LOConst)logOp;
+ assertTrue(loConst.getType() == DataType.BAG);
+ assertTrue(loConst.getValue() instanceof DataBag);
+ assertTrue(loConst.getValue().equals(BagFactory.getInstance().newDefaultBag()));
+
+ Schema.FieldSchema bagFs = new Schema.FieldSchema(null, null, DataType.BAG);
+ Schema expectedSchema = new Schema(bagFs);
+
+ assertTrue(Schema.equals(foreach.getSchema(), expectedSchema, false, true));
+ }
+
+ @Test
+ public void testEmptyTupConstRecursive1() throws FrontendException{
+
+ LogicalPlan lp = buildPlan("a = foreach (load 'b') generate (());");
+ LOForEach foreach = (LOForEach) lp.getLeaves().get(0);
+
+ Schema.FieldSchema tupleFs = new Schema.FieldSchema(null, null, DataType.TUPLE);
+ Schema tupleSchema = new Schema(tupleFs);
+ Schema.FieldSchema tupleFs2 = new Schema.FieldSchema(null, tupleSchema, DataType.TUPLE);
+ Schema expectedSchema = new Schema(tupleFs2);
+
+ assertTrue(Schema.equals(foreach.getSchema(), expectedSchema, false, true));
+ }
+
+ @Test
+ public void testEmptyTupConstRecursive2() throws FrontendException{
+
+ LogicalPlan lp = buildPlan("a = foreach (load 'b') generate ([]);");
+ LOForEach foreach = (LOForEach) lp.getLeaves().get(0);
+
+ Schema.FieldSchema mapFs = new Schema.FieldSchema(null, null, DataType.MAP);
+ Schema tupleSchema = new Schema(mapFs);
+ Schema.FieldSchema tupleFs = new Schema.FieldSchema(null, tupleSchema, DataType.TUPLE);
+ Schema expectedSchema = new Schema(tupleFs);
+
+ assertTrue(Schema.equals(foreach.getSchema(), expectedSchema, false, true));
+ }
+
+ @Test
+ public void testEmptyTupConstRecursive3() throws FrontendException{
+
+ LogicalPlan lp = buildPlan("a = foreach (load 'b') generate ({});");
+ LOForEach foreach = (LOForEach) lp.getLeaves().get(0);
+
+ Schema.FieldSchema bagFs = new Schema.FieldSchema(null, null, DataType.BAG);
+ Schema innerSchema = new Schema(bagFs);
+ Schema.FieldSchema outerTupleFs = new Schema.FieldSchema(null,innerSchema,DataType.TUPLE);
+ Schema expectedSchema = new Schema(outerTupleFs);
+
+ assertTrue(Schema.equals(foreach.getSchema(), expectedSchema, false, true));
+ }
+
+ @Test
+ public void testEmptyBagConstRecursive() throws FrontendException{
+
+ LogicalPlan lp = buildPlan("a = foreach (load 'b') generate {()};");
+ LOForEach foreach = (LOForEach) lp.getLeaves().get(0);
+
+ Schema.FieldSchema bagFs = new Schema.FieldSchema(null,null,DataType.TUPLE);
+ Schema bagSchema = new Schema(bagFs);
+ bagSchema.setTwoLevelAccessRequired(true);
+
+ Schema.FieldSchema outerBagFs = new Schema.FieldSchema(null,bagSchema,DataType.BAG);
+ Schema expectedSchema = new Schema(outerBagFs);
+
+ assertTrue(Schema.equals(foreach.getSchema(), expectedSchema, false, true));
+ }
+
+ @Test
+ public void testRandomEmptyConst(){
+ // Various random scripts to test recursive nature of parser with empty constants.
+
+ buildPlan("a = foreach (load 'b') generate {({})};");
+ buildPlan("a = foreach (load 'b') generate ({()});");
+ buildPlan("a = foreach (load 'b') generate {(),()};");
+ buildPlan("a = foreach (load 'b') generate ({},{});");
+ buildPlan("a = foreach (load 'b') generate ((),());");
+ buildPlan("a = foreach (load 'b') generate ([],[]);");
+ buildPlan("a = foreach (load 'b') generate {({},{})};");
+ buildPlan("a = foreach (load 'b') generate {([],[])};");
+ buildPlan("a = foreach (load 'b') generate (({},{}));");
+ buildPlan("a = foreach (load 'b') generate (([],[]));");
+ }
+
private void printPlan(LogicalPlan lp) {
LOPrinter graphPrinter = new LOPrinter(System.err, lp);