You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2011/12/07 22:21:45 UTC

svn commit: r1211642 - in /pig/trunk: ./ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/ src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/ src/org/apache/pig/backend/hadoop/executionengi...

Author: daijy
Date: Wed Dec  7 21:21:44 2011
New Revision: 1211642

URL: http://svn.apache.org/viewvc?rev=1211642&view=rev
Log:
PIG-2370: SkewedParitioner results in Kerberos error

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartitionRearrange.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1211642&r1=1211641&r2=1211642&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Wed Dec  7 21:21:44 2011
@@ -324,6 +324,8 @@ Release 0.9.2 - Unreleased
 
 BUG FIXES
 
+PIG-2370: SkewedParitioner results in Kerberos error (daijy)
+
 PIG-2374: streaming regression with dotNext (daijy)
 
 PIG-2387: BinStorageRecordReader causes negative progress (xutingz via daijy)

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java?rev=1211642&r1=1211641&r2=1211642&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/partitioners/SkewedPartitioner.java Wed Dec  7 21:21:44 2011
@@ -112,7 +112,7 @@ public class SkewedPartitioner extends P
         try {
             Integer [] redCnt = new Integer[1]; 
             reducerMap = MapRedUtil.loadPartitionFileFromLocalCache(
-                    keyDistFile, redCnt, DataType.TUPLE);
+                    keyDistFile, redCnt, DataType.TUPLE, job);
             // check if the partition file is empty
             totalReducers = (redCnt[0] == null) ? -1 : redCnt[0];
         } catch (Exception e) {

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartitionRearrange.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartitionRearrange.java?rev=1211642&r1=1211641&r2=1211642&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartitionRearrange.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPartitionRearrange.java Wed Dec  7 21:21:44 2011
@@ -101,7 +101,7 @@ public class POPartitionRearrange extend
             Integer [] redCnt = new Integer[1];
 
             reducerMap = MapRedUtil.loadPartitionFileFromLocalCache(
-                    keyDistFile, redCnt, DataType.NULL);
+                    keyDistFile, redCnt, DataType.NULL, PigMapReduce.sJobConfInternal.get());
 
             // check if the partition file is empty
             totalReducers = (redCnt[0] == null) ? -1 : redCnt[0];

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java?rev=1211642&r1=1211641&r2=1211642&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/util/MapRedUtil.java Wed Dec  7 21:21:44 2011
@@ -78,7 +78,7 @@ public class MapRedUtil {
      */
     @SuppressWarnings("unchecked")
     public static <E> Map<E, Pair<Integer, Integer>> loadPartitionFileFromLocalCache(
-            String keyDistFile, Integer[] totalReducers, byte keyType)
+            String keyDistFile, Integer[] totalReducers, byte keyType, Configuration mapConf)
             throws IOException {
 
         Map<E, Pair<Integer, Integer>> reducerMap = new HashMap<E, Pair<Integer, Integer>>();
@@ -86,6 +86,10 @@ public class MapRedUtil {
         // use local file system to get the keyDistFile
         Configuration conf = new Configuration(false);            
         
+        if (mapConf.get("yarn.resourcemanager.principal")!=null) {
+            conf.set("yarn.resourcemanager.principal", mapConf.get("yarn.resourcemanager.principal"));
+        }
+        
         if (PigMapReduce.sJobConfInternal.get().get("fs.file.impl")!=null)
             conf.set("fs.file.impl", PigMapReduce.sJobConfInternal.get().get("fs.file.impl"));
         if (PigMapReduce.sJobConfInternal.get().get("fs.hdfs.impl")!=null)