You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ha...@apache.org on 2010/08/31 03:55:04 UTC
svn commit: r991056 - in /hadoop/pig/trunk: ./
src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
src/org/apache/pig/impl/logicalLayer/validators/ test/org/apache/pig/test/
Author: hashutosh
Date: Tue Aug 31 01:55:04 2010
New Revision: 991056
URL: http://svn.apache.org/viewvc?rev=991056&view=rev
Log:
PIG-1531: Pig gobbles up error messages
Modified:
hadoop/pig/trunk/CHANGES.txt
hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigInputFormat.java
hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/InputOutputFileVisitor.java
hadoop/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java
Modified: hadoop/pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=991056&r1=991055&r2=991056&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Tue Aug 31 01:55:04 2010
@@ -26,6 +26,8 @@ PIG-1249: Safe-guards against misconfigu
IMPROVEMENTS
+PIG-1531: Pig gobbles up error messages (nrai via hashutosh)
+
PIG-1458: aggregate files for replicated join (rding)
PIG-1205: Enhance HBaseStorage-- Make it support loading row key and implement StoreFunc (zjffdu and dvryaboy)
Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigInputFormat.java?rev=991056&r1=991055&r2=991056&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigInputFormat.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigInputFormat.java Tue Aug 31 01:55:04 2010
@@ -276,7 +276,11 @@ public class PigInputFormat extends Inpu
} catch (Exception e) {
int errCode = 2118;
String msg = "Unable to create input splits for: " + inputs.get(i).getFileName();
- throw new ExecException(msg, errCode, PigException.BUG, e);
+ if(e.getMessage() !=null && (!e.getMessage().isEmpty()) ){
+ throw new ExecException(e.getMessage(), errCode, PigException.BUG, e);
+ }else{
+ throw new ExecException(msg, errCode, PigException.BUG, e);
+ }
}
}
Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/InputOutputFileVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/InputOutputFileVisitor.java?rev=991056&r1=991055&r2=991056&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/InputOutputFileVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/InputOutputFileVisitor.java Tue Aug 31 01:55:04 2010
@@ -24,8 +24,6 @@ import org.apache.pig.PigException;
import org.apache.pig.ResourceSchema;
import org.apache.pig.StoreFuncInterface;
import org.apache.pig.impl.PigContext ;
-import org.apache.pig.impl.logicalLayer.FrontendException;
-import org.apache.pig.impl.logicalLayer.LONative;
import org.apache.pig.impl.logicalLayer.LOStore;
import org.apache.pig.impl.logicalLayer.LOVisitor;
import org.apache.pig.impl.logicalLayer.LogicalOperator;
@@ -37,18 +35,16 @@ import org.apache.pig.impl.plan.PlanVali
import org.apache.pig.impl.plan.CompilationMessageCollector.MessageType;
/***
- * Visitor for checking input/output files
- * Exceptions in here do not affect later operations
- * so we don't throw any exception but log all of
- * them in msgCollector.
+ * Visitor for checking output specification
+ * In addition to throwing exception we also log them in msgCollector.
*
* We assume input/output files can exist only in the top level plan.
*
*/
public class InputOutputFileVisitor extends LOVisitor {
- private PigContext pigCtx = null ;
- private CompilationMessageCollector msgCollector = null ;
+ private PigContext pigCtx;
+ private CompilationMessageCollector msgCollector;
public InputOutputFileVisitor(LogicalPlan plan,
CompilationMessageCollector messageCollector,
@@ -58,7 +54,7 @@ public class InputOutputFileVisitor exte
msgCollector = messageCollector ;
}
-
+
/***
* The logic here is to delegate the validation of output specification
* to output format implementation.
@@ -68,36 +64,31 @@ public class InputOutputFileVisitor exte
StoreFuncInterface sf = store.getStoreFunc();
String outLoc = store.getOutputFile().getFileName();
- String errMsg = "Unexpected error. Could not validate the output " +
- "specification for: "+outLoc;
int errCode = 2116;
-
+ String validationErrStr ="Output Location Validation Failed for: '" + outLoc ;
+ Job dummyJob;
+
try {
if(store.getSchema() != null){
sf.checkSchema(new ResourceSchema(store.getSchema(), store.getSortInfo()));
}
- } catch (FrontendException e) {
- msgCollector.collect(errMsg, MessageType.Error) ;
- throw new PlanValidationException(errMsg, errCode, pigCtx.getErrorSource(), e);
- } catch (IOException e) {
- msgCollector.collect(errMsg, MessageType.Error) ;
- throw new PlanValidationException(errMsg, errCode, pigCtx.getErrorSource(), e);
- }
-
- Job dummyJob;
-
- try {
dummyJob = new Job(ConfigurationUtil.toConfiguration(pigCtx.getProperties()));
sf.setStoreLocation(outLoc, dummyJob);
} catch (IOException ioe) {
- msgCollector.collect(errMsg, MessageType.Error) ;
- throw new PlanValidationException(errMsg, errCode, pigCtx.getErrorSource(), ioe);
+ if(ioe instanceof PigException){
+ errCode = ((PigException)ioe).getErrorCode();
+ }
+ String exceptionMsg = ioe.getMessage();
+ validationErrStr += (exceptionMsg == null) ? "" : " More info to follow:\n" +exceptionMsg;
+ msgCollector.collect(validationErrStr, MessageType.Error) ;
+ throw new PlanValidationException(validationErrStr, errCode, pigCtx.getErrorSource(), ioe);
}
+
+ validationErrStr += " More info to follow:\n";
try {
sf.getOutputFormat().checkOutputSpecs(dummyJob);
} catch (IOException ioe) {
byte errSrc = pigCtx.getErrorSource();
- errCode = 0;
switch(errSrc) {
case PigException.BUG:
errCode = 2002;
@@ -109,13 +100,13 @@ public class InputOutputFileVisitor exte
errCode = 4000;
break;
}
- errMsg = "Output specification '"+outLoc+"' is invalid or already exists";
- msgCollector.collect(errMsg, MessageType.Error) ;
- throw new PlanValidationException(errMsg, errCode, errSrc, ioe);
+ validationErrStr += ioe.getMessage();
+ msgCollector.collect(validationErrStr, MessageType.Error) ;
+ throw new PlanValidationException(validationErrStr, errCode, errSrc, ioe);
} catch (InterruptedException ie) {
- msgCollector.collect(errMsg, MessageType.Error) ;
- throw new PlanValidationException(errMsg, errCode, pigCtx.getErrorSource(), ie);
+ validationErrStr += ie.getMessage();
+ msgCollector.collect(validationErrStr, MessageType.Error) ;
+ throw new PlanValidationException(validationErrStr, errCode, pigCtx.getErrorSource(), ie);
}
}
-
}
Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java?rev=991056&r1=991055&r2=991056&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java Tue Aug 31 01:55:04 2010
@@ -26,7 +26,6 @@ import org.apache.pig.FuncSpec;
import org.apache.pig.PigServer;
import org.apache.pig.backend.datastorage.DataStorage;
import org.apache.pig.backend.datastorage.ElementDescriptor;
-import org.apache.pig.backend.executionengine.ExecException;
import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
import org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil;
import org.apache.pig.data.Tuple;
@@ -233,6 +232,7 @@ public class TestInputOutputFileValidato
pig.store("a", output);
} catch (Exception e) {
assertEquals(6000, LogUtils.getPigException(e).getErrorCode());
+ assertTrue(LogUtils.getPigException(e).getMessage().contains("Output Location Validation Failed for"));
exceptionCaught = true;
}
if(!exceptionCaught) {