You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by an...@apache.org on 2014/03/07 08:02:48 UTC

svn commit: r1575198 - in /pig/trunk: CHANGES.txt src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/InputSizeReducerEstimator.java test/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/TestInputSizeReducerEstimator.java

Author: aniket486
Date: Fri Mar  7 07:02:48 2014
New Revision: 1575198

URL: http://svn.apache.org/r1575198
Log:
PIG-3754: InputSizeReducerEstimator.getTotalInputFileSize reports incorrect size (aniket486)

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/InputSizeReducerEstimator.java
    pig/trunk/test/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/TestInputSizeReducerEstimator.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1575198&r1=1575197&r2=1575198&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Fri Mar  7 07:02:48 2014
@@ -95,6 +95,8 @@ OPTIMIZATIONS
  
 BUG FIXES
 
+PIG-3754: InputSizeReducerEstimator.getTotalInputFileSize reports incorrect size (aniket486)
+
 PIG-3679: e2e StreamingPythonUDFs_10 fails in trunk (cheolsoo)
 
 PIG-3776: Conflicting versions of jline is present in trunk (cheolsoo)

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/InputSizeReducerEstimator.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/InputSizeReducerEstimator.java?rev=1575198&r1=1575197&r2=1575198&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/InputSizeReducerEstimator.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/InputSizeReducerEstimator.java Fri Mar  7 07:02:48 2014
@@ -97,33 +97,35 @@ public class InputSizeReducerEstimator i
      * their size nor can pig look that up itself are excluded from this size.
      */
     static long getTotalInputFileSize(Configuration conf,
-                                      List<POLoad> lds, Job job) throws IOException {
+            List<POLoad> lds, Job job) throws IOException {
         long totalInputFileSize = 0;
-        boolean foundSize = false;
         for (POLoad ld : lds) {
             long size = getInputSizeFromLoader(ld, job);
-            if (size > -1) { foundSize = true; }
-            if (size > 0) {
+            if (size > -1) {
                 totalInputFileSize += size;
                 continue;
-            }
-            // the input file location might be a list of comma separated files,
-            // separate them out
-            for (String location : LoadFunc.getPathStrings(ld.getLFile().getFileName())) {
-                if (UriUtil.isHDFSFileOrLocalOrS3N(location)) {
-                    Path path = new Path(location);
-                    FileSystem fs = path.getFileSystem(conf);
-                    FileStatus[] status = fs.globStatus(path);
-                    if (status != null) {
-                        for (FileStatus s : status) {
-                            totalInputFileSize += MapRedUtil.getPathLength(fs, s);
-                            foundSize = true;
+            } else {
+
+                // the input file location might be a list of comma separated files,
+                // separate them out
+                for (String location : LoadFunc.getPathStrings(ld.getLFile().getFileName())) {
+                    if (UriUtil.isHDFSFileOrLocalOrS3N(location)) {
+                        Path path = new Path(location);
+                        FileSystem fs = path.getFileSystem(conf);
+                        FileStatus[] status = fs.globStatus(path);
+                        if (status != null) {
+                            for (FileStatus s : status) {
+                                totalInputFileSize += MapRedUtil.getPathLength(fs, s);
+                            }
                         }
+                    } else {
+                        // If we cannot estimate size of a location, we should report -1
+                        return -1;
                     }
                 }
             }
         }
-        return foundSize ? totalInputFileSize : -1;
+        return totalInputFileSize;
     }
 
     /**

Modified: pig/trunk/test/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/TestInputSizeReducerEstimator.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/TestInputSizeReducerEstimator.java?rev=1575198&r1=1575197&r2=1575198&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/TestInputSizeReducerEstimator.java (original)
+++ pig/trunk/test/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/TestInputSizeReducerEstimator.java Fri Mar  7 07:02:48 2014
@@ -17,16 +17,20 @@
  */
 package org.apache.pig.backend.hadoop.executionengine.mapReduceLayer;
 
-import com.google.common.collect.Lists;
+import java.util.Collections;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.pig.LoadFunc;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
 import org.apache.pig.builtin.PigStorage;
+import org.apache.pig.impl.io.FileSpec;
 import org.apache.pig.test.PigStorageWithStatistics;
 import org.apache.pig.test.TestJobControlCompiler;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+
 public class TestInputSizeReducerEstimator {
 
     private static final Configuration CONF = new Configuration(false);
@@ -48,6 +52,15 @@ public class TestInputSizeReducerEstimat
                 Lists.newArrayList(
                         createPOLoadWithSize(size, new PigStorage()),
                         createPOLoadWithSize(size, new PigStorageWithStatistics())),
+                        new org.apache.hadoop.mapreduce.Job(CONF)));
+
+        // Negative test - PIG-3754
+        POLoad poLoad = createPOLoadWithSize(size, new PigStorage());
+        poLoad.setLFile(new FileSpec("hbase://users", null));
+
+        Assert.assertEquals(-1, InputSizeReducerEstimator.getTotalInputFileSize(
+                CONF,
+                Collections.singletonList(poLoad),
                 new org.apache.hadoop.mapreduce.Job(CONF)));
     }