You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by pr...@apache.org on 2010/02/03 01:26:14 UTC

svn commit: r905850 - in /hadoop/pig/branches/load-store-redesign/src/org/apache/pig: LoadFunc.java backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java

Author: pradeepkth
Date: Wed Feb  3 00:26:13 2010
New Revision: 905850

URL: http://svn.apache.org/viewvc?rev=905850&view=rev
Log:
changes to make IndexableLoadFunc work

Modified:
    hadoop/pig/branches/load-store-redesign/src/org/apache/pig/LoadFunc.java
    hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java

Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/LoadFunc.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/LoadFunc.java?rev=905850&r1=905849&r2=905850&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/LoadFunc.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/LoadFunc.java Wed Feb  3 00:26:13 2010
@@ -117,7 +117,9 @@
     public abstract void prepareToRead(RecordReader reader, PigSplit split) throws IOException;
 
     /**
-     * Retrieves the next tuple to be processed.
+     * Retrieves the next tuple to be processed. Implementations should NOT reuse
+     * tuple objects they return across calls and should return a different tuple 
+     * object in each call.
      * @return the next tuple to be processed or null if there are no more tuples
      * to be processed.
      * @throws IOException if there is an exception while retrieving the next

Modified: hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java?rev=905850&r1=905849&r2=905850&view=diff
==============================================================================
--- hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java (original)
+++ hadoop/pig/branches/load-store-redesign/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java Wed Feb  3 00:26:13 2010
@@ -26,6 +26,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.IndexableLoadFunc;
@@ -397,9 +398,11 @@
         pc = (PigContext)ObjectSerializer.deserialize(PigMapReduce.sJobConf.get("pig.pigContext"));
         pc.connect();
         // Pass signature of the loader to rightLoader
-        PigMapReduce.sJobConf.set("pig.loader.signature", signature);
-        rightLoader.initialize(PigMapReduce.sJobConf);
-        rightLoader.setLocation(rightInputFileName, new Job(PigMapReduce.sJobConf));
+        // make a copy of the conf to use in calls to rightLoader.
+        Configuration conf = new Configuration(PigMapReduce.sJobConf);
+        conf.set("pig.loader.signature", signature);
+        rightLoader.setLocation(rightInputFileName, new Job(conf));
+        rightLoader.initialize(conf);
         rightLoader.seekNear(
                 firstLeftKey instanceof Tuple ? (Tuple)firstLeftKey : mTupleFactory.newTuple(firstLeftKey));
     }