You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ch...@apache.org on 2014/04/12 21:06:26 UTC

svn commit: r1586885 [2/2] - in /pig/branches/tez: ./ ivy/ src/ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/fetch/ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pig/backend/hadoop/executionen...

Modified: pig/branches/tez/src/org/apache/pig/scripting/groovy/GroovyScriptEngine.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/scripting/groovy/GroovyScriptEngine.java?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/scripting/groovy/GroovyScriptEngine.java (original)
+++ pig/branches/tez/src/org/apache/pig/scripting/groovy/GroovyScriptEngine.java Sat Apr 12 19:06:25 2014
@@ -136,7 +136,7 @@ public class GroovyScriptEngine extends 
   public void registerFunctions(String path, String namespace, PigContext pigContext) throws IOException {
 
     if (!isInitialized) {
-      pigContext.scriptJars.add(getJarPath(groovy.util.GroovyScriptEngine.class));
+      pigContext.addScriptJar(getJarPath(groovy.util.GroovyScriptEngine.class));
       isInitialized = true;
     }
 

Modified: pig/branches/tez/src/org/apache/pig/scripting/jruby/JrubyScriptEngine.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/scripting/jruby/JrubyScriptEngine.java?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/scripting/jruby/JrubyScriptEngine.java (original)
+++ pig/branches/tez/src/org/apache/pig/scripting/jruby/JrubyScriptEngine.java Sat Apr 12 19:06:25 2014
@@ -129,7 +129,7 @@ public class JrubyScriptEngine extends S
     @Override
     public void registerFunctions(String path, String namespace, PigContext pigContext) throws IOException {
         if (!isInitialized) {
-            pigContext.scriptJars.add(getJarPath(Ruby.class));
+            pigContext.addScriptJar(getJarPath(Ruby.class));
             pigContext.addScriptFile("pigudf.rb", "pigudf.rb");
             isInitialized = true;
         }
@@ -137,8 +137,6 @@ public class JrubyScriptEngine extends S
         for (Map.Entry<String,Object> entry : RubyFunctions.getFunctions("evalfunc", path).entrySet()) {
             String method = entry.getKey();
 
-            String functionType = rubyEngine.callMethod(entry.getValue(), "name", String.class);
-
             FuncSpec funcspec = new FuncSpec(JrubyEvalFunc.class.getCanonicalName() + "('" + path + "','" + method +"')");
             pigContext.registerFunction(namespace + "." + method, funcspec);
         }
@@ -197,14 +195,14 @@ public class JrubyScriptEngine extends S
                     if (file.isDirectory()) {
                         continue;
                     } else if (file.getName().endsWith(".jar") || file.getName().endsWith(".zip")) {
-                        pigContext.scriptJars.add(file.getPath());
+                        pigContext.addScriptJar(file.getPath());
                     } else {
                         String localPath = libFile.getName() + file.getPath().replaceFirst(libFile.getPath(), "");
                         pigContext.addScriptFile(localPath, file.getPath());
                     }
                 }
             } else {
-                pigContext.scriptJars.add(lib);
+                pigContext.addScriptJar(lib);
             }
         }
     }

