Return-Path: Delivered-To: apmail-hadoop-pig-commits-archive@www.apache.org Received: (qmail 89835 invoked from network); 13 Feb 2009 02:00:08 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 13 Feb 2009 02:00:08 -0000 Received: (qmail 54042 invoked by uid 500); 13 Feb 2009 02:00:08 -0000 Delivered-To: apmail-hadoop-pig-commits-archive@hadoop.apache.org Received: (qmail 54023 invoked by uid 500); 13 Feb 2009 02:00:08 -0000 Mailing-List: contact pig-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: pig-dev@hadoop.apache.org Delivered-To: mailing list pig-commits@hadoop.apache.org Received: (qmail 54014 invoked by uid 500); 13 Feb 2009 02:00:08 -0000 Delivered-To: apmail-incubator-pig-commits@incubator.apache.org Received: (qmail 54011 invoked by uid 99); 13 Feb 2009 02:00:08 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 Feb 2009 18:00:08 -0800 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 13 Feb 2009 01:59:58 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id C81A52388AF5; Fri, 13 Feb 2009 01:59:36 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r743952 [4/4] - in /hadoop/pig/trunk: ./ src/org/apache/pig/ src/org/apache/pig/backend/executionengine/ src/org/apache/pig/backend/hadoop/ src/org/apache/pig/backend/hadoop/datastorage/ src/org/apache/pig/backend/hadoop/executionengine/ sr... Date: Fri, 13 Feb 2009 01:59:30 -0000 To: pig-commits@incubator.apache.org From: olga@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20090213015936.C81A52388AF5@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/data/DefaultAbstractBag.java Fri Feb 13 01:59:27 2009 @@ -28,6 +28,7 @@ import java.util.Iterator; import java.util.ArrayList; +import org.apache.pig.PigException; import org.apache.pig.backend.executionengine.ExecException; import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator; import org.apache.pig.impl.util.Spillable; @@ -243,7 +244,7 @@ Object o = DataReaderWriter.readDatum(in); add((Tuple)o); } catch (ExecException ee) { - throw new RuntimeException(ee); + throw ee; } } } @@ -327,8 +328,9 @@ if (tmpDir.exists()) { log.info("Temporary directory already exists: " + tmpDir.getAbsolutePath()); } else { - log.error("Unable to create temporary directory: " + tmpDir.getAbsolutePath()); - throw new IOException("Unable to create temporary directory: " + tmpDir.getAbsolutePath() ); + int errCode = 2111; + String msg = "Unable to create temporary directory: " + tmpDir.getAbsolutePath(); + throw new ExecException(msg, errCode, PigException.BUG); } } } Modified: hadoop/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/data/DefaultDataBag.java Fri Feb 13 01:59:27 2009 @@ -89,7 +89,7 @@ } catch (IOException ioe) { // Do not remove last file from spilled array. It was not // added as File.createTmpFile threw an IOException - log.error( + log.warn( "Unable to create tmp file to spill to disk", ioe); return 0; } @@ -106,7 +106,7 @@ // Remove the last file from the spilled array, since we failed to // write to it. mSpillFiles.remove(mSpillFiles.size() - 1); - log.error( + log.warn( "Unable to spill contents to disk", ioe); return 0; } finally { @@ -114,7 +114,7 @@ try { out.close(); } catch (IOException e) { - log.error("Error closing spill", e); + log.warn("Error closing spill", e); } } } @@ -193,9 +193,9 @@ } catch (FileNotFoundException fnfe) { // We can't find our own spill file? That should never // happen. - log.fatal( - "Unable to find our spill file", fnfe); - throw new RuntimeException(fnfe); + String msg = "Unable to find our spill file."; + log.fatal(msg, fnfe); + throw new RuntimeException(msg, fnfe); } Tuple t = gTupleFactory.newTuple(); for (int i = 0; i < mMemoryPtr; i++) { @@ -204,13 +204,13 @@ } catch (EOFException eof) { // This should never happen, it means we // didn't dump all of our tuples to disk. - log.fatal( - "Ran out of tuples too soon.", eof); - throw new RuntimeException("Ran out of tuples to read prematurely.", eof); + String msg = "Ran out of tuples to read prematurely."; + log.fatal(msg, eof); + throw new RuntimeException(msg, eof); } catch (IOException ioe) { - log.fatal( - "Unable to read our spill file", ioe); - throw new RuntimeException(ioe); + String msg = "Unable to read our spill file."; + log.fatal(msg, ioe); + throw new RuntimeException(msg, ioe); } } mMemoryPtr = 0; @@ -238,9 +238,9 @@ // Fall through to the next case where we find the // next file, or go to memory } catch (IOException ioe) { - log.fatal( - "Unable to read our spill file", ioe); - throw new RuntimeException(ioe); + String msg = "Unable to read our spill file."; + log.fatal(msg, ioe); + throw new RuntimeException(msg, ioe); } } @@ -267,9 +267,9 @@ } catch (FileNotFoundException fnfe) { // We can't find our own spill file? That should never // happen. - log.fatal("Unable to find our spill file", - fnfe); - throw new RuntimeException(fnfe); + String msg = "Unable to find our spill file."; + log.fatal(msg, fnfe); + throw new RuntimeException(msg, fnfe); } return readFromFile(); } Modified: hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java Fri Feb 13 01:59:27 2009 @@ -299,8 +299,10 @@ // Make sure it's a tuple. byte b = in.readByte(); if (b != DataType.TUPLE) { - throw new IOException("Unexpected data while reading tuple " + - "from binary file"); + int errCode = 2112; + String msg = "Unexpected data while reading tuple " + + "from binary file."; + throw new ExecException(msg, errCode, PigException.BUG); } // Read the number of fields int sz = in.readInt(); @@ -308,7 +310,7 @@ try { append(DataReaderWriter.readDatum(in)); } catch (ExecException ee) { - throw new RuntimeException(ee); + throw ee; } } } Modified: hadoop/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java Fri Feb 13 01:59:27 2009 @@ -135,7 +135,7 @@ } catch (IOException ioe) { // Do not remove last file from spilled array. It was not // added as File.createTmpFile threw an IOException - log.error( + log.warn( "Unable to create tmp file to spill to disk", ioe); return 0; } @@ -167,7 +167,7 @@ // Remove the last file from the spilled array, since we failed to // write to it. mSpillFiles.remove(mSpillFiles.size() - 1); - log.error( + log.warn( "Unable to spill contents to disk", ioe); return 0; } finally { @@ -175,7 +175,7 @@ try { out.close(); } catch (IOException e) { - log.error("Error closing spill", e); + log.warn("Error closing spill", e); } } } @@ -283,9 +283,9 @@ } catch (FileNotFoundException fnfe) { // We can't find our own spill file? That should never // happen. - log.fatal( - "Unable to find our spill file", fnfe); - throw new RuntimeException(fnfe); + String msg = "Unable to find our spill file."; + log.fatal(msg, fnfe); + throw new RuntimeException(msg, fnfe); } // Fast foward past the tuples we've already put in the @@ -299,9 +299,9 @@ // didn't dump all of our tuples to disk. throw new RuntimeException("Ran out of tuples to read prematurely.", eof); } catch (IOException ioe) { - log.fatal( - "Unable to read our spill file", ioe); - throw new RuntimeException(ioe); + String msg = "Unable to find our spill file."; + log.fatal(msg, ioe); + throw new RuntimeException(msg, ioe); } } mMemoryPtr = 0; @@ -344,9 +344,9 @@ } catch (FileNotFoundException fnfe) { // We can't find our own spill file? That should // never happen. - log.fatal( - "Unable to find out spill file.", fnfe); - throw new RuntimeException(fnfe); + String msg = "Unable to find our spill file."; + log.fatal(msg, fnfe); + throw new RuntimeException(msg, fnfe); } } @@ -419,9 +419,9 @@ mStreams.set(fileNum, null); return; } catch (IOException ioe) { - log.fatal( - "Unable to read our spill file", ioe); - throw new RuntimeException(ioe); + String msg = "Unable to find our spill file."; + log.fatal(msg, ioe); + throw new RuntimeException(msg, ioe); } } while (true); } @@ -486,9 +486,9 @@ } catch (FileNotFoundException fnfe) { // We can't find our own spill file? That should // neer happen. - log.fatal( - "Unable to find out spill file.", fnfe); - throw new RuntimeException(fnfe); + String msg = "Unable to find our spill file."; + log.fatal(msg, fnfe); + throw new RuntimeException(msg, fnfe); } } @@ -505,9 +505,9 @@ } out.flush(); } catch (IOException ioe) { - log.fatal( - "Unable to read our spill file", ioe); - throw new RuntimeException(ioe); + String msg = "Unable to find our spill file."; + log.fatal(msg, ioe); + throw new RuntimeException(msg, ioe); } } Modified: hadoop/pig/trunk/src/org/apache/pig/data/NonSpillableDataBag.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/NonSpillableDataBag.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/data/NonSpillableDataBag.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/data/NonSpillableDataBag.java Fri Feb 13 01:59:27 2009 @@ -176,7 +176,7 @@ Object o = DataReaderWriter.readDatum(in); add((Tuple)o); } catch (ExecException ee) { - throw new RuntimeException(ee); + throw ee; } } } Modified: hadoop/pig/trunk/src/org/apache/pig/data/SingleTupleBag.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/SingleTupleBag.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/data/SingleTupleBag.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/data/SingleTupleBag.java Fri Feb 13 01:59:27 2009 @@ -8,6 +8,9 @@ import java.io.IOException; import java.util.Iterator; +import org.apache.pig.PigException; +import org.apache.pig.backend.executionengine.ExecException; + /** * A simple performant implementation of the DataBag * interface which only holds a single tuple. This will @@ -110,8 +113,9 @@ @Override public void readFields(DataInput in) throws IOException { // TODO Auto-generated method stub - throw new IOException("SingleTupleBag should never be serialized or serialized"); - + int errCode = 2113; + String msg = "SingleTupleBag should never be serialized or serialized."; + throw new ExecException(msg, errCode, PigException.BUG); } /* (non-Javadoc) @@ -120,7 +124,9 @@ @Override public void write(DataOutput out) throws IOException { // TODO Auto-generated method stub - throw new IOException("SingleTupleBag should never be serialized or deserialized"); + int errCode = 2113; + String msg = "SingleTupleBag should never be serialized or serialized."; + throw new ExecException(msg, errCode, PigException.BUG); } /* (non-Javadoc) Modified: hadoop/pig/trunk/src/org/apache/pig/data/SortedDataBag.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/SortedDataBag.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/data/SortedDataBag.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/data/SortedDataBag.java Fri Feb 13 01:59:27 2009 @@ -108,7 +108,7 @@ } catch (IOException ioe) { // Do not remove last file from spilled array. It was not // added as File.createTmpFile threw an IOException - log.error( + log.warn( "Unable to create tmp file to spill to disk", ioe); return 0; } @@ -135,7 +135,7 @@ // Remove the last file from the spilled array, since we failed to // write to it. mSpillFiles.remove(mSpillFiles.size() - 1); - log.error( + log.warn( "Unable to spill contents to disk", ioe); return 0; } finally { @@ -143,7 +143,7 @@ try { out.close(); } catch (IOException e) { - log.error("Error closing spill", e); + log.warn("Error closing spill", e); } } } @@ -254,9 +254,9 @@ } catch (FileNotFoundException fnfe) { // We can't find our own spill file? That should never // happen. - log.fatal( - "Unable to find our spill file", fnfe); - throw new RuntimeException(fnfe); + String msg = "Unable to find our spill file."; + log.fatal(msg, fnfe); + throw new RuntimeException(msg, fnfe); } // Fast foward past the tuples we've already put in the @@ -268,13 +268,13 @@ } catch (EOFException eof) { // This should never happen, it means we // didn't dump all of our tuples to disk. - log.fatal( - "Ran out of tuples too soon.", eof); - throw new RuntimeException("Ran out of tuples to read prematurely.", eof); + String msg = "Ran out of tuples to read prematurely."; + log.fatal(msg, eof); + throw new RuntimeException(msg, eof); } catch (IOException ioe) { - log.fatal( - "Unable to read our spill file", ioe); - throw new RuntimeException(ioe); + String msg = "Unable to find our spill file."; + log.fatal(msg, ioe); + throw new RuntimeException(msg, ioe); } } mMemoryPtr = 0; @@ -319,9 +319,9 @@ } catch (FileNotFoundException fnfe) { // We can't find our own spill file? That should // never happen. - log.fatal( - "Unable to find our spill file", fnfe); - throw new RuntimeException(fnfe); + String msg = "Unable to find our spill file."; + log.fatal(msg, fnfe); + throw new RuntimeException(msg, fnfe); } } @@ -378,9 +378,9 @@ // this file. mStreams.set(fileNum, null); } catch (IOException ioe) { - log.fatal( - "Unable to read our spill file", ioe); - throw new RuntimeException(ioe); + String msg = "Unable to find our spill file."; + log.fatal(msg, ioe); + throw new RuntimeException(msg, ioe); } } @@ -442,9 +442,9 @@ } catch (FileNotFoundException fnfe) { // We can't find our own spill file? That should // neer happen. - log.fatal( - "Unable to find our spill file", fnfe); - throw new RuntimeException(fnfe); + String msg = "Unable to find our spill file."; + log.fatal(msg, fnfe); + throw new RuntimeException(msg, fnfe); } } @@ -461,9 +461,9 @@ } out.flush(); } catch (IOException ioe) { - log.fatal( - "Unable to read our spill file", ioe); - throw new RuntimeException(ioe); + String msg = "Unable to find our spill file."; + log.fatal(msg, ioe); + throw new RuntimeException(msg, ioe); } } Modified: hadoop/pig/trunk/src/org/apache/pig/impl/PigContext.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/PigContext.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/impl/PigContext.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/PigContext.java Fri Feb 13 01:59:27 2009 @@ -244,7 +244,7 @@ break; } String msg = "Unable to rename " + oldName + " to " + newName; - throw new PigException(msg, errCode, errSrc, e); + throw new ExecException(msg, errCode, errSrc, e); } if (dst.exists()) { @@ -284,7 +284,7 @@ break; } String msg = "Unable to copy " + src + " to " + dst; - throw new PigException(msg, errCode, errSrc, e); + throw new ExecException(msg, errCode, errSrc, e); } srcElement.copy(dstElement, this.properties, false); @@ -412,7 +412,7 @@ catch (UnsupportedClassVersionError e) { int errCode = 1069; String msg = "Problem resolving class version numbers for class " + name; - throw new PigException(msg, errCode, PigException.INPUT, e) ; + throw new ExecException(msg, errCode, PigException.INPUT, e) ; } } @@ -421,7 +421,7 @@ // so that we don't need to buble interface changes throughout the code int errCode = 1070; String msg = "Could not resolve " + name + " using imports: " + packageImportList; - throw new PigException(msg, errCode, PigException.INPUT); + throw new ExecException(msg, errCode, PigException.INPUT); } Modified: hadoop/pig/trunk/src/org/apache/pig/impl/builtin/FindQuantiles.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/builtin/FindQuantiles.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/impl/builtin/FindQuantiles.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/builtin/FindQuantiles.java Fri Feb 13 01:59:27 2009 @@ -109,9 +109,7 @@ numQuantiles = (Integer)input.get(0); samples = (DataBag)input.get(1); }catch(ExecException e){ - IOException ioe = new IOException(); - ioe.initCause(e); - throw ioe; + throw e; } // TODO If user provided a comparator we should be using that. DataBag output = mBagFactory.newSortedBag(mComparator); Modified: hadoop/pig/trunk/src/org/apache/pig/impl/builtin/GFCross.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/builtin/GFCross.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/impl/builtin/GFCross.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/builtin/GFCross.java Fri Feb 13 01:59:27 2009 @@ -63,9 +63,7 @@ return output; }catch(ExecException e){ - IOException ioe = new IOException(); - ioe.initCause(e); - throw ioe; + throw e; } } Modified: hadoop/pig/trunk/src/org/apache/pig/impl/builtin/RandomSampleLoader.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/builtin/RandomSampleLoader.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/impl/builtin/RandomSampleLoader.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/builtin/RandomSampleLoader.java Fri Feb 13 01:59:27 2009 @@ -21,6 +21,8 @@ import java.io.InputStream; import java.io.OutputStream; +import org.apache.pig.PigException; +import org.apache.pig.backend.executionengine.ExecException; import org.apache.pig.builtin.BinStorage; import org.apache.pig.data.Tuple; import org.apache.pig.impl.io.BufferedPositionedInputStream; @@ -66,8 +68,10 @@ } @Override - public void bindTo(OutputStream os) throws IOException { - throw new RuntimeException(this.getClass().getName() + " should not be used for writing"); + public void bindTo(OutputStream os) throws IOException { + int errCode = 2101; + String msg = this.getClass().getName() + " should not be used for storing."; + throw new ExecException(msg, errCode, PigException.BUG); } Modified: hadoop/pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java Fri Feb 13 01:59:27 2009 @@ -451,7 +451,6 @@ public static boolean fileExists(String filename, DataStorage store) throws IOException { ElementDescriptor elem = store.asElement(filename); - return elem.exists() || globMatchesFiles(elem, store); } @@ -496,7 +495,8 @@ } } catch (DataStorageException e) { - throw WrappedIOException.wrap("Unable to get collect for pattern " + elem.toString(), e); + //throw WrappedIOException.wrap("Unable to get collect for pattern " + elem.toString(), e); + throw e; } } Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOFRJoin.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOFRJoin.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOFRJoin.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/LOFRJoin.java Fri Feb 13 01:59:27 2009 @@ -25,6 +25,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.pig.PigException; import org.apache.pig.data.DataType; import org.apache.pig.impl.logicalLayer.optimizer.SchemaRemover; import org.apache.pig.impl.logicalLayer.schema.Schema; @@ -183,8 +184,10 @@ } public byte getAtomicJoinColType() throws FrontendException { if (isTupleJoinCol()) { - throw new FrontendException("getAtomicjoinByType is used only when" - + " dealing with atomic join col") ; + int errCode = 1010; + String msg = "getAtomicGroupByType is used only when" + + " dealing with atomic join col"; + throw new FrontendException(msg, errCode, PigException.INPUT, false, null) ; } byte joinColType = DataType.BYTEARRAY ; @@ -195,8 +198,10 @@ List innerPlans = new ArrayList(getJoinColPlans().get(input)) ; if (innerPlans.size() != 1) { - throw new FrontendException("Each join input has to have " - + "the same number of inner plans") ; + int errCode = 1012; + String msg = "Each join input has to have " + + "the same number of inner plans"; + throw new FrontendException(msg, errCode, PigException.INPUT, false, null) ; } byte innerType = innerPlans.get(0).getSingleLeafPlanOutputType() ; joinColType = DataType.mergeType(joinColType, innerType) ; @@ -207,8 +212,10 @@ public Schema getTupleJoinColSchema() throws FrontendException { if (!isTupleJoinCol()) { - throw new FrontendException("getTupleJoinColSchema is used only when" - + " dealing with tuple join col") ; + int errCode = 1011; + String msg = "getTupleGroupBySchema is used only when" + + " dealing with tuple join col"; + throw new FrontendException(msg, errCode, PigException.INPUT, false, null) ; } // this fsList represents all the columns in join tuple @@ -250,8 +257,10 @@ } } - if(seenProjectStar && innerPlans.size() > 1) { - throw new FrontendException("joining attributes can either be star (*) or a list of expressions, but not both."); + if(seenProjectStar && innerPlans.size() > 1) { + int errCode = 1013; + String msg = "Join attributes can either be star (*) or a list of expressions, but not both."; + throw new FrontendException(msg, errCode, PigException.INPUT, false, null); } 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=743952&r1=743951&r2=743952&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 Fri Feb 13 01:59:27 2009 @@ -88,8 +88,7 @@ } } catch (PlanValidationException pve) { throw pve; - } - catch (IOException ioe) { + } catch (IOException ioe) { byte errSrc = pigCtx.getErrorSource(); int errCode = 0; switch(errSrc) { @@ -108,6 +107,11 @@ + filename + " will be stored "; msgCollector.collect(msg, MessageType.Error) ; throw new PlanValidationException(msg, errCode, errSrc, ioe); + } catch (Exception e) { + int errCode = 2116; + String msg = "Unexpected error. Could not check for the existence of the file(s): " + filename; + msgCollector.collect(msg, MessageType.Error) ; + throw new PlanValidationException(msg, errCode, PigException.BUG, e); } } Modified: hadoop/pig/trunk/src/org/apache/pig/impl/plan/optimizer/OptimizerException.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/plan/optimizer/OptimizerException.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/impl/plan/optimizer/OptimizerException.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/impl/plan/optimizer/OptimizerException.java Fri Feb 13 01:59:27 2009 @@ -18,8 +18,9 @@ package org.apache.pig.impl.plan.optimizer; import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.plan.VisitorException; -public class OptimizerException extends FrontendException { +public class OptimizerException extends VisitorException { /** * Create a new OptimizerException with null as the error message. Modified: hadoop/pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java Fri Feb 13 01:59:27 2009 @@ -72,7 +72,8 @@ parser.parseStopOnError(); break; } catch(Throwable t) { - writeLog(t, verbose, append); + //writeLog(t, verbose, append); + Utils.writeLog(t, pig.getPigContext().getProperties().getProperty("pig.logfile"), log, verbose); append = true; parser.ReInit(in); } @@ -86,12 +87,13 @@ parser.setInteractive(false); parser.parseStopOnError(); } catch (Throwable t) { - writeLog(t, verbose, false); + //writeLog(t, verbose, false); + Utils.writeLog(t, pig.getPigContext().getProperties().getProperty("pig.logfile"), log, verbose); throw (t); } } - private void writeLog(Throwable t, boolean verbose, boolean append) { +/* private void writeLog(Throwable t, boolean verbose, boolean append) { String message = null; @@ -154,5 +156,5 @@ log.warn("Could not write to log file: " + logFileName + " :" + ioe.getMessage()); log.error(bs.toString()); } - } + }*/ } Modified: hadoop/pig/trunk/src/org/apache/pig/tools/grunt/Utils.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/tools/grunt/Utils.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/src/org/apache/pig/tools/grunt/Utils.java (original) +++ hadoop/pig/trunk/src/org/apache/pig/tools/grunt/Utils.java Fri Feb 13 01:59:27 2009 @@ -17,7 +17,16 @@ */ package org.apache.pig.tools.grunt; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.PrintStream; + +import org.apache.commons.logging.Log; import org.apache.pig.PigException; +import org.apache.pig.impl.PigContext; +import org.apache.pig.tools.pigscript.parser.ParseException; public class Utils { static Exception getPermissionException(Exception top){ @@ -42,6 +51,78 @@ return (pigException instanceof PigException? (PigException)pigException : null); } + + public static void writeLog(Throwable t, String logFileName, Log log, boolean verbose) { + writeLog(t, logFileName, log, verbose, null, true, true); + } + + public static void writeLog(Throwable t, String logFileName, Log log, boolean verbose, + String headerMessage, boolean displayFooter, boolean displayMessage) { + + String message = null; + + if(t instanceof Exception) { + Exception pe = Utils.getPermissionException((Exception)t); + if (pe != null) { + log.error("You don't have permission to perform the operation. Error from the server: " + pe.getMessage()); + } + } + + PigException pigException = Utils.getPigException(t); + + if(pigException != null) { + message = "ERROR " + pigException.getErrorCode() + ": " + pigException.getMessage(); + } else { + if((t instanceof ParseException + || t instanceof org.apache.pig.tools.pigscript.parser.TokenMgrError + || t instanceof org.apache.pig.impl.logicalLayer.parser.TokenMgrError)) { + message = "ERROR 1000: Error during parsing. " + t.getMessage(); + } else if (t instanceof RuntimeException) { + message = "ERROR 2999: Unexpected internal error. " + t.getMessage(); + } else { + message = "ERROR 2998: Unhandled internal error. " + t.getMessage(); + } + } + + + FileOutputStream fos = null; + ByteArrayOutputStream bs = new ByteArrayOutputStream(); + t.printStackTrace(new PrintStream(bs)); + + if(displayMessage) log.error(message); + + if(verbose) { + log.error(bs.toString()); + } + + if(logFileName == null) { + //if exec is invoked programmatically then logFileName will be null + log.warn("There is no log file to write to"); + log.error(bs.toString()); + return; + } + + + File logFile = new File(logFileName); + try { + fos = new FileOutputStream(logFile, true); + if(headerMessage != null) fos.write((headerMessage + "\n").getBytes("UTF-8")); + fos.write((message + "\n").getBytes("UTF-8")); + fos.write(bs.toString().getBytes("UTF-8")); + fos.close(); + if(displayFooter) { + if(verbose) { + System.err.println("Details also at logfile: " + logFileName); + } else { + System.err.println("Details at logfile: " + logFileName); + } + } + } catch (IOException ioe) { + log.warn("Could not write to log file: " + logFileName + " :" + ioe.getMessage()); + log.error(bs.toString()); + } + } + } Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestPOCogroup.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestPOCogroup.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestPOCogroup.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestPOCogroup.java Fri Feb 13 01:59:27 2009 @@ -39,6 +39,7 @@ import org.apache.pig.data.Tuple; import org.apache.pig.data.TupleFactory; import org.apache.pig.impl.plan.OperatorKey; +import org.apache.pig.impl.plan.PlanException; public class TestPOCogroup extends TestCase { Random r = new Random(); @@ -146,7 +147,7 @@ assertEquals(expected.size(), obtained.size()); } - public void testCogroup1Input() throws ExecException { + public void testCogroup1Input() throws ExecException, PlanException { DataBag input = BagFactory.getInstance().newDefaultBag(); Tuple t = TupleFactory.getInstance().newTuple(); t.append(1); Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestPigContext.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestPigContext.java?rev=743952&r1=743951&r2=743952&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestPigContext.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestPigContext.java Fri Feb 13 01:59:27 2009 @@ -27,6 +27,7 @@ import junit.framework.TestCase; +import org.apache.hadoop.mapred.FileAlreadyExistsException; import org.apache.pig.ExecType; import org.apache.pig.PigServer; import org.apache.pig.impl.PigContext; @@ -83,6 +84,12 @@ check_asserts(); } + + @Test + public void testHadoopExceptionCreation() throws Exception { + Object object = PigContext.instantiateFuncFromSpec("org.apache.hadoop.mapred.FileAlreadyExistsException"); + assertTrue(object instanceof FileAlreadyExistsException); + } @After @Override