svn commit: r921974 - in /hadoop/pig/trunk: ./ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/ src/org/
Author: daijy Date: Thu Mar 11 18:33:26 2010 New Revision: 921974 URL: http://svn.apache.org/viewvc?rev=921974view=rev Log: PIG-1262: Additional findbugs and javac warnings Modified: hadoop/pig/trunk/CHANGES.txt hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReducePOStoreImpl.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PhyPlanSetter.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigOutputCommitter.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigSplit.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/SecondaryKeyOptimizer.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/WeightedRangePartitioner.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java hadoop/pig/trunk/src/org/apache/pig/builtin/PigStorage.java hadoop/pig/trunk/src/org/apache/pig/impl/io/ReadToEndLoader.java hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/schema/Schema.java hadoop/pig/trunk/src/org/apache/pig/impl/util/LogUtils.java hadoop/pig/trunk/src/org/apache/pig/pen/physicalOperators/POCogroup.java hadoop/pig/trunk/src/org/apache/pig/pen/physicalOperators/POCross.java hadoop/pig/trunk/src/org/apache/pig/pen/physicalOperators/POSplit.java hadoop/pig/trunk/test/findbugsExcludeFile.xml Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=921974r1=921973r2=921974view=diff == --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Thu Mar 11 18:33:26 2010 @@ -66,6 +66,8 @@ manner (rding via pradeepkth) IMPROVEMENTS +PIG-1262: Additional findbugs and javac warnings (daijy) + PIG-1248: [piggybank] some useful String functions (dvryaboy) PIG-1251: Move SortInfo calculation earlier in compilation (ashutoshc) Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java?rev=921974r1=921973r2=921974view=diff == --- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java Thu Mar 11 18:33:26 2010 @@ -2063,7 +2063,7 @@ public class MRCompiler extends PhyPlanV if(val=0) val = pigContext.defaultParallel; if (val=0) -val = ((JobConf)((HExecutionEngine)eng).getJobConf()).getNumReduceTasks(); +val = ((HExecutionEngine)eng).getJobConf().getNumReduceTasks(); if (val=0) val = 1; } catch (Exception e) { Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReducePOStoreImpl.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReducePOStoreImpl.java?rev=921974r1=921973r2=921974view=diff == --- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReducePOStoreImpl.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReducePOStoreImpl.java Thu Mar 11 18:33:26 2010 @@ -69,7 +69,7 @@ public class MapReducePOStoreImpl extend PigOutputFormat.setLocation(context, store); OutputFormat outputFormat = null; try { -outputFormat = (OutputFormat)storeFunc.getOutputFormat(); +outputFormat = storeFunc.getOutputFormat(); // create a new record writer writer = outputFormat.getRecordWriter(context); Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PhyPlanSetter.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PhyPlanSetter.java?rev=921974r1=921973r2=921974view=diff == --- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PhyPlanSetter.java (original) +++
svn commit: r921975 - in /hadoop/pig/trunk: CHANGES.txt src/org/apache/pig/builtin/Utf8StorageConverter.java test/org/apache/pig/test/TestTextDataParser.java
Author: daijy Date: Thu Mar 11 18:37:11 2010 New Revision: 921975 URL: http://svn.apache.org/viewvc?rev=921975view=rev Log: PIG-1275: empty bag in PigStorage read as null Modified: hadoop/pig/trunk/CHANGES.txt hadoop/pig/trunk/src/org/apache/pig/builtin/Utf8StorageConverter.java hadoop/pig/trunk/test/org/apache/pig/test/TestTextDataParser.java Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=921975r1=921974r2=921975view=diff == --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Thu Mar 11 18:37:11 2010 @@ -147,6 +147,8 @@ OPTIMIZATIONS BUG FIXES +PIG-1275: empty bag in PigStorage read as null (daijy) + PIG-1252: Diamond splitter does not generate correct results when using Multi-query optimization (rding) Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/Utf8StorageConverter.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/Utf8StorageConverter.java?rev=921975r1=921974r2=921975view=diff == --- hadoop/pig/trunk/src/org/apache/pig/builtin/Utf8StorageConverter.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/builtin/Utf8StorageConverter.java Thu Mar 11 18:37:11 2010 @@ -20,6 +20,7 @@ package org.apache.pig.builtin; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.PushbackInputStream; import java.util.Arrays; import java.util.HashMap; import java.util.Map; @@ -67,7 +68,7 @@ public class Utf8StorageConverter implem } } -private DataBag consumeBag(ByteArrayInputStream in, ResourceFieldSchema fieldSchema) throws IOException { +private DataBag consumeBag(PushbackInputStream in, ResourceFieldSchema fieldSchema) throws IOException { if (fieldSchema==null) { throw new IOException(Schema is null); } @@ -85,7 +86,8 @@ public class Utf8StorageConverter implem DataBag db = DefaultBagFactory.getInstance().newDefaultBag(); while (true) { t = consumeTuple(in, fs); -db.add(t); +if (t!=null) +db.add(t); while ((buf=in.read())!='}'buf!=',') { if (buf==-1) { throw new IOException(Unexpect end of bag); @@ -97,17 +99,21 @@ public class Utf8StorageConverter implem return db; } -private Tuple consumeTuple(ByteArrayInputStream in, ResourceFieldSchema fieldSchema) throws IOException { +private Tuple consumeTuple(PushbackInputStream in, ResourceFieldSchema fieldSchema) throws IOException { if (fieldSchema==null) { throw new IOException(Schema is null); } int buf; ByteArrayOutputStream mOut; -while ((buf=in.read())!='(') { +while ((buf=in.read())!='('||buf=='}') { if (buf==-1) { throw new IOException(Unexpect end of tuple); } +if (buf=='}') { +in.unread(buf); +return null; +} } Tuple t = DefaultTupleFactory.getInstance().newTuple(); if (fieldSchema.getSchema()!=null fieldSchema.getSchema().getFields().length!=0) { @@ -172,7 +178,7 @@ public class Utf8StorageConverter implem return t; } -private MapString, Object consumeMap(ByteArrayInputStream in, ResourceFieldSchema fieldSchema) throws IOException { +private MapString, Object consumeMap(PushbackInputStream in, ResourceFieldSchema fieldSchema) throws IOException { if (fieldSchema==null) { throw new IOException(Schema is null); } @@ -232,7 +238,7 @@ public class Utf8StorageConverter implem return m; } -private Object consumeComplexType(ByteArrayInputStream in, ResourceFieldSchema complexFieldSchema) throws IOException { +private Object consumeComplexType(PushbackInputStream in, ResourceFieldSchema complexFieldSchema) throws IOException { Object field; switch (complexFieldSchema.getType()) { case DataType.BAG: @@ -285,7 +291,8 @@ public class Utf8StorageConverter implem return null; DataBag db; try { -ByteArrayInputStream in = new ByteArrayInputStream(b); +ByteArrayInputStream bis = new ByteArrayInputStream(b); +PushbackInputStream in = new PushbackInputStream(bis); db = consumeBag(in, schema); } catch (IOException e) { LogUtils.warn(this, Unable to interpret value + Arrays.toString(b) + in field being + @@ -424,7 +431,8 @@ public class Utf8StorageConverter implem ResourceFieldSchema fs = new ResourceFieldSchema(); fs.setType(DataType.MAP); try { -
[Pig Wiki] Update of LoadStoreMigrationGuide by Prade epKamath
Dear Wiki user, You have subscribed to a wiki page or wiki category on Pig Wiki for change notification. The LoadStoreMigrationGuide page has been changed by PradeepKamath. http://wiki.apache.org/pig/LoadStoreMigrationGuide?action=diffrev1=35rev2=36 -- ||No equivalent method ||setUDFContextSignature() ||!LoadFunc ||This method will be called by Pig both in the front end and back end to pass a unique signature to the Loader. The signature can be used to store into the !UDFContext any information which the Loader needs to store between various method invocations in the front end and back end. A use case is to store !RequiredFieldList passed to it in !LoadPushDown.pushProjection(!RequiredFieldList) for use in the back end before returning tuples in getNext(). The default implementation in !LoadFunc has an empty body. This method will be called before other methods.|| ||No equivalent method ||relativeToAbsolutePath() ||!LoadFunc ||Pig runtime will call this method to allow the Loader to convert a relative load location to an absolute location. The default implementation provided in !LoadFunc handles this for !FileSystem locations. If the load source is something else, loader implementation may choose to override this. || ||determineSchema() ||getSchema() ||!LoadMetadata ||determineSchema() was used by old code to ask the loader to provide a schema for the data returned by it - the same semantics are now achieved through getSchema() of the !LoadMetadata interface. !LoadMetadata is an optional interface for loaders to implement - if a loader does not implement it, this will indicate to the pig runtime that the loader cannot return a schema for the data || - ||fieldsToRead() ||pushProject() ||!LoadPushDown ||fieldsToRead() was used by old code to convey to the loader the exact fields required by the pig script -the same semantics are now achieved through pushProject() of the !LoadPushDown interface. !LoadPushDown is an optional interface for loaders to implement - if a loader does not implement it, this will indicate to the pig runtime that the loader is not capable of returning just the required fields and will return all fields in the data. If a loader implementation is able to efficiently return only required fields, it should implement !LoadPushDown to improve query performance || + ||fieldsToRead() ||pushProjection() ||!LoadPushDown ||fieldsToRead() was used by old code to convey to the loader the exact fields required by the pig script -the same semantics are now achieved through pushProject() of the !LoadPushDown interface. !LoadPushDown is an optional interface for loaders to implement - if a loader does not implement it, this will indicate to the pig runtime that the loader is not capable of returning just the required fields and will return all fields in the data. If a loader implementation is able to efficiently return only required fields, it should implement !LoadPushDown to improve query performance || ||No equivalent method ||getInputFormat() ||!LoadFunc ||This method will be called by Pig to get the !InputFormat used by the loader. The methods in the !InputFormat (and underlying !RecordReader) will be called by pig in the same manner (and in the same context) as by Hadoop in a map-reduce java program. '''If the !InputFormat is a hadoop packaged one, the implementation should use the new API based one under org.apache.hadoop.mapreduce. If it is a custom !InputFormat, it should be implemented using the new API in org.apache.hadoop.mapreduce'''|| ||No equivalent method ||setLocation() ||!LoadFunc ||This method is called by Pig to communicate the load location to the loader. The loader should use this method to communicate the same information to the underlying !InputFormat. This method is called multiple times by pig - implementations should bear in mind that this method is called multiple times and should ensure there are no inconsistent side effects due to the multiple calls. || ||bindTo() ||prepareToRead() ||!LoadFunc ||bindTo() was the old method which would provide an !InputStream among other things to the !LoadFunc. The !LoadFunc implementation would then read from the !InputStream in getNext(). In the new API, reading of the data is through the !InputFormat provided by the !LoadFunc. So the equivalent call is prepareToRead() wherein the !RecordReader associated with the !InputFormat provided by the !LoadFunc is passed to the !LoadFunc. The !RecordReader can then be used by the implementation in getNext() to return a tuple representing a record of data back to pig. ||
[Pig Wiki] Update of Pig070IncompatibleChanges by Pra deepKamath
Dear Wiki user, You have subscribed to a wiki page or wiki category on Pig Wiki for change notification. The Pig070IncompatibleChanges page has been changed by PradeepKamath. http://wiki.apache.org/pig/Pig070IncompatibleChanges?action=diffrev1=31rev2=32 -- || Switching to Hadoop's local mode || Local Mode || Low || None || Main change is 10-20x performance slowdown. Also, local mode now uses the same UDF interfaces to execute UDFs as the MR mode. || || Removing support for Load-Stream or Stream-Store optimization || Streaming || Low to None || None || This feature was never documented so it is unlikely it was ever used || || We no longer support serialization and decerialization via load/store functions || Streaming || Unknown but hopefully low to medium || Implement new PigToStream and StreamToPig interfaces for non-standard serialization || LoadStoreRedesignProposal || + || Output part files now have a -m- and -r in the name || Output file names || Low to medium || If you have a system which depends on output file names tghe names now have changed from part-X to part-m-X if the output is being written from the map phase of the job or part-r- if it is being written from the reduce phase || || || Removing BinaryStorage builtin || Streaming || Low to None || None || As far as we know, this class was only used internally by streaming || || Removing Split by file feature || Split by File || Low to None || Input format of the loader would need to support this || We don't know that this feature was widely/ever used || || Local files no longer accessible from cluster || Access to Local Files from Map-Reduce Mode || low to none || copy the file to the cluster using copyToLocal command prior to the load || This feature was not documented ||
[Pig Wiki] Update of Pig070IncompatibleChanges by Pra deepKamath
Dear Wiki user, You have subscribed to a wiki page or wiki category on Pig Wiki for change notification. The Pig070IncompatibleChanges page has been changed by PradeepKamath. http://wiki.apache.org/pig/Pig070IncompatibleChanges?action=diffrev1=32rev2=33 -- || Switching to Hadoop's local mode || Local Mode || Low || None || Main change is 10-20x performance slowdown. Also, local mode now uses the same UDF interfaces to execute UDFs as the MR mode. || || Removing support for Load-Stream or Stream-Store optimization || Streaming || Low to None || None || This feature was never documented so it is unlikely it was ever used || || We no longer support serialization and decerialization via load/store functions || Streaming || Unknown but hopefully low to medium || Implement new PigToStream and StreamToPig interfaces for non-standard serialization || LoadStoreRedesignProposal || - || Output part files now have a -m- and -r in the name || Output file names || Low to medium || If you have a system which depends on output file names tghe names now have changed from part-X to part-m-X if the output is being written from the map phase of the job or part-r- if it is being written from the reduce phase || || + || Output part files now have a -m- and -r in the name || Output file names || Low to medium || If you have a system which depends on output file names the names now have changed from part-X to part-m-X if the output is being written from the map phase of the job or part-r- if it is being written from the reduce phase || || || Removing BinaryStorage builtin || Streaming || Low to None || None || As far as we know, this class was only used internally by streaming || || Removing Split by file feature || Split by File || Low to None || Input format of the loader would need to support this || We don't know that this feature was widely/ever used || || Local files no longer accessible from cluster || Access to Local Files from Map-Reduce Mode || low to none || copy the file to the cluster using copyToLocal command prior to the load || This feature was not documented ||
[Pig Wiki] Update of Pig070IncompatibleChanges by Pra deepKamath
Dear Wiki user, You have subscribed to a wiki page or wiki category on Pig Wiki for change notification. The Pig070IncompatibleChanges page has been changed by PradeepKamath. http://wiki.apache.org/pig/Pig070IncompatibleChanges?action=diffrev1=33rev2=34 -- || Switching to Hadoop's local mode || Local Mode || Low || None || Main change is 10-20x performance slowdown. Also, local mode now uses the same UDF interfaces to execute UDFs as the MR mode. || || Removing support for Load-Stream or Stream-Store optimization || Streaming || Low to None || None || This feature was never documented so it is unlikely it was ever used || || We no longer support serialization and decerialization via load/store functions || Streaming || Unknown but hopefully low to medium || Implement new PigToStream and StreamToPig interfaces for non-standard serialization || LoadStoreRedesignProposal || + || Removing BinaryStorage builtin || Streaming || Low to None || None || As far as we know, this class was only used internally by streaming || || Output part files now have a -m- and -r in the name || Output file names || Low to medium || If you have a system which depends on output file names the names now have changed from part-X to part-m-X if the output is being written from the map phase of the job or part-r- if it is being written from the reduce phase || || - || Removing BinaryStorage builtin || Streaming || Low to None || None || As far as we know, this class was only used internally by streaming || || Removing Split by file feature || Split by File || Low to None || Input format of the loader would need to support this || We don't know that this feature was widely/ever used || || Local files no longer accessible from cluster || Access to Local Files from Map-Reduce Mode || low to none || copy the file to the cluster using copyToLocal command prior to the load || This feature was not documented || || Removing Custom Comparators || Removing Custom Comparators || Low to None || None || This feature has been deprecated since Pig 0.5.0 release. We don't have a single known use case ||
svn commit: r922097 - in /hadoop/pig/trunk: CHANGES.txt src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java test/org/apache/pig/test/TestPruneColumn.java
Author: daijy Date: Fri Mar 12 01:08:30 2010 New Revision: 922097 URL: http://svn.apache.org/viewvc?rev=922097view=rev Log: PIG-1272: Column pruner causes wrong results Modified: hadoop/pig/trunk/CHANGES.txt hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=922097r1=922096r2=922097view=diff == --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Fri Mar 12 01:08:30 2010 @@ -147,6 +147,8 @@ OPTIMIZATIONS BUG FIXES +PIG-1272: Column pruner causes wrong results (daijy) + PIG-1275: empty bag in PigStorage read as null (daijy) PIG-1252: Diamond splitter does not generate correct results when using Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java?rev=922097r1=922096r2=922097view=diff == --- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java Fri Mar 12 01:08:30 2010 @@ -220,12 +220,25 @@ public class PruneColumns extends Logica { ListRequiredFields requiredInputFieldsList = new ArrayListRequiredFields(); RequiredFields requiredFields = new RequiredFields(false); -for (RequiredFields rf : requiredOutputInfo.requiredFieldsList) +for (int i=0;imPlan.getSuccessors(rlo).size();i++) { +RequiredFields rf = null; +try { +rf = requiredOutputInfo.requiredFieldsList.get(i); +} catch (Exception e) { +} if (rf!=null) { rf.reIndex(0); requiredFields.merge(rf); +} else { +// need all fields +ListPairInteger, Integer l = new ArrayListPairInteger, Integer(); +for (int j=0;jrlo.getSchema().size();j++) +l.add(new PairInteger, Integer(0, j)); +rf = new RequiredFields(l); +requiredFields.merge(rf); +break; } } requiredInputFieldsList.add(requiredFields); Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java?rev=922097r1=922096r2=922097view=diff == --- hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java Fri Mar 12 01:08:30 2010 @@ -1844,5 +1844,25 @@ public class TestPruneColumn extends Tes assertTrue(checkLogFileMessage(new String[]{No column pruned for A, No map keys pruned for A, [0,1,2]})); } + +// See PIG-1272 +@Test +public void testSplit4() throws Exception { +pigServer.registerQuery(A = load '+ Util.generateURI(tmpFile1.toString(), pigServer.getPigContext()) + ' AS (a0, a1, a2);); +pigServer.registerQuery(B = foreach A generate a0;); +pigServer.registerQuery(C = join A by a0, B by a0;); +IteratorTuple iter = pigServer.openIterator(C); + +assertTrue(iter.hasNext()); +Tuple t = iter.next(); +assertTrue(t.toString().equals((1,2,3,1))); + +assertTrue(iter.hasNext()); +t = iter.next(); +assertTrue(t.toString().equals((2,5,2,2))); + +assertTrue(checkLogFileMessage(new String[]{No column pruned for A, +No map keys pruned for A})); +} }
svn commit: r922169 - in /hadoop/pig/trunk: CHANGES.txt src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java test/org/apache/pig/test/TestPruneColumn.java
Author: daijy Date: Fri Mar 12 07:53:40 2010 New Revision: 922169 URL: http://svn.apache.org/viewvc?rev=922169view=rev Log: Temporarily rollback PIG-1272 due to one unit test fail Modified: hadoop/pig/trunk/CHANGES.txt hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=922169r1=922168r2=922169view=diff == --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Fri Mar 12 07:53:40 2010 @@ -147,8 +147,6 @@ OPTIMIZATIONS BUG FIXES -PIG-1272: Column pruner causes wrong results (daijy) - PIG-1275: empty bag in PigStorage read as null (daijy) PIG-1252: Diamond splitter does not generate correct results when using Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java?rev=922169r1=922168r2=922169view=diff == --- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/optimizer/PruneColumns.java Fri Mar 12 07:53:40 2010 @@ -220,25 +220,12 @@ public class PruneColumns extends Logica { ListRequiredFields requiredInputFieldsList = new ArrayListRequiredFields(); RequiredFields requiredFields = new RequiredFields(false); -for (int i=0;imPlan.getSuccessors(rlo).size();i++) +for (RequiredFields rf : requiredOutputInfo.requiredFieldsList) { -RequiredFields rf = null; -try { -rf = requiredOutputInfo.requiredFieldsList.get(i); -} catch (Exception e) { -} if (rf!=null) { rf.reIndex(0); requiredFields.merge(rf); -} else { -// need all fields -ListPairInteger, Integer l = new ArrayListPairInteger, Integer(); -for (int j=0;jrlo.getSchema().size();j++) -l.add(new PairInteger, Integer(0, j)); -rf = new RequiredFields(l); -requiredFields.merge(rf); -break; } } requiredInputFieldsList.add(requiredFields); Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java?rev=922169r1=922168r2=922169view=diff == --- hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java Fri Mar 12 07:53:40 2010 @@ -1844,25 +1844,5 @@ public class TestPruneColumn extends Tes assertTrue(checkLogFileMessage(new String[]{No column pruned for A, No map keys pruned for A, [0,1,2]})); } - -// See PIG-1272 -@Test -public void testSplit4() throws Exception { -pigServer.registerQuery(A = load '+ Util.generateURI(tmpFile1.toString(), pigServer.getPigContext()) + ' AS (a0, a1, a2);); -pigServer.registerQuery(B = foreach A generate a0;); -pigServer.registerQuery(C = join A by a0, B by a0;); -IteratorTuple iter = pigServer.openIterator(C); - -assertTrue(iter.hasNext()); -Tuple t = iter.next(); -assertTrue(t.toString().equals((1,2,3,1))); - -assertTrue(iter.hasNext()); -t = iter.next(); -assertTrue(t.toString().equals((2,5,2,2))); - -assertTrue(checkLogFileMessage(new String[]{No column pruned for A, -No map keys pruned for A})); -} }