You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ro...@apache.org on 2017/11/28 15:53:22 UTC
svn commit: r1816557 - in /pig/trunk: CHANGES.txt
src/org/apache/pig/backend/hadoop/executionengine/tez/runtime/PigProcessor.java
Author: rohini
Date: Tue Nov 28 15:53:22 2017
New Revision: 1816557
URL: http://svn.apache.org/viewvc?rev=1816557&view=rev
Log:
PIG-5314: Abort method is not implemented in PigProcessor (satishsaley via rohini)
Modified:
pig/trunk/CHANGES.txt
pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/runtime/PigProcessor.java
Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1816557&r1=1816556&r2=1816557&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Tue Nov 28 15:53:22 2017
@@ -60,6 +60,8 @@ OPTIMIZATIONS
BUG FIXES
+PIG-5314: Abort method is not implemented in PigProcessor (satishsaley via rohini)
+
PIG-5307: NPE in TezOperDependencyParallelismEstimator (rohini)
PIG-5272: BagToTuple output schema is incorrect (juen1jp via rohini)
Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/runtime/PigProcessor.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/runtime/PigProcessor.java?rev=1816557&r1=1816556&r2=1816557&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/runtime/PigProcessor.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/runtime/PigProcessor.java Tue Nov 28 15:53:22 2017
@@ -115,6 +115,7 @@ public class PigProcessor extends Abstra
public static String sampleVertex;
public static Map<String, Object> sampleMap;
+ private volatile boolean isAborted = false;
public PigProcessor(ProcessorContext context) {
super(context);
@@ -305,9 +306,12 @@ public class PigProcessor extends Abstra
}
if (!fileOutputs.isEmpty()) {
- while (!getContext().canCommit()) {
+ while (!getContext().canCommit() && !isAborted) {
Thread.sleep(100);
}
+ if (isAborted) {
+ return;
+ }
for (MROutput fileOutput : fileOutputs){
fileOutput.flush();
if (fileOutput.isCommitRequired()) {
@@ -464,4 +468,11 @@ public class PigProcessor extends Abstra
}
}
+ // TODO add @Override when we upgrade to Tez 0.9 dependency
+ public void abort() {
+ isAborted = true;
+ LOG.warn("Aborting execution");
+ abortOutput();
+ }
+
}