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 2014/10/31 22:44:36 UTC
svn commit: r1635868 - in /pig/trunk: ./
src/org/apache/pig/backend/hadoop/accumulo/
src/org/apache/pig/backend/hadoop/hbase/ src/org/apache/pig/builtin/
src/org/apache/pig/data/ src/org/apache/pig/impl/io/
src/org/apache/pig/impl/util/ src/org/apache/...
Author: daijy
Date: Fri Oct 31 21:44:35 2014
New Revision: 1635868
URL: http://svn.apache.org/r1635868
Log:
PIG-4250: Fix Security Risks found by Coverity
Modified:
pig/trunk/CHANGES.txt
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java
pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java
pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseBinaryConverter.java
pig/trunk/src/org/apache/pig/builtin/AvroStorage.java
pig/trunk/src/org/apache/pig/builtin/Bloom.java
pig/trunk/src/org/apache/pig/builtin/TextLoader.java
pig/trunk/src/org/apache/pig/data/DistinctDataBag.java
pig/trunk/src/org/apache/pig/data/InternalDistinctBag.java
pig/trunk/src/org/apache/pig/data/InternalSortedBag.java
pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java
pig/trunk/src/org/apache/pig/data/SortedDataBag.java
pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java
pig/trunk/src/org/apache/pig/impl/util/JarManager.java
pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java
pig/trunk/src/org/apache/pig/scripting/Pig.java
pig/trunk/src/org/apache/pig/scripting/jython/JythonScriptEngine.java
pig/trunk/src/org/apache/pig/scripting/streaming/python/PythonScriptEngine.java
pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java
Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Fri Oct 31 21:44:35 2014
@@ -110,6 +110,8 @@ OPTIMIZATIONS
BUG FIXES
+PIG-4250: Fix Security Risks found by Coverity (daijy)
+
PIG-4258: Fix several e2e tests on Windows (daijy)
PIG-4256: Fix StreamingPythonUDFs e2e test failure on Windows (daijy)
Modified: pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/AccumuloBinaryConverter.java Fri Oct 31 21:44:35 2014
@@ -172,9 +172,12 @@ public class AccumuloBinaryConverter imp
len = len ^ 0x80000000;
- dos.writeInt(len);
- dos.write(bytes);
- dos.close();
+ try {
+ dos.writeInt(len);
+ dos.write(bytes);
+ } finally {
+ dos.close();
+ }
return ret;
}
Modified: pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/accumulo/Utils.java Fri Oct 31 21:44:35 2014
@@ -297,8 +297,6 @@ public class Utils {
zos.write(arr, 0, read);
read = is.read(arr);
}
- is.close();
- zos.closeEntry();
}
public static void jarDir(File dir, String relativePath, ZipOutputStream zos)
@@ -317,7 +315,16 @@ public class Utils {
zos.closeEntry();
} else {
InputStream is = new FileInputStream(manifestFile);
- copyToZipStream(is, manifestEntry, zos);
+ try {
+ copyToZipStream(is, manifestEntry, zos);
+ } finally {
+ if (is != null) {
+ is.close();
+ }
+ if (zos != null) {
+ zos.closeEntry();
+ }
+ }
}
zos.closeEntry();
zipDir(dir, relativePath, zos, true);
@@ -345,7 +352,16 @@ public class Utils {
if (!path.equals(JarFile.MANIFEST_NAME)) {
ZipEntry anEntry = new ZipEntry(path);
InputStream is = new FileInputStream(f);
- copyToZipStream(is, anEntry, zos);
+ try {
+ copyToZipStream(is, anEntry, zos);
+ } finally {
+ if (is != null) {
+ is.close();
+ }
+ if (zos != null) {
+ zos.closeEntry();
+ }
+ }
}
}
}
Modified: pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseBinaryConverter.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseBinaryConverter.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseBinaryConverter.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/hbase/HBaseBinaryConverter.java Fri Oct 31 21:44:35 2014
@@ -93,7 +93,7 @@ public class HBaseBinaryConverter implem
@Override
public Map<String, Object> bytesToMap(byte[] b, ResourceFieldSchema fieldSchema) throws IOException {
- return bytesToMap(b, fieldSchema);
+ throw new ExecException("Can't generate a Map from byte[]");
}
/**
Modified: pig/trunk/src/org/apache/pig/builtin/AvroStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/AvroStorage.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/AvroStorage.java (original)
+++ pig/trunk/src/org/apache/pig/builtin/AvroStorage.java Fri Oct 31 21:44:35 2014
@@ -596,7 +596,11 @@ public class AvroStorage extends LoadFun
} else {
rr = new AvroRecordReader(s);
}
- rr.initialize(is, tc);
+ try {
+ rr.initialize(is, tc);
+ } finally {
+ rr.close();
+ }
tc.setStatus(is.toString());
return rr;
}
Modified: pig/trunk/src/org/apache/pig/builtin/Bloom.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/Bloom.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/Bloom.java (original)
+++ pig/trunk/src/org/apache/pig/builtin/Bloom.java Fri Oct 31 21:44:35 2014
@@ -105,7 +105,12 @@ public class Bloom extends FilterFunc {
});
String dcFile = dir + "/" + partFiles[0];
- filter.readFields(new DataInputStream(new FileInputStream(dcFile)));
+ DataInputStream dis = new DataInputStream(new FileInputStream(dcFile));
+ try {
+ filter.readFields(dis);
+ } finally {
+ dis.close();
+ }
}
/**
Modified: pig/trunk/src/org/apache/pig/builtin/TextLoader.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/TextLoader.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/TextLoader.java (original)
+++ pig/trunk/src/org/apache/pig/builtin/TextLoader.java Fri Oct 31 21:44:35 2014
@@ -148,7 +148,9 @@ public class TextLoader extends LoadFunc
@Override
public Map<String, Object> bytesToMap(byte[] b, ResourceFieldSchema schema) throws IOException {
- return bytesToMap(b, schema);
+ int errCode = 2109;
+ String msg = "TextLoader does not support conversion to Map.";
+ throw new ExecException(msg, errCode, PigException.BUG);
}
/**
Modified: pig/trunk/src/org/apache/pig/data/DistinctDataBag.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/data/DistinctDataBag.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/data/DistinctDataBag.java (original)
+++ pig/trunk/src/org/apache/pig/data/DistinctDataBag.java Fri Oct 31 21:44:35 2014
@@ -516,19 +516,27 @@ public class DistinctDataBag extends Def
// the spill files list. So I need to append it to my
// linked list as well so that it's still there when I
// move my linked list back to the spill files.
+ DataOutputStream out = null;
try {
- DataOutputStream out = getSpillFile();
+ out = getSpillFile();
ll.add(mSpillFiles.get(mSpillFiles.size() - 1));
Tuple t;
while ((t = readFromTree()) != null) {
t.write(out);
}
out.flush();
- out.close();
} catch (IOException ioe) {
String msg = "Unable to find our spill file.";
log.fatal(msg, ioe);
throw new RuntimeException(msg, ioe);
+ } finally {
+ if (out != null) {
+ try {
+ out.close();
+ } catch (IOException e) {
+ warn("Error closing spill", PigWarning.UNABLE_TO_CLOSE_SPILL_FILE, e);
+ }
+ }
}
}
// delete files that have been merged into new files
Modified: pig/trunk/src/org/apache/pig/data/InternalDistinctBag.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/data/InternalDistinctBag.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/data/InternalDistinctBag.java (original)
+++ pig/trunk/src/org/apache/pig/data/InternalDistinctBag.java Fri Oct 31 21:44:35 2014
@@ -36,6 +36,7 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.pig.PigConfiguration;
+import org.apache.pig.PigWarning;
import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
import org.apache.pig.classification.InterfaceAudience;
import org.apache.pig.classification.InterfaceStability;
@@ -424,19 +425,25 @@ public class InternalDistinctBag extends
// the spill files list. So I need to append it to my
// linked list as well so that it's still there when I
// move my linked list back to the spill files.
+ DataOutputStream out = null;
try {
- DataOutputStream out = getSpillFile();
+ out = getSpillFile();
ll.add(mSpillFiles.get(mSpillFiles.size() - 1));
Tuple t;
while ((t = readFromTree()) != null) {
t.write(out);
}
out.flush();
- out.close();
} catch (IOException ioe) {
String msg = "Unable to find our spill file.";
log.fatal(msg, ioe);
throw new RuntimeException(msg, ioe);
+ } finally {
+ try {
+ out.close();
+ } catch (IOException e) {
+ warn("Error closing spill", PigWarning.UNABLE_TO_CLOSE_SPILL_FILE, e);
+ }
}
}
Modified: pig/trunk/src/org/apache/pig/data/InternalSortedBag.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/data/InternalSortedBag.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/data/InternalSortedBag.java (original)
+++ pig/trunk/src/org/apache/pig/data/InternalSortedBag.java Fri Oct 31 21:44:35 2014
@@ -35,6 +35,7 @@ import java.util.PriorityQueue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigWarning;
/**
@@ -401,19 +402,27 @@ public class InternalSortedBag extends S
// the spill files list. So I need to append it to my
// linked list as well so that it's still there when I
// move my linked list back to the spill files.
+ DataOutputStream out = null;
try {
- DataOutputStream out = getSpillFile();
+ out = getSpillFile();
ll.add(mSpillFiles.get(mSpillFiles.size() - 1));
Tuple t;
while ((t = readFromPriorityQ()) != null) {
t.write(out);
}
out.flush();
- out.close();
} catch (IOException ioe) {
String msg = "Unable to find our spill file.";
log.fatal(msg, ioe);
throw new RuntimeException(msg, ioe);
+ } finally {
+ if (out != null) {
+ try {
+ out.close();
+ } catch (IOException e) {
+ warn("Error closing spill", PigWarning.UNABLE_TO_CLOSE_SPILL_FILE, e);
+ }
+ }
}
}
// delete files that have been merged into new files
Modified: pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java (original)
+++ pig/trunk/src/org/apache/pig/data/SchemaTupleBackend.java Fri Oct 31 21:44:35 2014
@@ -184,14 +184,22 @@ public class SchemaTupleBackend {
LOG.info("Attempting to read file: " + s);
// The string is the symlink into the distributed cache
File src = new File(s);
- FileInputStream fin = new FileInputStream(src);
- FileOutputStream fos = new FileOutputStream(new File(codeDir, s));
-
- fin.getChannel().transferTo(0, src.length(), fos.getChannel());
-
- fin.close();
- fos.close();
- LOG.info("Successfully copied file to local directory.");
+ FileInputStream fin = null;
+ FileOutputStream fos = null;
+ try {
+ fin = new FileInputStream(src);
+ fos = new FileOutputStream(new File(codeDir, s));
+
+ fin.getChannel().transferTo(0, src.length(), fos.getChannel());
+ LOG.info("Successfully copied file to local directory.");
+ } finally {
+ if (fin != null) {
+ fin.close();
+ }
+ if (fos != null) {
+ fos.close();
+ }
+ }
}
}
Modified: pig/trunk/src/org/apache/pig/data/SortedDataBag.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/data/SortedDataBag.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/data/SortedDataBag.java (original)
+++ pig/trunk/src/org/apache/pig/data/SortedDataBag.java Fri Oct 31 21:44:35 2014
@@ -491,19 +491,25 @@ public class SortedDataBag extends Defau
// the spill files list. So I need to append it to my
// linked list as well so that it's still there when I
// move my linked list back to the spill files.
+ DataOutputStream out = null;
try {
- DataOutputStream out = getSpillFile();
+ out = getSpillFile();
ll.add(mSpillFiles.get(mSpillFiles.size() - 1));
Tuple t;
while ((t = readFromPriorityQ()) != null) {
t.write(out);
}
out.flush();
- out.close();
} catch (IOException ioe) {
String msg = "Unable to find our spill file.";
log.fatal(msg, ioe);
throw new RuntimeException(msg, ioe);
+ } finally {
+ try {
+ out.close();
+ } catch (IOException e) {
+ warn("Error closing spill", PigWarning.UNABLE_TO_CLOSE_SPILL_FILE, e);
+ }
}
}
// delete files that have been merged into new files
Modified: pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java (original)
+++ pig/trunk/src/org/apache/pig/impl/io/FileLocalizer.java Fri Oct 31 21:44:35 2014
@@ -862,14 +862,20 @@ public class FileLocalizer {
dest.getParentFile().mkdirs();
dest.deleteOnExit();
- OutputStream outputStream = new BufferedOutputStream(new FileOutputStream(dest));
- byte[] buffer = new byte[1024];
- int len;
- while ((len=resourceStream.read(buffer)) > 0) {
- outputStream.write(buffer,0,len);
+ OutputStream outputStream = null;
+ try {
+ outputStream = new BufferedOutputStream(new FileOutputStream(dest));
+ byte[] buffer = new byte[1024];
+ int len;
+ while ((len=resourceStream.read(buffer)) > 0) {
+ outputStream.write(buffer,0,len);
+ }
+ } finally {
+ resourceStream.close();
+ if (outputStream != null) {
+ outputStream.close();
+ }
}
- outputStream.close();
-
localFileRet = new FetchFileRet(dest,false);
}
else
Modified: pig/trunk/src/org/apache/pig/impl/util/JarManager.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/impl/util/JarManager.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/impl/util/JarManager.java (original)
+++ pig/trunk/src/org/apache/pig/impl/util/JarManager.java Fri Oct 31 21:44:35 2014
@@ -90,9 +90,16 @@ public class JarManager {
createPigScriptUDFJar(fos, pigContext, contents);
if (!contents.isEmpty()) {
- FileInputStream fis = new FileInputStream(scriptUDFJarFile);
- String md5 = org.apache.commons.codec.digest.DigestUtils.md5Hex(fis);
- fis.close();
+ FileInputStream fis = null;
+ String md5 = null;
+ try {
+ fis = new FileInputStream(scriptUDFJarFile);
+ md5 = org.apache.commons.codec.digest.DigestUtils.md5Hex(fis);
+ } finally {
+ if (fis != null) {
+ fis.close();
+ }
+ }
File newScriptUDFJarFile = new File(scriptUDFJarFile.getParent(), "PigScriptUDF-" + md5 + ".jar");
scriptUDFJarFile.renameTo(newScriptUDFJarFile);
return newScriptUDFJarFile;
@@ -114,7 +121,11 @@ public class JarManager {
if (stream==null) {
throw new IOException("Cannot find " + path);
}
- addStream(jarOutputStream, path, stream, contents, inputFile.lastModified());
+ try {
+ addStream(jarOutputStream, path, stream, contents, inputFile.lastModified());
+ } finally {
+ stream.close();
+ }
}
for (Map.Entry<String, File> entry : pigContext.getScriptFiles().entrySet()) {
log.debug("Adding entry " + entry.getKey() + " to job jar" );
@@ -127,7 +138,11 @@ public class JarManager {
if (stream==null) {
throw new IOException("Cannot find " + entry.getValue().getPath());
}
- addStream(jarOutputStream, entry.getKey(), stream, contents, entry.getValue().lastModified());
+ try {
+ addStream(jarOutputStream, entry.getKey(), stream, contents, entry.getValue().lastModified());
+ } finally {
+ stream.close();
+ }
}
if (!contents.isEmpty()) {
jarOutputStream.close();
Modified: pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java Fri Oct 31 21:44:35 2014
@@ -574,6 +574,7 @@ public class QueryParserDriver {
String macroText = null;
try {
+ in.close();
in = new BufferedReader(new StringReader(sb.toString()));
macroText = pigContext.doParamSubstitution(in);
} catch (IOException e) {
Modified: pig/trunk/src/org/apache/pig/scripting/Pig.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/scripting/Pig.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/scripting/Pig.java (original)
+++ pig/trunk/src/org/apache/pig/scripting/Pig.java Fri Oct 31 21:44:35 2014
@@ -349,13 +349,16 @@ public class Pig {
private static String getScriptFromFile(String filename) throws IOException {
LineNumberReader rd = new LineNumberReader(new FileReader(filename));
StringBuilder sb = new StringBuilder();
- String line = rd.readLine();
- while (line != null) {
- sb.append(line);
- sb.append("\n");
- line = rd.readLine();
+ try {
+ String line = rd.readLine();
+ while (line != null) {
+ sb.append(line);
+ sb.append("\n");
+ line = rd.readLine();
+ }
+ } finally {
+ rd.close();
}
- rd.close();
return sb.toString();
}
Modified: pig/trunk/src/org/apache/pig/scripting/jython/JythonScriptEngine.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/scripting/jython/JythonScriptEngine.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/scripting/jython/JythonScriptEngine.java (original)
+++ pig/trunk/src/org/apache/pig/scripting/jython/JythonScriptEngine.java Fri Oct 31 21:44:35 2014
@@ -409,13 +409,15 @@ public class JythonScriptEngine extends
throw new IOException("Can't read file: " + scriptFile);
}
- // TODO: fis1 is not closed
FileInputStream fis1 = new FileInputStream(scriptFile);
- if (hasFunction(fis1)) {
- registerFunctions(scriptFile, null, pigContext);
+ try {
+ if (hasFunction(fis1)) {
+ registerFunctions(scriptFile, null, pigContext);
+ }
+ } finally {
+ fis1.close();
}
-
Interpreter.setMain(true);
FileInputStream fis = new FileInputStream(scriptFile);
try {
Modified: pig/trunk/src/org/apache/pig/scripting/streaming/python/PythonScriptEngine.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/scripting/streaming/python/PythonScriptEngine.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/scripting/streaming/python/PythonScriptEngine.java (original)
+++ pig/trunk/src/org/apache/pig/scripting/streaming/python/PythonScriptEngine.java Fri Oct 31 21:44:35 2014
@@ -56,7 +56,12 @@ public class PythonScriptEngine extends
}
FileInputStream fin = new FileInputStream(f);
- List<String[]> functions = getFunctions(fin);
+ List<String[]> functions = null;
+ try {
+ functions = getFunctions(fin);
+ } finally {
+ fin.close();
+ }
namespace = namespace == null ? "" : namespace + NAMESPACE_SEPARATOR;
for(String[] functionInfo : functions) {
String name = functionInfo[0];
@@ -75,7 +80,6 @@ public class PythonScriptEngine extends
execType, isIllustrate
}));
}
- fin.close();
}
@Override
Modified: pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java?rev=1635868&r1=1635867&r2=1635868&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java (original)
+++ pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java Fri Oct 31 21:44:35 2014
@@ -45,6 +45,7 @@ import java.util.Set;
import jline.ConsoleReader;
import jline.ConsoleReaderInputStream;
+import org.apache.commons.io.output.NullOutputStream;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
@@ -390,26 +391,9 @@ public class GruntParser extends PigScri
explainCurrentBatch(false);
}
- /**
- * A {@link PrintStream} implementation which does not write anything
- * Used with '-check' command line option to pig Main
- * (through {@link GruntParser#explainCurrentBatch(boolean) } )
- */
- static class NullPrintStream extends PrintStream {
- public NullPrintStream(String fileName) throws FileNotFoundException {
- super(fileName);
- }
- @Override
- public void write(byte[] buf, int off, int len) {}
- @Override
- public void write(int b) {}
- @Override
- public void write(byte [] b) {}
- }
-
protected void explainCurrentBatch(boolean dontPrintOutput) throws IOException {
- PrintStream lp = (dontPrintOutput) ? new NullPrintStream("dummy") : System.out;
- PrintStream ep = (dontPrintOutput) ? new NullPrintStream("dummy") : System.out;
+ PrintStream lp = (dontPrintOutput) ? new PrintStream(new NullOutputStream()) : System.out;
+ PrintStream ep = (dontPrintOutput) ? new PrintStream(new NullOutputStream()) : System.out;
if (!(mExplain.mLast && mExplain.mCount == 0)) {
if (mPigServer.isBatchEmpty()) {
@@ -489,7 +473,9 @@ public class GruntParser extends PigScri
PigContext context = mPigServer.getPigContext();
BufferedReader reader = new BufferedReader(new FileReader(scriptPath));
- return context.doParamSubstitution(reader, params, paramFiles);
+ String result = context.doParamSubstitution(reader, params, paramFiles);
+ reader.close();
+ return result;
}
@Override