Modified: pig/branches/tez/src/org/apache/pig/scripting/js/JsScriptEngine.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/scripting/js/JsScriptEngine.java?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/scripting/js/JsScriptEngine.java (original)
+++ pig/branches/tez/src/org/apache/pig/scripting/js/JsScriptEngine.java Sat Apr 12 19:06:25 2014
@@ -230,9 +230,9 @@ public class JsScriptEngine extends Scri
             PigContext pigContext) throws IOException {
         // to enable passing of information to the slave
         this.scriptPath = path;
-        this.clientInstance = this;
-        
-        pigContext.scriptJars.add(getJarPath(Context.class));
+        JsScriptEngine.clientInstance = this;
+
+        pigContext.addScriptJar(getJarPath(Context.class));
         namespace = (namespace == null) ? "" : namespace + NAMESPACE_SEPARATOR;
         FileInputStream fis = new FileInputStream(path);
         try {

Modified: pig/branches/tez/src/org/apache/pig/scripting/jython/JythonScriptEngine.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/scripting/jython/JythonScriptEngine.java?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/scripting/jython/JythonScriptEngine.java (original)
+++ pig/branches/tez/src/org/apache/pig/scripting/jython/JythonScriptEngine.java Sat Apr 12 19:06:25 2014
@@ -228,7 +228,7 @@ public class JythonScriptEngine extends 
                         if (modulepath.equals(JVM_JAR)) {
                             continue;
                         } else if (modulepath.endsWith(".jar") || modulepath.endsWith(".zip")) {
-                            pigContext.scriptJars.add(modulepath);
+                            pigContext.addScriptJar(modulepath);
                         } else {
                             pigContext.addScriptFile(modulename, modulepath);
                         }
@@ -342,7 +342,7 @@ public class JythonScriptEngine extends 
     throws IOException {
         Interpreter.setMain(false);
         Interpreter.init(path, pigContext);
-        pigContext.scriptJars.add(getJarPath(PythonInterpreter.class));
+        pigContext.addScriptJar(getJarPath(PythonInterpreter.class));
         PythonInterpreter pi = Interpreter.interpreter;
         @SuppressWarnings("unchecked")
         List<PyTuple> locals = ((PyStringMap) pi.getLocals()).items();

Modified: pig/branches/tez/src/org/apache/pig/tools/grunt/GruntParser.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/tools/grunt/GruntParser.java?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/tools/grunt/GruntParser.java (original)
+++ pig/branches/tez/src/org/apache/pig/tools/grunt/GruntParser.java Sat Apr 12 19:06:25 2014
@@ -177,7 +177,7 @@ public class GruntParser extends PigScri
                         mNumFailedJobs++;
                         Exception exp = (js.getException() != null) ? js.getException()
                                 : new ExecException(
-                                        "Job failed, hadoop does not return any error message",
+                                        "Job " + js.getName() + " failed, hadoop does not return any error message",
                                         2244);
                         LogUtils.writeLog(exp,
                                 mPigServer.getPigContext().getProperties().getProperty("pig.logfile"),
@@ -360,12 +360,6 @@ public class GruntParser extends PigScri
             mExplain = new ExplainState(alias, target, script, isVerbose, format);
 
             if (script != null) {
-                if (!"true".equalsIgnoreCase(mPigServer.
-                                             getPigContext()
-                                             .getProperties().
-                                             getProperty("opt.multiquery","true"))) {
-                    throw new ParseException("Cannot explain script if multiquery is disabled.");
-                }
                 setBatchOn();
                 try {
                     loadScript(script, true, true, false, params, files);
@@ -1335,7 +1329,7 @@ public class GruntParser extends PigScri
     private int mNumSucceededJobs;
     private FsShell shell;
     private boolean mScriptIllustrate;
-    
+
     //For Testing Only
     protected void setExplainState(ExplainState explainState) {
         this.mExplain = explainState;

Modified: pig/branches/tez/src/org/apache/pig/tools/pigstats/PigStatusReporter.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/tools/pigstats/PigStatusReporter.java?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/tools/pigstats/PigStatusReporter.java (original)
+++ pig/branches/tez/src/org/apache/pig/tools/pigstats/PigStatusReporter.java Sat Apr 12 19:06:25 2014
@@ -22,6 +22,7 @@ import org.apache.hadoop.mapreduce.Count
 import org.apache.hadoop.mapreduce.StatusReporter;
 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
 import org.apache.hadoop.util.Progressable;
+import org.apache.pig.backend.hadoop.executionengine.fetch.FetchContext;
 import org.apache.pig.classification.InterfaceAudience;
 import org.apache.pig.classification.InterfaceStability;
 
@@ -31,7 +32,10 @@ import org.apache.pig.classification.Int
 public class PigStatusReporter extends StatusReporter implements Progressable {
 
     private TaskInputOutputContext context;
+    private FetchContext fetchContext;
+
     private static PigStatusReporter reporter = null;
+
     /**
      * Get singleton instance of the context
      */
@@ -41,35 +45,41 @@ public class PigStatusReporter extends S
         }
         return reporter;
     }
-    
+
     public static void setContext(TaskInputOutputContext context) {
         reporter = new PigStatusReporter(context);
     }
-    
+
     private PigStatusReporter(TaskInputOutputContext context) {
         this.context = context;
     }
-    
+
     @Override
-    public Counter getCounter(Enum<?> name) {        
-        return (context == null) ? null : context.getCounter(name);
+    public Counter getCounter(Enum<?> name) {
+        if (fetchContext != null) {
+            return fetchContext.getCounter(name);  
+        }
+        return (context == null) ? null : context.getCounter(name); 
     }
 
     @Override
     public Counter getCounter(String group, String name) {
+        if (fetchContext != null) {
+            return fetchContext.getCounter(group, name);
+        }
         return (context == null) ? null : context.getCounter(group, name);
     }
 
     @Override
     public void progress() {
-        if (context != null) {
+        if (fetchContext == null && context != null) {
             context.progress();
         }
     }
 
     @Override
     public void setStatus(String status) {
-        if (context != null) {
+        if (fetchContext == null && context != null) {
             context.setStatus(status);
         }
     }
@@ -77,4 +87,13 @@ public class PigStatusReporter extends S
     public float getProgress() {
         return 0;
     }
+
+    /**
+     * Sets a dummy counter handler for fetch tasks
+     * @param fetchContext
+     */
+    public void setFetchContext(FetchContext fetchContext) {
+        this.fetchContext = fetchContext;
+    }
+
 }

Modified: pig/branches/tez/src/org/apache/pig/tools/pigstats/ScriptState.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/tools/pigstats/ScriptState.java?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/tools/pigstats/ScriptState.java (original)
+++ pig/branches/tez/src/org/apache/pig/tools/pigstats/ScriptState.java Sat Apr 12 19:06:25 2014
@@ -130,7 +130,12 @@ public abstract class ScriptState {
 
     private static final Log LOG = LogFactory.getLog(ScriptState.class);
 
-    private static ThreadLocal<ScriptState> tss = new ThreadLocal<ScriptState>();
+    /**
+     * PIG-3844. Each thread should have its own copy of ScriptState. We initialize the ScriptState
+     * for new threads with the ScriptState of its parent thread, using InheritableThreadLocal.
+     * Used eg. in PPNL running in separate thread.
+     */
+    private static InheritableThreadLocal<ScriptState> tss = new InheritableThreadLocal<ScriptState>();
 
     protected String id;
 

Propchange: pig/branches/tez/src/pig-default.properties
------------------------------------------------------------------------------
  Merged /pig/trunk/src/pig-default.properties:r1582882-1586680

Modified: pig/branches/tez/test/e2e/pig/tests/macro.conf
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/e2e/pig/tests/macro.conf?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/test/e2e/pig/tests/macro.conf (original)
+++ pig/branches/tez/test/e2e/pig/tests/macro.conf Sat Apr 12 19:06:25 2014
@@ -365,7 +365,7 @@ $cfg = {
         {
           #define for streaming combines with define for macros
           'num' => 3,
-          ,'pig' => q#define CMD `perl -ne "print $_;"`;
+          ,'pig' => q#define CMD `perl -ne 'print $_;'`;
                       define test(in) returns B {
                           $B = stream $in through CMD as (name, age, gpa);
                       }
@@ -374,7 +374,7 @@ $cfg = {
                       x = test(A);
                       store x into ':OUTPATH:';#,
           'verify_pig_script' => q#A = load ':INPATH:/singlefile/studenttab10k' as (name, age, gpa);
-                                   define CMD `perl -ne "print $_;"`;
+                                   define CMD `perl -ne 'print $_;'`;
                                    B = stream A through CMD as (name, age, gpa);
                                    store B into ':OUTPATH:';#,
           'floatpostprocess' => 1,

Modified: pig/branches/tez/test/e2e/pig/tests/multiquery.conf
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/e2e/pig/tests/multiquery.conf?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/test/e2e/pig/tests/multiquery.conf (original)
+++ pig/branches/tez/test/e2e/pig/tests/multiquery.conf Sat Apr 12 19:06:25 2014
@@ -228,8 +228,8 @@ $cfg = {
             {
             # Streaming with multiple stores
             'num' => 11,
-            'pig' => q# define CMD1 `perl -ne \"print $_;\"`;
-                        define CMD2 `perl -ne \"print $_;\"`;
+            'pig' => q# define CMD1 `perl -ne 'print $_;'`;
+                        define CMD2 `perl -ne 'print $_;'`;
                         A = load ':INPATH:/singlefile/studenttab10k' as (name, age, gpa);
                         B = stream A through CMD1 as (name, age, gpa);
                         store B into ':OUTPATH:.1';

Modified: pig/branches/tez/test/e2e/pig/tests/turing_jython.conf
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/e2e/pig/tests/turing_jython.conf?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/test/e2e/pig/tests/turing_jython.conf (original)
+++ pig/branches/tez/test/e2e/pig/tests/turing_jython.conf Sat Apr 12 19:06:25 2014
@@ -692,7 +692,7 @@ result = P.bind().runSingle()
 if not result.isSuccessful():
     raise "Pig job FAILED"    
 
-pigfile = open(":TMP:/iterator_output.txt", 'w+')
+pigfile = open(":TMP:/iterator_output.txt", 'wb+')
 iter = result.result("A").iterator()
 while iter.hasNext():
     t = iter.next()

Modified: pig/branches/tez/test/org/apache/pig/test/TestExampleGenerator.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestExampleGenerator.java?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestExampleGenerator.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestExampleGenerator.java Sat Apr 12 19:06:25 2014
@@ -45,7 +45,7 @@ public class TestExampleGenerator {
     static int MAX = 100;
     static String A, B;
     static  File fileA, fileB;
-    
+
     @BeforeClass
     public static void oneTimeSetup() throws Exception {
         pigContext.connect();
@@ -55,7 +55,7 @@ public class TestExampleGenerator {
 
         writeData(fileA);
         writeData(fileB);
-     
+
 
         fileA.deleteOnExit();
         fileB.deleteOnExit();
@@ -122,7 +122,7 @@ public class TestExampleGenerator {
 
         assertNotNull(derivedData);
     }
-    
+
     @Test
     public void testFilter3() throws Exception {
 
@@ -144,7 +144,7 @@ public class TestExampleGenerator {
         assertNotNull(derivedData);
 
     }
-    
+
     @Test
     public void testForeach() throws ExecException, IOException {
         PigServer pigServer = new PigServer(pigContext);
@@ -157,7 +157,7 @@ public class TestExampleGenerator {
 
         assertNotNull(derivedData);
     }
-    
+
     //see PIG-2170
     @Test
     public void testForeachBinCondWithBooleanExp() throws ExecException, IOException {
@@ -171,7 +171,7 @@ public class TestExampleGenerator {
 
         assertNotNull(derivedData);
     }
-    
+
     @Test
     public void testForeachWithTypeCastCounter() throws ExecException, IOException {
         PigServer pigServer = new PigServer(pigContext);
@@ -245,7 +245,7 @@ public class TestExampleGenerator {
         assertNotNull(derivedData);
 
     }
-    
+
     @Test
     public void testGroup2() throws Exception {
         PigServer pigServer = new PigServer(pigContext);
@@ -270,7 +270,7 @@ public class TestExampleGenerator {
         assertNotNull(derivedData);
 
     }
-    
+
     @Test
     public void testFilterUnion() throws Exception {
         PigServer pigServer = new PigServer(pigContext);
@@ -283,7 +283,7 @@ public class TestExampleGenerator {
         assertNotNull(derivedData);
 
     }
-    
+
     @Test
     public void testForEachNestedBlock() throws Exception {
         PigServer pigServer = new PigServer(pigContext);
@@ -307,7 +307,7 @@ public class TestExampleGenerator {
         assertNotNull(derivedData);
 
     }
-    
+
     @Test
     public void testUnion() throws Exception {
         PigServer pigServer = new PigServer(pigContext);
@@ -328,7 +328,7 @@ public class TestExampleGenerator {
 
         assertNotNull(derivedData);
     }
-    
+
     @Test
     public void testCross() throws Exception {
         PigServer pigServer = new PigServer(pigContext);
@@ -339,7 +339,7 @@ public class TestExampleGenerator {
 
         assertNotNull(derivedData);
     }
-    
+
     @Test
     public void testLimit() throws Exception {
         PigServer pigServer = new PigServer(pigContext);
@@ -349,7 +349,7 @@ public class TestExampleGenerator {
 
         assertNotNull(derivedData);
     }
-    
+
     //see PIG-2275
     @Test
     public void testFilterWithIsNull() throws ExecException, IOException {
@@ -363,7 +363,7 @@ public class TestExampleGenerator {
 
         assertNotNull(derivedData);
     }
-    
+
     @Test
     public void testFilterWithUDF() throws ExecException, IOException {
         PigServer pigServer = new PigServer(pigContext);
@@ -383,7 +383,7 @@ public class TestExampleGenerator {
         File out = File.createTempFile("testFilterGroupCountStoreOutput", "");
         out.deleteOnExit();
         out.delete();
-    
+
         PigServer pigServer = new PigServer(pigContext);
         pigServer.setBatchOn();
         pigServer.registerQuery("A = load " + A.toString() + " as (x, y);");
@@ -392,16 +392,26 @@ public class TestExampleGenerator {
         pigServer.registerQuery("D = foreach C generate group as x, COUNT(B) as the_count;");
         pigServer.registerQuery("store D into '" +  Util.encodeEscape(out.getAbsolutePath()) + "';");
         Map<Operator, DataBag> derivedData = pigServer.getExamples(null);
-    
+
         assertNotNull(derivedData);
     }
-    
+
     @Test
     public void testLoaderWithContext() throws Exception {
         PigServer pigServer = new PigServer(pigContext);
         pigServer.registerQuery("A = load " + A.toString() + " using " + UDFContextTestLoaderWithSignature.class.getName() + "('a') as (x, y);");
         Map<Operator, DataBag> derivedData = pigServer.getExamples("A");
-        
+
+        assertNotNull(derivedData);
+    }
+
+    @Test
+    public void testOrderBy() throws Exception {
+        PigServer pigServer = new PigServer(pigContext);
+        pigServer.registerQuery("A = load " + A.toString() + " as (x, y);");
+        pigServer.registerQuery("B = order A by x;");
+        Map<Operator, DataBag> derivedData = pigServer.getExamples("B");
+
         assertNotNull(derivedData);
     }
 

Modified: pig/branches/tez/test/org/apache/pig/test/TestJobSubmission.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestJobSubmission.java?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestJobSubmission.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestJobSubmission.java Sat Apr 12 19:06:25 2014
@@ -194,9 +194,9 @@ public class TestJobSubmission {
         // default_parallel is considered only at runtime, so here we only test requested parallel
         // more thorough tests can be found in TestNumberOfReducers.java
         String query = "a = load 'input';" +
-                       "b = load 'input';" +
-                       "c = join a by $0, b by $0 using 'skewed' parallel 100;" +
-                       "store c into 'output';";
+                "b = load 'input';" +
+                "c = join a by $0, b by $0 using 'skewed' parallel 100;" +
+                "store c into 'output';";
         PigServer ps = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         PhysicalPlan pp = Util.buildPp(ps, query);
         MROperPlan mrPlan = Util.buildMRPlan(pp, pc);
@@ -228,8 +228,8 @@ public class TestJobSubmission {
         util.startMiniHBaseCluster(1, 1);
 
         String query = "a = load '/passwd';" +
-                       "b = group a by $0;" +
-                       "store b into 'output';";
+                "b = group a by $0;" +
+                "store b into 'output';";
         PigServer ps = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         PhysicalPlan pp = Util.buildPp(ps, query);
         MROperPlan mrPlan = Util.buildMRPlan(pp, pc);
@@ -297,13 +297,13 @@ public class TestJobSubmission {
 
     @Test
     public void testReducerNumEstimationForOrderBy() throws Exception{
-       // use the estimation
+        // use the estimation
         pc.getProperties().setProperty("pig.exec.reducers.bytes.per.reducer", "100");
         pc.getProperties().setProperty("pig.exec.reducers.max", "10");
 
         String query = "a = load '/passwd';" +
-                       "b = order a by $0;" +
-                       "store b into 'output';";
+                "b = order a by $0;" +
+                "store b into 'output';";
         PigServer ps = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         PhysicalPlan pp = Util.buildPp(ps, query);
 

Modified: pig/branches/tez/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld (original)
+++ pig/branches/tez/test/org/apache/pig/test/data/GoldenFiles/MRC17.gld Sat Apr 12 19:06:25 2014
@@ -14,4 +14,4 @@ MapReduce(1) - -0:
 |   |
 |   |---Limit - --43001471365805096
 |       |
-|       |---Load(DummyFil:DummyLdr) - -5534705358975373945
\ No newline at end of file
+|       |---Load(DummyFil:DummyLdr) - -5534705358975373945

Modified: pig/branches/tez/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld?rev=1586885&r1=1586884&r2=1586885&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld (original)
+++ pig/branches/tez/test/org/apache/pig/test/data/GoldenFiles/MRC18.gld Sat Apr 12 19:06:25 2014
@@ -18,4 +18,4 @@ Reduce Plan Empty
     |   |   |
     |   |   Project[tuple][*] - scope-111
     |   |
-    |   |---b: Load(/tmp/input2:org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MergeJoinIndexer('org.apache.pig.builtin.PigStorage','eNqtVb1v00AUf3WbNoS2tHwtiAHxNWFLLCAxQFuoCJgmaruQiVf76ricfcf5XBwGpC4wsMLAgMTAyMg/gBiYYWRC7EhsjPDukjRuCmIIN0S59+7ex+9+v+e336GSKTiyhdvo5jrm7oJS2PHjTBc7n0++/IivxmGsDhNZ/IgVEgDGHk6YX7q0IlTkosSgzVwZR+4GBvdZGrptDIWQLitYQBFFytIoTulIu5PFAXIfO0y5kmOauc2erUk76K4xB5wWzFCgxuYC5/VU5tqHipAJSg2nfUrqdZN6lNSLE8k9U7h3J+c6voPySqHg7FBp5pRN6TYkU6iFMhl/4IWlS0/eLThQ9aGWLCuR3Agjlj2AxzBOOZPbrJNpOOQbdHpJKL4PU4nPcJuRb67kM6CRcyJpSBui4sNksiqEtrtJH2aTNbGph9NMW+u6KNuqSXlfSEL71J9asi/Wb7wHoOPAmKmCLLYKe3t+8MA3MWuTrzL15f2H4/c+jYOzDDUuMFzGgICpwwHdVixrCx4W8uo1G3P6YZV+58zfwtZU2+cwjDjzT9gJ0dI73wInDgnoLBCSaTjcBZPOR96aVnEaXSn6XbV17xiluTsK8YjF1F1G7n5NRMNGU4ktFuhBbZMtmI2znpkeIw1bUBPbTBmkGG2OSSUCEyiNFjFqbK7nkrOsBUcpOj3IGtk5s0Zy16GaaVR6SXBiTyB4nqTEnmMl9uzqzvB34393eGOfbdAqUEJ/lIR9Fe8PPd2CCgZBnpCiY6PkBa1NjrBucHqQk4RY2ESFnDMeZ8ki1LrwrXckI2Igj9HybZY2NoCGE8MjIESNnkWa1DdpT/UVN8OpWozYusKAKQ
 3nh+9KlvbFWzpIceaFiqlValEEaDrvh5wSuS5lqElULNVmnGhoDofvweh1YfSGYPT2wOjZkeiVRyLVMa7MmLk+UuBVC6nh1al/CnTwdnaIkF41nPvj0DXXvJKuSasGk0MAv2hdlLQsSHbZj0atkGYWLY7CtW4vpRmyCNOK6Vylaxp1nhEDugzaO04aG0bHVKLTHyiwW1z3i1bbnZLmgltPNYuYOvzt9ZufO08vO+YjWNlGnjOCcW5wbiVPNph68vbFyYPPvz4zYrIJimI4vNme+OvwtF4LX7EPNGuo/uUmjOgGWfwG+uh74A==','','b_45-1','scope','true')) - scope-102
\ No newline at end of file
+    |   |---b: Load(/tmp/input2:org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MergeJoinIndexer('org.apache.pig.builtin.PigStorage','eNqtVTtvE0EQnlzixJiQhGeDKBCv7k6iQqKABIgwHNgiaXDF5G5zPti7XXb3wpkCiQYKKKFAAomCkt+AhCiooaRC9NSUMLu248MBUZgtLO/M7jy+/b65t9+hphXsv41b6Bcm5f6yUtgLU23Kh5+PvPiIr6ZhqgkzOr3PSgkAU/dm7C9dui5U4qPEqMt8mSb+BkZ3WB77XYyFkD4rWUQRRc7yJM3pSLen0wh5iD2mfMkx1357YGvTDvprygOvA3soUGtzmfNmLgsTQk3IDKWBYyElDfpJA0oapJnkgS08uFZwk15DebZUcGKsNHvKpfRbkik0QtmMT9/pl1/DD10P6iE0slUlsktxwvRdeADTlDO7ynrawGJo0RkkofghzGUhwy1GvqWKz4JGzpmsJV2IWgiz2Q0hjNvNhrCQrYlNM55m3lnXRdVWz6r7UhLaR//UknuxYeMDAD0PpmwVZHFVuNt7Rw98GXWXfLW5L+8/HLr1aRq8VWhwgfEqRgRME3aZrmK6K3hcynPnXcz5e3X6XbJ/S1dTY4fDMuL4P2EnRCvvfAW8NCagdSQkM7CvDyadT4I1o9I8OVsOu+qawTFKc3MS4hGLqTtN7mFNRMNWW4nbLDKj2mY7sJDqgZkeI4870BBbTFmkGG0OSiUiGyhPVjBpba4XkjPdgQMUnR5kjeycOSO5m1DXBpW5IDixJxK8yHJiz8EKe7Z1Z/m78b87vLTDNmoVKGE4ScKhineGnu9ADaOoyEjRqVXysjE2R9y0ON0tSEIsbqNCzhlPdbYCjT586z3JiBjIU3R8W6CNC2Dg8PgIiNFg4JAm9c26U0PF7eFULSZsXWHElIFT43
 cly4firRykOHuFSqlValFEaDsfhpwThalkaEhULDd2nBhoj4cfwBj0YQzGYAx+gzFwIzGojkSqY1rZMXNxosA3HKSWV0f/KdDR27khQno1cPKPQ9deCyq6Jq1aTBYBftI6LWk5kNxyH41GKe0sWpmEa/1eKjNkBeYVM4XK1wyaQhMD+gz6fZy0NqyOqURvOFBgu7j+F62xPSXtBb+ZG5Ywte/b6zc/Hj4+49mPYG0LecEIxqXRuetFtsHUo7fPj+x+9vWJFZNLUJbj4e328F+Hp/M6+ModoDlD/S83YUI3yPIXqjl9HQ==','','b_45-1','scope','true')) - scope-102

Propchange: pig/branches/tez/test/org/apache/pig/test/data/bzipdir1.bz2/bzipdir2.bz2/recordLossblockHeaderEndsAt136500.txt.bz2
------------------------------------------------------------------------------
  Merged /pig/trunk/test/org/apache/pig/test/data/bzipdir1.bz2/bzipdir2.bz2/recordLossblockHeaderEndsAt136500.txt.bz2:r1582882-1586680