pig-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From da...@apache.org
Subject svn commit: r1721448 - in /pig/trunk: CHANGES.txt src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java
Date Tue, 22 Dec 2015 19:21:06 GMT
Author: daijy
Date: Tue Dec 22 19:21:05 2015
New Revision: 1721448

URL: http://svn.apache.org/viewvc?rev=1721448&view=rev
Log:
PIG-4760: TezDAGStats.convertToHadoopCounters is not used, but impose MR counter limit

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1721448&r1=1721447&r2=1721448&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Tue Dec 22 19:21:05 2015
@@ -77,6 +77,8 @@ PIG-4639: Add better parser for Apache H
 
 BUG FIXES
 
+PIG-4760: TezDAGStats.convertToHadoopCounters is not used, but impose MR counter limit (daijy)
+
 PIG-4755: Typo in runpigmix script (mitdesai via daijy)
 
 PIG-4736: Removing empty keys in UDFContext broke one LoadFunc (rohini)

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java?rev=1721448&r1=1721447&r2=1721448&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java Tue Dec
22 19:21:05 2015
@@ -19,6 +19,7 @@ package org.apache.pig.backend.hadoop.ex
 
 import java.io.IOException;
 import java.io.PrintStream;
+import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -120,16 +121,26 @@ public class TezLauncher extends Launche
         Configuration conf = ConfigurationUtil.toConfiguration(pc.getProperties(), true);
         // Make sure MR counter does not exceed limit
         if (conf.get(TezConfiguration.TEZ_COUNTERS_MAX) != null) {
-            conf.setInt(org.apache.hadoop.mapreduce.MRJobConfig.COUNTER_GROUPS_MAX_KEY, Math.max(
-                    conf.getInt(org.apache.hadoop.mapreduce.MRJobConfig.COUNTER_GROUPS_MAX_KEY,
0),
+            conf.setInt(org.apache.hadoop.mapreduce.MRJobConfig.COUNTERS_MAX_KEY, Math.max(
+                    conf.getInt(org.apache.hadoop.mapreduce.MRJobConfig.COUNTERS_MAX_KEY,
0),
                     conf.getInt(TezConfiguration.TEZ_COUNTERS_MAX, 0)));
         }
         if (conf.get(TezConfiguration.TEZ_COUNTERS_MAX_GROUPS) != null) {
-            conf.setInt(org.apache.hadoop.mapreduce.MRJobConfig.COUNTERS_MAX_KEY, Math.max(
-                    conf.getInt(org.apache.hadoop.mapreduce.MRJobConfig.COUNTERS_MAX_KEY,
0),
+            conf.setInt(org.apache.hadoop.mapreduce.MRJobConfig.COUNTER_GROUPS_MAX_KEY, Math.max(
+                    conf.getInt(org.apache.hadoop.mapreduce.MRJobConfig.COUNTER_GROUPS_MAX_KEY,
0),
                     conf.getInt(TezConfiguration.TEZ_COUNTERS_MAX_GROUPS, 0)));
         }
-        Limits.init(conf);
+
+        // This is hacky, but Limits cannot be initialized twice
+        try {
+            Field f = Limits.class.getDeclaredField("isInited");
+            f.setAccessible(true);
+            f.setBoolean(null, false);
+            Limits.init(conf);
+        } catch (Throwable e) {
+            log.warn("Error when setting counter limit: " + e.getMessage());
+        }
+
         if (pc.defaultParallel == -1 && !conf.getBoolean(PigConfiguration.PIG_TEZ_AUTO_PARALLELISM,
true)) {
             pc.defaultParallel = 1;
         }



Mime
View raw message