Author: prkommireddi
Date: Fri Mar 7 07:12:59 2014
New Revision: 1575200
URL: http://svn.apache.org/r1575200
Log:
PIG-3765: Ability to disable Pig commands and operators (prkommireddi)
Added:
pig/trunk/src/org/apache/pig/newplan/logical/rules/LogicalRelationalNodeValidator.java
pig/trunk/src/org/apache/pig/validator/
pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistFilter.java
pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistValidator.java
pig/trunk/src/org/apache/pig/validator/PigCommandFilter.java
pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java
Modified:
pig/trunk/CHANGES.txt
pig/trunk/conf/pig.properties
pig/trunk/src/org/apache/pig/PigConfiguration.java
pig/trunk/src/org/apache/pig/PigServer.java
pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java
pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java
pig/trunk/src/org/apache/pig/scripting/BoundScript.java
pig/trunk/src/org/apache/pig/tools/ToolsPigServer.java
pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java
pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java
pig/trunk/test/org/apache/pig/pigunit/pig/GruntParser.java
pig/trunk/test/org/apache/pig/pigunit/pig/PigServer.java
pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java
pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java
pig/trunk/test/org/apache/pig/test/TestPigServer.java
pig/trunk/test/org/apache/pig/test/Util.java
pig/trunk/test/org/apache/pig/test/pigunit/pig/TestGruntParser.java
pig/trunk/test/org/apache/pig/tools/grunt/TestGruntParser.java
Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Fri Mar 7 07:12:59 2014
@@ -30,6 +30,8 @@ PIG-2207: Support custom counters for ag
IMPROVEMENTS
+PIG-3765: Ability to disable Pig commands and operators (prkommireddi)
+
PIG-3731: Ability to specify local-mode specific configuration (useful for local/auto-local mode) (aniket486)
PIG-3793: Provide info on number of LogicalRelationalOperator(s) used in the script through LogicalPlanData (prkommireddi)
Modified: pig/trunk/conf/pig.properties
URL: http://svn.apache.org/viewvc/pig/trunk/conf/pig.properties?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/conf/pig.properties (original)
+++ pig/trunk/conf/pig.properties Fri Mar 7 07:12:59 2014
@@ -259,3 +259,18 @@ pig.location.check.strict=false
# Additional jar will be cached under PIG_USER_CACHE_LOCATION/${user.name}/.pigcache
# and will be re-used across the jobs run by the user if the jar has not changed.
# pig.user.cache.location=/tmp
+
+####################################################################################
+
+# Comma-delimited entries of commands/operators that must be disallowed. This is a
+# security feature to be used by administrators to block use of commands by users.
+# For eg, an admin might like to block all filesystem commands and setting configs
+# in pig script. In which case, the entry would be
+# pig.blacklist=fs,set
+
+# Comma-delimited entries of commands/operators that must be allowed. This is a
+# security feature to be used by administrators to block use of commands by users
+# that are not a part of the whitelist.
+# For eg, an admin might like to allow only LOAD, STORE, FILTER, GROUP
+# in pig script. In which case, the entry would be
+# pig.whitelist=load,store,filter,group
Modified: pig/trunk/src/org/apache/pig/PigConfiguration.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/PigConfiguration.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/PigConfiguration.java (original)
+++ pig/trunk/src/org/apache/pig/PigConfiguration.java Fri Mar 7 07:12:59 2014
@@ -182,5 +182,22 @@ public class PigConfiguration {
* and will be re-used across the jobs run by the user if the jar has not changed
*/
public static final String PIG_USER_CACHE_LOCATION = "pig.user.cache.location";
-}
+ /**
+ * Comma-delimited entries of commands/operators that must be disallowed.
+ * This is a security feature to be used by administrators to block use of
+ * commands by users. For eg, an admin might like to block all filesystem
+ * commands and setting configs in pig script. In which case, the entry
+ * would be "pig.blacklist=fs,set"
+ */
+ public static final String PIG_BLACKLIST = "pig.blacklist";
+
+ /**
+ * Comma-delimited entries of commands/operators that must be allowed. This
+ * is a security feature to be used by administrators to block use of
+ * commands by users that are not a part of the whitelist. For eg, an admin
+ * might like to allow only LOAD, STORE, FILTER, GROUP in pig script. In
+ * which case, the entry would be "pig.whitelist=load,store,filter,group"
+ */
+ public static final String PIG_WHITELIST = "pig.whitelist";
+}
Modified: pig/trunk/src/org/apache/pig/PigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/PigServer.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/PigServer.java (original)
+++ pig/trunk/src/org/apache/pig/PigServer.java Fri Mar 7 07:12:59 2014
@@ -105,6 +105,9 @@ import org.apache.pig.tools.pigstats.Pig
import org.apache.pig.tools.pigstats.PigStats.JobGraph;
import org.apache.pig.tools.pigstats.ScriptState;
import org.apache.pig.tools.pigstats.SimpleFetchPigStats;
+import org.apache.pig.validator.BlackAndWhitelistFilter;
+import org.apache.pig.validator.BlackAndWhitelistValidator;
+import org.apache.pig.validator.PigCommandFilter;
/**
*
@@ -156,6 +159,8 @@ public class PigServer {
private boolean validateEachStatement = false;
private boolean skipParseInRegisterForBatch = false;
+ private final BlackAndWhitelistFilter filter;
+
private String constructScope() {
// scope servers for now as a session id
@@ -239,6 +244,8 @@ public class PigServer {
if (ScriptState.get() == null) {
ScriptState.start(pigContext.getExecutionEngine().instantiateScriptState());
}
+
+ this.filter = new BlackAndWhitelistFilter(this);
}
private void addJarsFromProperties() throws ExecException {
@@ -529,6 +536,9 @@ public class PigServer {
* @throws IOException
*/
public void registerJar(String name) throws IOException {
+ // Check if this operation is permitted
+ filter.validate(PigCommandFilter.Command.REGISTER);
+
if (pigContext.hasJar(name)) {
log.debug("Ignoring duplicate registration for jar " + name);
return;
@@ -683,9 +693,8 @@ public class PigServer {
public void registerScript(InputStream in, Map<String,String> params,List<String> paramsFiles) throws IOException {
try {
String substituted = pigContext.doParamSubstitution(in, paramMapToList(params), paramsFiles);
- GruntParser grunt = new GruntParser(new StringReader(substituted));
+ GruntParser grunt = new GruntParser(new StringReader(substituted), this);
grunt.setInteractive(false);
- grunt.setParams(this);
grunt.parseStopOnError(true);
} catch (org.apache.pig.tools.pigscript.parser.ParseException e) {
log.error(e.getLocalizedMessage());
@@ -1164,10 +1173,14 @@ public class PigServer {
* @throws IOException
*/
public boolean deleteFile(String filename) throws IOException {
+ // Check if this operation is permitted
+ filter.validate(PigCommandFilter.Command.RM);
+ filter.validate(PigCommandFilter.Command.RMF);
+
ElementDescriptor elem = pigContext.getDfs().asElement(filename);
elem.delete();
return true;
- }
+ }
/**
* Rename a file.
@@ -1177,6 +1190,9 @@ public class PigServer {
* @throws IOException
*/
public boolean renameFile(String source, String target) throws IOException {
+ // Check if this operation is permitted
+ filter.validate(PigCommandFilter.Command.MV);
+
pigContext.rename(source, target);
return true;
}
@@ -1188,6 +1204,9 @@ public class PigServer {
* @throws IOException
*/
public boolean mkdirs(String dirs) throws IOException {
+ // Check if this operation is permitted
+ filter.validate(PigCommandFilter.Command.MKDIR);
+
ContainerDescriptor container = pigContext.getDfs().asContainer(dirs);
container.create();
return true;
@@ -1200,6 +1219,9 @@ public class PigServer {
* @throws IOException
*/
public String[] listPaths(String dir) throws IOException {
+ // Check if this operation is permitted
+ filter.validate(PigCommandFilter.Command.LS);
+
Collection<String> allPaths = new ArrayList<String>();
ContainerDescriptor container = pigContext.getDfs().asContainer(dir);
Iterator<ElementDescriptor> iter = container.iterator();
@@ -1336,9 +1358,13 @@ public class PigServer {
return stats;
}
- private PigStats executeCompiledLogicalPlan() throws ExecException, FrontendException {
+ private PigStats executeCompiledLogicalPlan() throws ExecException,
+ FrontendException {
// discover pig features used in this script
- ScriptState.get().setScriptFeatures( currDAG.lp );
+ ScriptState.get().setScriptFeatures(currDAG.lp);
+
+ BlackAndWhitelistValidator validator = new BlackAndWhitelistValidator(getPigContext(), currDAG.lp);
+ validator.validate();
return launchPlan(currDAG.lp, "job_pigexec_");
}
Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java Fri Mar 7 07:12:59 2014
@@ -64,10 +64,12 @@ import org.apache.pig.newplan.logical.re
import org.apache.pig.newplan.logical.relational.LogicalPlan;
import org.apache.pig.newplan.logical.relational.LogicalRelationalOperator;
import org.apache.pig.newplan.logical.rules.InputOutputFileValidator;
+import org.apache.pig.newplan.logical.rules.LogicalRelationalNodeValidator;
import org.apache.pig.newplan.logical.visitor.SortInfoSetter;
import org.apache.pig.newplan.logical.visitor.StoreAliasSetter;
import org.apache.pig.pen.POOptimizeDisabler;
import org.apache.pig.tools.pigstats.PigStats;
+import org.apache.pig.validator.BlackAndWhitelistValidator;
import com.google.common.base.Splitter;
import com.google.common.collect.Lists;
@@ -308,9 +310,15 @@ public abstract class HExecutionEngine i
// Validate input/output file. Currently no validation framework in
// new logical plan, put this validator here first.
// We might decide to move it out to a validator framework in future
- InputOutputFileValidator validator = new InputOutputFileValidator(
+ LogicalRelationalNodeValidator validator = new InputOutputFileValidator(
plan, pigContext);
validator.validate();
+
+ // Check for blacklist and whitelist properties and disable
+ // commands/operators accordingly. Note if a user does not
+ // specify these, Pig will work without any filters or validations
+ validator = new BlackAndWhitelistValidator(pigContext, plan);
+ validator.validate();
}
// translate new logical plan to physical plan
Modified: pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java (original)
+++ pig/trunk/src/org/apache/pig/newplan/logical/rules/InputOutputFileValidator.java Fri Mar 7 07:12:59 2014
@@ -34,7 +34,7 @@ import org.apache.pig.newplan.OperatorPl
import org.apache.pig.newplan.logical.relational.LOStore;
import org.apache.pig.newplan.logical.relational.LogicalRelationalNodesVisitor;
-public class InputOutputFileValidator {
+public class InputOutputFileValidator implements LogicalRelationalNodeValidator {
private PigContext pigCtx;
OperatorPlan plan;
public InputOutputFileValidator(OperatorPlan plan, PigContext pigContext) {
Added: pig/trunk/src/org/apache/pig/newplan/logical/rules/LogicalRelationalNodeValidator.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/rules/LogicalRelationalNodeValidator.java?rev=1575200&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/rules/LogicalRelationalNodeValidator.java (added)
+++ pig/trunk/src/org/apache/pig/newplan/logical/rules/LogicalRelationalNodeValidator.java Fri Mar 7 07:12:59 2014
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.newplan.logical.rules;
+
+import org.apache.pig.backend.hadoop.executionengine.HExecutionEngine;
+import org.apache.pig.classification.InterfaceAudience;
+import org.apache.pig.classification.InterfaceStability;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+
+/**
+ *
+ * Implementors of this interface would define validations based on logical
+ * operators within a Pig script. The validations could be called from
+ * {@link HExecutionEngine#compile(org.apache.pig.newplan.logical.relational.LogicalPlan, java.util.Properties)}
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface LogicalRelationalNodeValidator {
+
+ /**
+ * Validates logical operators as defined in the logical plan of a pig
+ * script.
+ *
+ * @throws FrontendException
+ */
+ public void validate() throws FrontendException;
+}
Modified: pig/trunk/src/org/apache/pig/scripting/BoundScript.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/scripting/BoundScript.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/scripting/BoundScript.java (original)
+++ pig/trunk/src/org/apache/pig/scripting/BoundScript.java Fri Mar 7 07:12:59 2014
@@ -272,9 +272,8 @@ public class BoundScript {
}
PigServer pigServer = new PigServer(scriptContext.getPigContext(), false);
pigServer.setBatchOn();
- GruntParser grunt = new GruntParser(new StringReader(query));
+ GruntParser grunt = new GruntParser(new StringReader(query), pigServer);
grunt.setInteractive(false);
- grunt.setParams(pigServer);
try {
grunt.parseStopOnError(true);
} catch (ParseException e) {
@@ -283,13 +282,12 @@ public class BoundScript {
pigServer.executeBatch();
return PigStats.get();
}
-
- private void registerQuery(PigServer pigServer, String pl) throws IOException {
- GruntParser grunt = new GruntParser(new StringReader(pl));
+
+ private void registerQuery(PigServer pigServer, String pl) throws IOException {
+ GruntParser grunt = new GruntParser(new StringReader(pl), pigServer);
grunt.setInteractive(false);
- grunt.setParams(pigServer);
pigServer.setBatchOn();
- try {
+ try {
grunt.parseStopOnError(true);
} catch (ParseException e) {
throw new IOException("Failed to parse query: " + pl, e);
@@ -338,9 +336,8 @@ public class BoundScript {
ScriptState.get().registerListener(adaptor);
PigServer pigServer = new PigServer(ctx, true);
pigServer.setBatchOn();
- GruntParser grunt = new GruntParser(new StringReader(query));
+ GruntParser grunt = new GruntParser(new StringReader(query), pigServer);
grunt.setInteractive(false);
- grunt.setParams(pigServer);
try {
grunt.parseStopOnError(true);
} catch (ParseException e) {
@@ -350,5 +347,5 @@ public class BoundScript {
return PigStats.get();
}
}
-
+
}
Modified: pig/trunk/src/org/apache/pig/tools/ToolsPigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/tools/ToolsPigServer.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/ToolsPigServer.java (original)
+++ pig/trunk/src/org/apache/pig/tools/ToolsPigServer.java Fri Mar 7 07:12:59 2014
@@ -113,9 +113,8 @@ public class ToolsPigServer extends PigS
// Parse in grunt so that register commands are recognized
try {
- GruntParser grunt = new GruntParser(new StringReader(substituted));
+ GruntParser grunt = new GruntParser(new StringReader(substituted), this);
grunt.setInteractive(false);
- grunt.setParams(this);
setBatchOn();
//grunt.setLoadOnly(true);
grunt.parseOnly();
Modified: pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java (original)
+++ pig/trunk/src/org/apache/pig/tools/grunt/Grunt.java Fri Mar 7 07:12:59 2014
@@ -41,15 +41,12 @@ public class Grunt
PigServer pig;
GruntParser parser;
- public Grunt(BufferedReader in, PigContext pigContext) throws ExecException
- {
+ public Grunt(BufferedReader in, PigContext pigContext) throws ExecException {
this.in = in;
this.pig = new PigServer(pigContext);
-
- if (in != null)
- {
- parser = new GruntParser(in);
- parser.setParams(pig);
+
+ if (in != null) {
+ parser = new GruntParser(in, pig);
}
}
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=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java (original)
+++ pig/trunk/src/org/apache/pig/tools/grunt/GruntParser.java Fri Mar 7 07:12:59 2014
@@ -72,6 +72,8 @@ import org.apache.pig.tools.pigscript.pa
import org.apache.pig.tools.pigstats.JobStats;
import org.apache.pig.tools.pigstats.PigStats;
import org.apache.pig.tools.pigstats.PigStats.JobGraph;
+import org.apache.pig.validator.BlackAndWhitelistFilter;
+import org.apache.pig.validator.PigCommandFilter;
import org.fusesource.jansi.Ansi;
import org.fusesource.jansi.AnsiConsole;
import org.python.google.common.collect.Lists;
@@ -79,14 +81,26 @@ import org.python.google.common.collect.
public class GruntParser extends PigScriptParser {
private static final Log log = LogFactory.getLog(GruntParser.class);
+ private PigCommandFilter filter;
- public GruntParser(Reader stream) {
- super(stream);
+ public GruntParser(Reader reader) {
+ this(reader, null);
+ init();
+ }
+
+ public GruntParser(Reader reader, PigServer pigServer) {
+ super(reader);
+ mPigServer = pigServer;
init();
}
public GruntParser(InputStream stream, String encoding) {
+ this(stream, encoding, null);
+ }
+
+ public GruntParser(InputStream stream, String encoding, PigServer pigServer) {
super(stream, encoding);
+ mPigServer = pigServer;
init();
}
@@ -95,8 +109,19 @@ public class GruntParser extends PigScri
init();
}
+ public GruntParser(InputStream stream, PigServer pigServer) {
+ super(stream);
+ mPigServer = pigServer;
+ init();
+ }
+
public GruntParser(PigScriptParserTokenManager tm) {
+ this(tm, null);
+ }
+
+ public GruntParser(PigScriptParserTokenManager tm, PigServer pigServer) {
super(tm);
+ mPigServer = pigServer;
init();
}
@@ -105,12 +130,23 @@ public class GruntParser extends PigScri
mLoadOnly = false;
mExplain = null;
mScriptIllustrate = false;
+
+ setProps();
+
+ filter = new BlackAndWhitelistFilter(mPigServer);
+ }
+
+ private void setProps() {
+ mDfs = mPigServer.getPigContext().getDfs();
+ mLfs = mPigServer.getPigContext().getLfs();
+ mConf = mPigServer.getPigContext().getProperties();
+ shell = new FsShell(ConfigurationUtil.toConfiguration(mConf));
}
@Override
- public void setInteractive(boolean isInteractive){
+ public void setInteractive(boolean isInteractive) {
super.setInteractive(isInteractive);
- if(isInteractive){
+ if(isInteractive) {
setValidateEachStatement(true);
}
}
@@ -216,16 +252,6 @@ public class GruntParser extends PigScri
mLoadOnly = loadOnly;
}
- public void setParams(PigServer pigServer)
- {
- mPigServer = pigServer;
-
- mDfs = mPigServer.getPigContext().getDfs();
- mLfs = mPigServer.getPigContext().getLfs();
- mConf = mPigServer.getPigContext().getProperties();
- shell = new FsShell(ConfigurationUtil.toConfiguration(mConf));
- }
-
public void setScriptIllustrate() {
mScriptIllustrate = true;
}
@@ -442,11 +468,13 @@ public class GruntParser extends PigScri
@Override
protected void processRegister(String jar) throws IOException {
+ filter.validate(PigCommandFilter.Command.REGISTER);
mPigServer.registerJar(jar);
}
@Override
protected void processRegister(String path, String scriptingLang, String namespace) throws IOException, ParseException {
+ filter.validate(PigCommandFilter.Command.REGISTER);
if(path.endsWith(".jar")) {
if(scriptingLang != null || namespace != null) {
throw new ParseException("Cannot register a jar with a scripting language or namespace");
@@ -530,8 +558,7 @@ public class GruntParser extends PigScri
throw new ParseException("Cannot access file: " + script);
}
- GruntParser parser = new GruntParser(inputReader);
- parser.setParams(mPigServer);
+ GruntParser parser = new GruntParser(inputReader, mPigServer);
parser.setConsoleReader(reader);
parser.setInteractive(interactive);
parser.setLoadOnly(loadOnly);
@@ -551,6 +578,7 @@ public class GruntParser extends PigScri
@Override
protected void processSet(String key, String value) throws IOException, ParseException {
+ filter.validate(PigCommandFilter.Command.SET);
if (key.equals("debug"))
{
if (value.equals("on"))
@@ -590,21 +618,21 @@ public class GruntParser extends PigScri
mPigServer.getPigContext().getExecutionEngine().setProperty(key, value);
}
}
-
+
@Override
protected void processSet() throws IOException, ParseException {
+ filter.validate(PigCommandFilter.Command.SET);
Properties jobProps = mPigServer.getPigContext().getProperties();
Properties sysProps = System.getProperties();
-
+
List<String> jobPropsList = Lists.newArrayList();
List<String> sysPropsList = Lists.newArrayList();
for (Object key : jobProps.keySet()) {
- String propStr = key + "=" + jobProps.getProperty((String)key);
+ String propStr = key + "=" + jobProps.getProperty((String) key);
if (sysProps.containsKey(key)) {
sysPropsList.add("system: " + propStr);
- }
- else {
+ } else {
jobPropsList.add(propStr);
}
}
@@ -617,8 +645,8 @@ public class GruntParser extends PigScri
}
@Override
- protected void processCat(String path) throws IOException
- {
+ protected void processCat(String path) throws IOException {
+ filter.validate(PigCommandFilter.Command.CAT);
if(mExplain == null) { // process only if not in "explain" mode
executeBatch();
@@ -666,8 +694,8 @@ public class GruntParser extends PigScri
}
@Override
- protected void processCD(String path) throws IOException
- {
+ protected void processCD(String path) throws IOException {
+ filter.validate(PigCommandFilter.Command.CD);
ContainerDescriptor container;
if(mExplain == null) { // process only if not in "explain" mode
@@ -704,8 +732,8 @@ public class GruntParser extends PigScri
}
@Override
- protected void processDump(String alias) throws IOException
- {
+ protected void processDump(String alias) throws IOException {
+ filter.validate(PigCommandFilter.Command.DUMP);
if (alias == null) {
if (mPigServer.isBatchOn()) {
mPigServer.parseAndBuild();
@@ -740,6 +768,7 @@ public class GruntParser extends PigScri
@Override
protected void processIllustrate(String alias, String script, String target, List<String> params, List<String> files) throws IOException, ParseException
{
+ filter.validate(PigCommandFilter.Command.ILLUSTRATE);
if (mScriptIllustrate)
throw new ParseException("'illustrate' statement can not appear in a script that is illustrated opon.");
@@ -795,13 +824,15 @@ public class GruntParser extends PigScri
@Override
protected void processKill(String jobid) throws IOException
{
+ filter.validate(PigCommandFilter.Command.KILL);
mPigServer.getPigContext().getExecutionEngine().killJob(jobid);
}
@Override
- protected void processLS(String path) throws IOException
- {
- if(mExplain == null) { // process only if not in "explain" mode
+ protected void processLS(String path) throws IOException {
+ filter.validate(PigCommandFilter.Command.LS);
+
+ if (mExplain == null) { // process only if not in "explain" mode
executeBatch();
@@ -858,6 +889,7 @@ public class GruntParser extends PigScri
@Override
protected void processPWD() throws IOException
{
+ filter.validate(PigCommandFilter.Command.PWD);
if(mExplain == null) { // process only if not in "explain" mode
executeBatch();
@@ -923,6 +955,7 @@ public class GruntParser extends PigScri
@Override
protected void processMove(String src, String dst) throws IOException
{
+ filter.validate(PigCommandFilter.Command.MV);
if(mExplain == null) { // process only if not in "explain" mode
executeBatch();
@@ -948,6 +981,7 @@ public class GruntParser extends PigScri
@Override
protected void processCopy(String src, String dst) throws IOException
{
+ filter.validate(PigCommandFilter.Command.CP);
if(mExplain == null) { // process only if not in "explain" mode
executeBatch();
@@ -969,6 +1003,7 @@ public class GruntParser extends PigScri
@Override
protected void processCopyToLocal(String src, String dst) throws IOException
{
+ filter.validate(PigCommandFilter.Command.COPYTOLOCAL);
if(mExplain == null) { // process only if not in "explain" mode
executeBatch();
@@ -990,6 +1025,7 @@ public class GruntParser extends PigScri
@Override
protected void processCopyFromLocal(String src, String dst) throws IOException
{
+ filter.validate(PigCommandFilter.Command.COPYFROMLOCAL);
if(mExplain == null) { // process only if not in "explain" mode
executeBatch();
@@ -1011,6 +1047,7 @@ public class GruntParser extends PigScri
@Override
protected void processMkdir(String dir) throws IOException
{
+ filter.validate(PigCommandFilter.Command.MKDIR);
if(mExplain == null) { // process only if not in "explain" mode
executeBatch();
@@ -1039,11 +1076,12 @@ public class GruntParser extends PigScri
}
@Override
- protected void processRemove(String path, String options ) throws IOException
- {
+ protected void processRemove(String path, String options) throws IOException {
+ filter.validate(PigCommandFilter.Command.RM);
+ filter.validate(PigCommandFilter.Command.RMF);
int MAX_MS_TO_WAIT_FOR_FILE_DELETION = 10 * 60 * 1000;
int MS_TO_SLEEP_WHILE_WAITING_FOR_FILE_DELETION = 250;
-
+
if(mExplain == null) { // process only if not in "explain" mode
Path filePath = new Path(path);
ElementDescriptor dfsPath = null;
@@ -1083,7 +1121,8 @@ public class GruntParser extends PigScri
}
@Override
- protected void processFsCommand(String[] cmdTokens) throws IOException{
+ protected void processFsCommand(String[] cmdTokens) throws IOException {
+ filter.validate(PigCommandFilter.Command.FS);
if(mExplain == null) { // process only if not in "explain" mode
executeBatch();
@@ -1108,7 +1147,8 @@ public class GruntParser extends PigScri
}
@Override
- protected void processShCommand(String[] cmdTokens) throws IOException{
+ protected void processShCommand(String[] cmdTokens) throws IOException {
+ filter.validate(PigCommandFilter.Command.SH);
if(mExplain == null) { // process only if not in "explain" mode
try {
executeBatch();
Added: pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistFilter.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistFilter.java?rev=1575200&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistFilter.java (added)
+++ pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistFilter.java Fri Mar 7 07:12:59 2014
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.validator;
+
+import java.util.Set;
+
+import org.apache.pig.PigConfiguration;
+import org.apache.pig.PigServer;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.python.google.common.base.Splitter;
+import org.python.google.common.collect.Sets;
+
+/**
+ *
+ * This Filter handles black and whitelisting of Pig commands.
+ */
+public final class BlackAndWhitelistFilter implements PigCommandFilter {
+ private static final int BLACKANDWHITELIST_ERROR_CODE = 1856;
+ private static final Splitter SPLITTER = Splitter.on(',').trimResults()
+ .omitEmptyStrings();
+
+ private final PigServer pigServer;
+ private final Set<String> whitelist;
+ private final Set<String> blacklist;
+
+ public BlackAndWhitelistFilter(PigServer pigServer) {
+ this.pigServer = pigServer;
+ whitelist = Sets.newHashSet();
+ blacklist = Sets.newHashSet();
+
+ init();
+ }
+
+ private void init() {
+ PigContext context = pigServer.getPigContext();
+ String whitelistConfig = context.getProperties().getProperty(PigConfiguration.PIG_WHITELIST);
+
+ if (whitelistConfig != null) {
+ Iterable<String> iter = SPLITTER.split(whitelistConfig);
+ for (String elem : iter) {
+ whitelist.add(elem.toUpperCase());
+ }
+ }
+
+ String blacklistConfig = context.getProperties().getProperty(PigConfiguration.PIG_BLACKLIST);
+ if (blacklistConfig != null) {
+ Iterable<String> iter = SPLITTER.split(blacklistConfig);
+ for(String elem : iter) {
+ String uElem = elem.toUpperCase();
+ if(whitelist.contains(uElem)) {
+ throw new IllegalStateException("Conflict between whitelist and blacklist. '"+elem+"' appears in both.");
+ }
+ blacklist.add(uElem);
+ }
+ }
+ }
+
+ @Override
+ public void validate(Command command) throws FrontendException {
+ if (blacklist.contains(command.name())) {
+ throw new FrontendException(command.name() + " command is not permitted. ", BLACKANDWHITELIST_ERROR_CODE);
+ }
+
+ // check for size of whitelist as an empty whitelist should not disallow using Pig commands altogether.
+ if (whitelist.size() > 0 && !whitelist.contains(command.name())) {
+ throw new FrontendException(command.name() + " command is not permitted. ", BLACKANDWHITELIST_ERROR_CODE);
+ }
+ }
+
+}
Added: pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistValidator.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistValidator.java?rev=1575200&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistValidator.java (added)
+++ pig/trunk/src/org/apache/pig/validator/BlackAndWhitelistValidator.java Fri Mar 7 07:12:59 2014
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.validator;
+
+import java.util.Set;
+
+import org.apache.pig.PigConfiguration;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.apache.pig.newplan.DepthFirstWalker;
+import org.apache.pig.newplan.OperatorPlan;
+import org.apache.pig.newplan.logical.relational.LOCogroup;
+import org.apache.pig.newplan.logical.relational.LOCross;
+import org.apache.pig.newplan.logical.relational.LOCube;
+import org.apache.pig.newplan.logical.relational.LODistinct;
+import org.apache.pig.newplan.logical.relational.LOFilter;
+import org.apache.pig.newplan.logical.relational.LOForEach;
+import org.apache.pig.newplan.logical.relational.LOGenerate;
+import org.apache.pig.newplan.logical.relational.LOInnerLoad;
+import org.apache.pig.newplan.logical.relational.LOJoin;
+import org.apache.pig.newplan.logical.relational.LOLimit;
+import org.apache.pig.newplan.logical.relational.LOLoad;
+import org.apache.pig.newplan.logical.relational.LONative;
+import org.apache.pig.newplan.logical.relational.LORank;
+import org.apache.pig.newplan.logical.relational.LOSort;
+import org.apache.pig.newplan.logical.relational.LOSplit;
+import org.apache.pig.newplan.logical.relational.LOSplitOutput;
+import org.apache.pig.newplan.logical.relational.LOStore;
+import org.apache.pig.newplan.logical.relational.LOStream;
+import org.apache.pig.newplan.logical.relational.LOUnion;
+import org.apache.pig.newplan.logical.relational.LogicalRelationalNodesVisitor;
+import org.apache.pig.newplan.logical.rules.LogicalRelationalNodeValidator;
+import org.python.google.common.base.Splitter;
+import org.python.google.common.collect.Sets;
+
+/**
+ * This validator walks through the list of operators defined in {@link PigConfiguration#PIG_BLACKLIST} and
+ * {@link PigConfiguration#PIG_WHITELIST} and checks whether the operation is permitted. In case these
+ * properties are not defined (default), we let everything pass as usual.
+ *
+ */
+public final class BlackAndWhitelistValidator implements LogicalRelationalNodeValidator {
+ private final PigContext pigContext;
+ private final OperatorPlan operatorPlan;
+
+ public BlackAndWhitelistValidator(PigContext pigContext, OperatorPlan operatorPlan) {
+ this.pigContext = pigContext;
+ this.operatorPlan = operatorPlan;
+ }
+
+ public void validate() throws FrontendException {
+ BlackAndWhitelistVisitor visitor = new BlackAndWhitelistVisitor(this.operatorPlan);
+ visitor.visit();
+ }
+
+ private class BlackAndWhitelistVisitor extends LogicalRelationalNodesVisitor {
+ private static final int ERROR_CODE = 1855;
+
+ private final Splitter splitter;
+ private final Set<String> blacklist;
+ private final Set<String> whitelist;
+
+ protected BlackAndWhitelistVisitor(OperatorPlan plan) throws FrontendException {
+ super(plan, new DepthFirstWalker(plan));
+ blacklist = Sets.newHashSet();
+ whitelist = Sets.newHashSet();
+ splitter = Splitter.on(',').trimResults().omitEmptyStrings();
+
+ init();
+ }
+
+ private void init() {
+ String blacklistConfig = pigContext.getProperties().getProperty(PigConfiguration.PIG_BLACKLIST);
+ // Set blacklist only if it's been defined by a user
+ if (blacklistConfig != null) {
+ Iterable<String> iter = splitter.split(blacklistConfig);
+ for(String elem : iter) {
+ blacklist.add(elem.toLowerCase());
+ }
+ }
+
+ String whitelistConfig = pigContext.getProperties().getProperty(PigConfiguration.PIG_WHITELIST);
+ // Set whitelist only if it's been defined by a user
+ if (whitelistConfig != null) {
+ Iterable<String> iter = splitter.split(whitelistConfig);
+ for(String elem : iter) {
+ String lElem = elem.toLowerCase();
+ if(blacklist.contains(lElem)) {
+ throw new IllegalStateException("Conflict between whitelist and blacklist. '"+elem+"' appears in both.");
+ }
+ whitelist.add(lElem);
+ }
+ }
+ }
+
+ private void check(String operator) throws FrontendException {
+ // throw an exception if the operator is not defined in whitelist
+ if(whitelist != null && whitelist.size() > 0 && !whitelist.contains(operator)) {
+ throw new FrontendException(operator +" is disabled. ", ERROR_CODE);
+ }
+
+ // throw an exception if operator is defined in blacklist
+ if(blacklist != null && blacklist.size() > 0 && blacklist.contains(operator)) {
+ throw new FrontendException(operator + " is disabled. ", ERROR_CODE);
+ }
+ }
+
+ @Override
+ public void visit(LOLoad load) throws FrontendException {
+ check("load");
+ }
+
+ @Override
+ public void visit(LOFilter filter) throws FrontendException {
+ check("filter");
+ }
+
+ @Override
+ public void visit(LOStore store) throws FrontendException {
+ check("store");
+ }
+
+ @Override
+ public void visit(LOJoin join) throws FrontendException {
+ check("join");
+ }
+
+ @Override
+ public void visit(LOForEach foreach) throws FrontendException {
+ check("foreach");
+ }
+
+ @Override
+ public void visit(LOGenerate gen) throws FrontendException {
+ }
+
+ public void visit(LOInnerLoad load) throws FrontendException {
+ }
+
+ @Override
+ public void visit(LOCube cube) throws FrontendException {
+ check("cube");
+ }
+
+ public void visit(LOCogroup loCogroup) throws FrontendException {
+ check("group");
+ check("cogroup");
+ }
+
+ @Override
+ public void visit(LOSplit loSplit) throws FrontendException {
+ check("split");
+ }
+
+ @Override
+ public void visit(LOSplitOutput loSplitOutput) throws FrontendException {
+ }
+
+ @Override
+ public void visit(LOUnion loUnion) throws FrontendException {
+ check("union");
+ }
+
+ @Override
+ public void visit(LOSort loSort) throws FrontendException {
+ check("order");
+ }
+
+ @Override
+ public void visit(LORank loRank) throws FrontendException {
+ check("rank");
+ }
+
+ @Override
+ public void visit(LODistinct loDistinct) throws FrontendException {
+ check("distinct");
+ }
+
+ @Override
+ public void visit(LOLimit loLimit) throws FrontendException {
+ check("limit");
+ }
+
+ @Override
+ public void visit(LOCross loCross) throws FrontendException {
+ check("cross");
+ }
+
+ @Override
+ public void visit(LOStream loStream) throws FrontendException {
+ check("stream");
+ }
+
+ @Override
+ public void visit(LONative nativeMR) throws FrontendException {
+ check("mapreduce");
+ }
+
+ }
+
+}
Added: pig/trunk/src/org/apache/pig/validator/PigCommandFilter.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/validator/PigCommandFilter.java?rev=1575200&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/validator/PigCommandFilter.java (added)
+++ pig/trunk/src/org/apache/pig/validator/PigCommandFilter.java Fri Mar 7 07:12:59 2014
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.validator;
+
+import java.io.IOException;
+
+import org.apache.pig.classification.InterfaceAudience;
+import org.apache.pig.classification.InterfaceStability;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+
+/**
+ *
+ * Interface defining Pig commands and a
+ * {@link PigCommandFilter#validate(Command)} method to operate on it
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface PigCommandFilter {
+
+ public enum Command {
+ FS, LS, SH, MAPREDUCE, REGISTER, SET, CAT, CD, DUMP, KILL, PWD, MV, CP, COPYTOLOCAL, COPYFROMLOCAL, MKDIR, RM, RMF, ILLUSTRATE
+ }
+
+ /**
+ * Validates a Pig command as defined by {@link Command}.
+ *
+ * @param command
+ * @throws IOException
+ */
+ public void validate(Command command) throws FrontendException;
+
+}
Modified: pig/trunk/test/org/apache/pig/pigunit/pig/GruntParser.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/pigunit/pig/GruntParser.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/pigunit/pig/GruntParser.java (original)
+++ pig/trunk/test/org/apache/pig/pigunit/pig/GruntParser.java Fri Mar 7 07:12:59 2014
@@ -36,79 +36,80 @@ import java.util.regex.Pattern;
* e.g. pig.registerQuery replace the query of a certain alias...
*/
public class GruntParser extends org.apache.pig.tools.grunt.GruntParser {
- /** A mapping <alias,query> to apply to the pig script. */
- private final Map<String, String> aliasOverride;
+ /** A mapping <alias,query> to apply to the pig script. */
+ private final Map<String, String> aliasOverride;
- /**
- * Initializes the Pig parser with its list of aliases to override.
- *
- * @param stream The Pig script stream.
- * @param aliasOverride The list of aliases to override in the Pig script.
- */
- public GruntParser(Reader stream, Map<String, String> aliasOverride) {
- super(stream);
- this.aliasOverride = aliasOverride;
- }
-
- /**
- * Pig relations that have been blanked are dropped.
- */
- @Override
- protected void processPig(String cmd) throws IOException {
- String command = override(cmd);
+ /**
+ * Initializes the Pig parser with its list of aliases to override.
+ *
+ * @param stream The Pig script stream.
+ * @param aliasOverride The list of aliases to override in the Pig script.
+ */
+ public GruntParser(Reader stream, Map<String, String> aliasOverride) {
+ this(stream, null, aliasOverride);
+ }
- if (! command.equals("")) {
- super.processPig(command);
+ public GruntParser(Reader stream, PigServer pigServer, Map<String, String> aliasOverride) {
+ super(stream, pigServer);
+ this.aliasOverride = aliasOverride;
}
- }
- /**
- * Overrides the relations of the pig script that we want to change.
- *
- * @param query The current pig query processed by the parser.
- * @return The same query, or a modified query, or blank.
- */
- public String override(String query) {
- // a path to be prepended to all the file names in the script
- String fsRoot = System.getProperty("pigunit.filesystem.prefix");
- if(fsRoot != null) {
- query = Pattern.compile("(LOAD\\s+'(([^:/?#]+)://)?)", Pattern.CASE_INSENSITIVE).
- matcher(query).
- replaceFirst("$1" + fsRoot);
- query = Pattern.compile("(STORE\\s+([^']+)\\s+INTO\\s+'(([^:/?#]+)://)?)", Pattern.CASE_INSENSITIVE).
- matcher(query).
- replaceFirst("$1" + fsRoot);
+ /**
+ * Pig relations that have been blanked are dropped.
+ */
+ @Override
+ protected void processPig(String cmd) throws IOException {
+ String command = override(cmd);
+
+ if (!command.equals("")) {
+ super.processPig(command);
+ }
}
- Map<String, String> metaData = new HashMap<String, String>();
+ /**
+ * Overrides the relations of the pig script that we want to change.
+ *
+ * @param query
+ * The current pig query processed by the parser.
+ * @return The same query, or a modified query, or blank.
+ */
+ public String override(String query) {
+ // a path to be prepended to all the file names in the script
+ String fsRoot = System.getProperty("pigunit.filesystem.prefix");
+ if (fsRoot != null) {
+ query = Pattern.compile("(LOAD\\s+'(([^:/?#]+)://)?)", Pattern.CASE_INSENSITIVE).matcher(query).replaceFirst("$1" + fsRoot);
+ query = Pattern.compile("(STORE\\s+([^']+)\\s+INTO\\s+'(([^:/?#]+)://)?)", Pattern.CASE_INSENSITIVE).matcher(query).replaceFirst("$1" + fsRoot);
+ }
+
+ Map<String, String> metaData = new HashMap<String, String>();
+
+ for (Entry<String, String> alias : aliasOverride.entrySet()) {
+ saveLastStoreAlias(query, metaData);
+
+ if (query.toLowerCase().startsWith(alias.getKey().toLowerCase() + " ")) {
+ System.out.println(String.format("%s\n--> %s", query, alias.getValue() == "" ? "none" : alias.getValue()));
+ query = alias.getValue();
+ }
+ }
- for (Entry<String, String> alias : aliasOverride.entrySet()) {
- saveLastStoreAlias(query, metaData);
+ aliasOverride.putAll(metaData);
- if (query.toLowerCase().startsWith(alias.getKey().toLowerCase() + " ")) {
- System.out.println(
- String.format("%s\n--> %s", query, alias.getValue() == "" ? "none" : alias.getValue()));
- query = alias.getValue();
- }
+ return query;
}
- aliasOverride.putAll(metaData);
-
- return query;
- }
-
- /**
- * Saves the name of the alias of the last store.
- *
- * <p>Maybe better to replace it by PigServer.getPigContext().getLastAlias().
- */
- void saveLastStoreAlias(String cmd, Map<String, String> metaData) {
- if (cmd.toUpperCase().startsWith("STORE")) {
- Pattern outputFile = Pattern.compile("STORE +([^']+) INTO.*", Pattern.CASE_INSENSITIVE);
- Matcher matcher = outputFile.matcher(cmd);
- if (matcher.matches()) {
- metaData.put("LAST_STORE_ALIAS", matcher.group(1));
- }
+ /**
+ * Saves the name of the alias of the last store.
+ *
+ * <p>
+ * Maybe better to replace it by PigServer.getPigContext().getLastAlias().
+ */
+ void saveLastStoreAlias(String cmd, Map<String, String> metaData) {
+ if (cmd.toUpperCase().startsWith("STORE")) {
+ Pattern outputFile = Pattern.compile("STORE +([^']+) INTO.*", Pattern.CASE_INSENSITIVE);
+ Matcher matcher = outputFile.matcher(cmd);
+ if (matcher.matches()) {
+ metaData.put("LAST_STORE_ALIAS", matcher.group(1));
+ }
+ }
}
- }
}
Modified: pig/trunk/test/org/apache/pig/pigunit/pig/PigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/pigunit/pig/PigServer.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/pigunit/pig/PigServer.java (original)
+++ pig/trunk/test/org/apache/pig/pigunit/pig/PigServer.java Fri Mar 7 07:12:59 2014
@@ -50,9 +50,8 @@ public class PigServer extends org.apach
throws IOException {
try {
InputStream compositeStream = Utils.getCompositeStream(new FileInputStream(fileName), pigContext.getProperties());
- GruntParser grunt = new GruntParser(new InputStreamReader(compositeStream), aliasOverride);
+ GruntParser grunt = new GruntParser(new InputStreamReader(compositeStream), this, aliasOverride);
grunt.setInteractive(false);
- grunt.setParams(this);
grunt.parseStopOnError(true);
} catch (FileNotFoundException e) {
e.printStackTrace();
Added: pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java?rev=1575200&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java (added)
+++ pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java Fri Mar 7 07:12:59 2014
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import static org.apache.pig.builtin.mock.Storage.resetData;
+import static org.apache.pig.builtin.mock.Storage.tuple;
+import static org.apache.pig.newplan.logical.relational.LOTestHelper.newLOLoad;
+
+import java.io.File;
+import java.util.Properties;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.pig.ExecType;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.PigConfiguration;
+import org.apache.pig.PigServer;
+import org.apache.pig.StoreFuncInterface;
+import org.apache.pig.backend.datastorage.DataStorage;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.builtin.mock.Storage.Data;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.io.FileSpec;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.apache.pig.newplan.logical.relational.LOFilter;
+import org.apache.pig.newplan.logical.relational.LOLoad;
+import org.apache.pig.newplan.logical.relational.LOStore;
+import org.apache.pig.newplan.logical.relational.LogicalPlan;
+import org.apache.pig.newplan.logical.rules.LogicalRelationalNodeValidator;
+import org.apache.pig.parser.QueryParser;
+import org.apache.pig.tools.grunt.GruntParser;
+import org.apache.pig.validator.BlackAndWhitelistFilter;
+import org.apache.pig.validator.BlackAndWhitelistValidator;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+/**
+ *
+ * Contains tests for {@link BlackAndWhitelistValidator} and
+ * {@link BlackAndWhitelistFilter}
+ *
+ */
+public class TestBlackAndWhitelistValidator {
+ private PigContext ctx;
+
+ @Rule
+ public ExpectedException expectedEx = ExpectedException.none();
+
+ @Before
+ public void setUp() throws Exception {
+ ctx = new PigContext(ExecType.LOCAL, new Properties());
+ ctx.connect();
+ }
+
+ /**
+ * Tests the blacklist filter. We blacklist "set" and make sure this test
+ * throws a {@link FrontendException}
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testBlacklist() throws Exception {
+ expectedEx.expect(FrontendException.class);
+ expectedEx.expectMessage("SET command is not permitted");
+ ctx.getProperties().setProperty(PigConfiguration.PIG_BLACKLIST, "set");
+ PigServer pigServer = new PigServer(ctx);
+ Data data = resetData(pigServer);
+
+ data.set("foo", tuple("a", 1, "b"), tuple("b", 2, "c"),
+ tuple("c", 3, "d"));
+
+ StringBuilder script = new StringBuilder();
+ script.append("set io.sort.mb 1000;")
+ .append("A = LOAD 'foo' USING mock.Storage() AS (f1:chararray,f2:int,f3:chararray);")
+ .append("B = order A by f1,f2,f3 DESC;")
+ .append("STORE B INTO 'bar' USING mock.Storage();");
+
+ pigServer.registerScript(IOUtils.toInputStream(script));
+ }
+
+ /**
+ * Tests {@link BlackAndWhitelistValidator}. The logical plan generated
+ * contains a filter, and the test must throw a {@link FrontendException} as
+ * we set "filter" in the blacklist
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testValidator() throws Exception {
+ expectedEx.expect(FrontendException.class);
+ expectedEx.expectMessage("filter is disabled");
+ // disabling filter
+ ctx.getProperties().setProperty(PigConfiguration.PIG_BLACKLIST,
+ "filter");
+
+ LogicalPlan plan = generateLogicalPlan("foo", "bar", ctx.getDfs());
+
+ LogicalRelationalNodeValidator executor = new BlackAndWhitelistValidator(ctx, plan);
+ executor.validate();
+ }
+
+ /**
+ * This test must pass as we allow load, store, filter to be a part of the
+ * whitelist. The logical plan being checked in this test contains these
+ * operators only.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testWhitelist1() throws Exception {
+ ctx.getProperties().setProperty(PigConfiguration.PIG_WHITELIST,
+ "load, store,filter");
+ LogicalPlan plan = generateLogicalPlan("foo", "bar", ctx.getDfs());
+
+ LogicalRelationalNodeValidator executor = new BlackAndWhitelistValidator(ctx, plan);
+ executor.validate();
+ }
+
+ @Test
+ public void testWhitelist2() throws Exception {
+ expectedEx.expect(FrontendException.class);
+ expectedEx.expectMessage("filter is disabled");
+ // only load and store are allowed. Having a filter in the logical plan
+ // must
+ // cause the script to fail
+ ctx.getProperties().setProperty(PigConfiguration.PIG_WHITELIST,
+ "load, store");
+ LogicalPlan plan = generateLogicalPlan("foo", "bar", ctx.getDfs());
+
+ LogicalRelationalNodeValidator executor = new BlackAndWhitelistValidator(ctx, plan);
+ executor.validate();
+ }
+
+ /**
+ * If there is a conflict between blacklist and whitelist contents, the
+ * validator or filter must throw an {@link IllegalStateException}.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testBlackAndWhitelist() throws Exception {
+ expectedEx.expect(IllegalStateException.class);
+ expectedEx
+ .expectMessage("Conflict between whitelist and blacklist. 'filter' appears in both.");
+
+ ctx.getProperties().setProperty(PigConfiguration.PIG_WHITELIST,
+ "load, store, filter");
+ ctx.getProperties().setProperty(PigConfiguration.PIG_BLACKLIST,
+ "filter");
+ LogicalPlan plan = generateLogicalPlan("foo", "bar", ctx.getDfs());
+
+ LogicalRelationalNodeValidator executor = new BlackAndWhitelistValidator(ctx, plan);
+ executor.validate();
+ }
+
+ /**
+ * This is to test the script fails when used with {@link PigServer}, which
+ * uses {@link QueryParser} and not the {@link GruntParser}
+ */
+ @Test(expected = FrontendException.class)
+ public void testBlacklistWithPigServer() throws Exception {
+ ctx.getProperties()
+ .setProperty(PigConfiguration.PIG_BLACKLIST, "order");
+ PigServer pigServer = new PigServer(ctx);
+ Data data = resetData(pigServer);
+
+ data.set("foo", tuple("a", 1, "b"), tuple("b", 2, "c"),
+ tuple("c", 3, "d"));
+
+ pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage() AS (f1:chararray,f2:int,f3:chararray);");
+ pigServer.registerQuery("B = order A by f1,f2,f3 DESC;");
+ pigServer.registerQuery("STORE B INTO 'bar' USING mock.Storage();");
+ }
+
+ /**
+ * Test listStatus fails if its disallowed via the blacklist
+ */
+ @Test
+ public void testBlacklistCmdWithPigServer() throws Exception {
+ expectedEx.expect(FrontendException.class);
+ expectedEx.expectMessage("LS command is not permitted");
+
+ ctx.getProperties().setProperty(PigConfiguration.PIG_BLACKLIST, "ls");
+ PigServer pigServer = new PigServer(ctx);
+
+ pigServer.listPaths("foo");
+ }
+
+ /**
+ * Test deleteFile fails if its disallowed via the blacklist
+ */
+ @Test(expected = FrontendException.class)
+ public void testBlacklistRemoveWithPigServer() throws Exception {
+ ctx.getProperties().setProperty(PigConfiguration.PIG_BLACKLIST, "rm");
+ PigServer pigServer = new PigServer(ctx);
+
+ pigServer.deleteFile("foo");
+ }
+
+ /**
+ * Test mkdirs fails if its disallowed via the blacklist
+ */
+ @Test(expected = FrontendException.class)
+ public void testBlacklistMkdirWithPigServer() throws Exception {
+ ctx.getProperties().setProperty(PigConfiguration.PIG_BLACKLIST, "mkdir");
+ PigServer pigServer = new PigServer(ctx);
+
+ pigServer.mkdirs("foo");
+ }
+
+ /**
+ * This is to test the script fails when used with {@link PigServer}, which
+ * uses {@link QueryParser} and not the {@link GruntParser}
+ */
+ @Test(expected = FrontendException.class)
+ public void testWhitelistWithPigServer() throws Exception {
+ ctx.getProperties().setProperty(PigConfiguration.PIG_WHITELIST, "load");
+ PigServer pigServer = new PigServer(ctx);
+ Data data = resetData(pigServer);
+
+ data.set("foo", tuple("a", 1, "b"), tuple("b", 2, "c"),
+ tuple("c", 3, "d"));
+
+ pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage() AS (f1:chararray,f2:int,f3:chararray);");
+ pigServer.registerQuery("B = order A by f1,f2,f3 DESC;");
+ pigServer.registerQuery("STORE B INTO 'bar' USING mock.Storage();");
+ }
+
+ /**
+ *
+ * Generate a {@link LogicalPlan} containing a Load, Filter and Store
+ * operators
+ *
+ * @param inputFile
+ * @param outputFile
+ * @param dfs
+ * @return
+ * @throws Exception
+ */
+ private LogicalPlan generateLogicalPlan(String inputFile,
+ String outputFile, DataStorage dfs) throws Exception {
+ LogicalPlan plan = new LogicalPlan();
+ FileSpec filespec1 = new FileSpec(generateTmpFile(inputFile).getAbsolutePath(), new FuncSpec("org.apache.pig.builtin.PigStorage"));
+ FileSpec filespec2 = new FileSpec(generateTmpFile(outputFile).getAbsolutePath(), new FuncSpec("org.apache.pig.builtin.PigStorage"));
+ LOLoad load = newLOLoad(filespec1, null, plan, ConfigurationUtil.toConfiguration(dfs.getConfiguration()));
+ LOStore store = new LOStore(plan, filespec2, (StoreFuncInterface) PigContext.instantiateFuncFromSpec(filespec2.getFuncSpec()), null);
+
+ LOFilter filter = new LOFilter(plan);
+
+ plan.add(load);
+ plan.add(store);
+ plan.add(filter);
+
+ plan.connect(load, filter);
+ plan.connect(filter, store);
+
+ return plan;
+ }
+
+ private File generateTmpFile(String filename) throws Exception {
+ return Util.createTempFileDelOnExit(filename, ".txt");
+ }
+}
Modified: pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java Fri Mar 7 07:12:59 2014
@@ -1232,10 +1232,9 @@ public class TestMultiQueryCompiler {
+ "b = filter a by uid > 5;"
+ "describe b;"
+ "store b into '/tmp/output1';\n";
-
- GruntParser parser = new GruntParser(new StringReader(script));
+
+ GruntParser parser = new GruntParser(new StringReader(script), myPig);
parser.setInteractive(false);
- parser.setParams(myPig);
parser.parseStopOnError();
} catch (Exception e) {
@@ -1255,10 +1254,9 @@ public class TestMultiQueryCompiler {
+ "b = filter a by uid > 5;"
+ "illustrate b;"
+ "store b into '/tmp/output1';\n";
-
- GruntParser parser = new GruntParser(new StringReader(script));
+
+ GruntParser parser = new GruntParser(new StringReader(script), myPig);
parser.setInteractive(false);
- parser.setParams(myPig);
parser.parseStopOnError();
} catch (Exception e) {
@@ -1278,10 +1276,9 @@ public class TestMultiQueryCompiler {
+ "b = filter a by uid > 5;"
+ "explain b;"
+ "store b into '/tmp/output1';\n";
-
- GruntParser parser = new GruntParser(new StringReader(script));
+
+ GruntParser parser = new GruntParser(new StringReader(script), myPig);
parser.setInteractive(false);
- parser.setParams(myPig);
parser.parseStopOnError();
} catch (Exception e) {
@@ -1301,10 +1298,9 @@ public class TestMultiQueryCompiler {
+ "b = filter a by uid > 5;"
+ "dump b;"
+ "store b into '/tmp/output1';\n";
-
- GruntParser parser = new GruntParser(new StringReader(script));
+
+ GruntParser parser = new GruntParser(new StringReader(script), myPig);
parser.setInteractive(false);
- parser.setParams(myPig);
parser.parseStopOnError();
} catch (Exception e) {
@@ -1335,9 +1331,8 @@ public class TestMultiQueryCompiler {
public void testLoadStoreLoop() {
try {
String script = "a = load 'dummy'; b = filter a by $0 == 1; store b into 'dummy';\n";
- GruntParser parser = new GruntParser(new StringReader(script));
+ GruntParser parser = new GruntParser(new StringReader(script), myPig);
parser.setInteractive(false);
- parser.setParams(myPig);
myPig.getPigContext().inExplain = true;
parser.parseStopOnError();
} catch (Exception e) {
Modified: pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestMultiQueryLocal.java Fri Mar 7 07:12:59 2014
@@ -18,15 +18,13 @@
package org.apache.pig.test;
import java.io.BufferedReader;
-import java.io.DataOutputStream;
+import java.io.File;
import java.io.FileReader;
-import java.io.StringReader;
import java.io.IOException;
-import java.io.File;
-import java.util.Iterator;
-import java.util.List;
+import java.io.StringReader;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.Iterator;
import java.util.Properties;
import junit.framework.Assert;
@@ -46,13 +44,13 @@ import org.apache.pig.backend.hadoop.exe
import org.apache.pig.builtin.PigStorage;
import org.apache.pig.data.Tuple;
import org.apache.pig.data.TupleFactory;
+import org.apache.pig.impl.PigContext;
import org.apache.pig.impl.io.FileLocalizer;
import org.apache.pig.impl.plan.Operator;
import org.apache.pig.impl.plan.OperatorPlan;
-import org.apache.pig.impl.PigContext;
-import org.apache.pig.tools.grunt.GruntParser;
import org.apache.pig.impl.util.LogUtils;
import org.apache.pig.newplan.logical.relational.LogicalPlan;
+import org.apache.pig.tools.grunt.GruntParser;
import org.apache.pig.tools.pigscript.parser.ParseException;
import org.apache.pig.tools.pigstats.JobStats;
import org.apache.pig.tools.pigstats.PigStats;
@@ -444,10 +442,9 @@ public class TestMultiQueryLocal {
+ "b = filter a by uid > 5;"
+ "explain b;"
+ "store b into '" + TMP_DIR + "/Pig-TestMultiQueryLocal1';\n";
-
- GruntParser parser = new GruntParser(new StringReader(script));
+
+ GruntParser parser = new GruntParser(new StringReader(script), myPig);
parser.setInteractive(false);
- parser.setParams(myPig);
parser.parseStopOnError();
} catch (Exception e) {
@@ -469,10 +466,9 @@ public class TestMultiQueryLocal {
+ "b = filter a by uid > 5;"
+ "dump b;"
+ "store b into '" + TMP_DIR + "/Pig-TestMultiQueryLocal1';\n";
-
- GruntParser parser = new GruntParser(new StringReader(script));
+
+ GruntParser parser = new GruntParser(new StringReader(script), myPig);
parser.setInteractive(false);
- parser.setParams(myPig);
parser.parseStopOnError();
} catch (Exception e) {
@@ -494,10 +490,9 @@ public class TestMultiQueryLocal {
+ "b = filter a by uid > 5;"
+ "describe b;"
+ "store b into '" + TMP_DIR + "/Pig-TestMultiQueryLocal1';\n";
-
- GruntParser parser = new GruntParser(new StringReader(script));
+
+ GruntParser parser = new GruntParser(new StringReader(script), myPig);
parser.setInteractive(false);
- parser.setParams(myPig);
parser.parseStopOnError();
} catch (Exception e) {
@@ -519,11 +514,10 @@ public class TestMultiQueryLocal {
+ "b = filter a by uid > 5;"
+ "illustrate b;"
+ "store b into '" + TMP_DIR + "/Pig-TestMultiQueryLocal1';\n";
-
- GruntParser parser = new GruntParser(new StringReader(script));
+
+ GruntParser parser = new GruntParser(new StringReader(script), myPig);
parser.setInteractive(false);
myPig.getPigContext().getProperties().setProperty("pig.usenewlogicalplan", "true");
- parser.setParams(myPig);
parser.parseStopOnError();
} catch (Exception e) {
Modified: pig/trunk/test/org/apache/pig/test/TestPigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigServer.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigServer.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigServer.java Fri Mar 7 07:12:59 2014
@@ -938,9 +938,8 @@ public class TestPigServer {
PigServer pigServer = new PigServer(pigContext);
data = resetData(pigServer);
data.set("foo", tuple("a", 1, "b"), tuple("b", 2, "c"), tuple("c", 3, "d"));
- GruntParser grunt = new GruntParser(in);
+ GruntParser grunt = new GruntParser(in, pigServer);
grunt.setInteractive(false);
- grunt.setParams(pigServer);
grunt.parseStopOnError(true); //not batch
}
Modified: pig/trunk/test/org/apache/pig/test/Util.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/Util.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/Util.java (original)
+++ pig/trunk/test/org/apache/pig/test/Util.java Fri Mar 7 07:12:59 2014
@@ -610,10 +610,8 @@ public class Util {
}
PigServer ps = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
String script = getMkDirCommandForHadoop2_0(fileNameOnCluster) + "fs -put " + localFileName + " " + fileNameOnCluster;
-
- GruntParser parser = new GruntParser(new StringReader(script));
+ GruntParser parser = new GruntParser(new StringReader(script), ps);
parser.setInteractive(false);
- parser.setParams(ps);
try {
parser.parseStopOnError();
} catch (org.apache.pig.tools.pigscript.parser.ParseException e) {
@@ -632,9 +630,8 @@ public class Util {
new File(toLocalFileName).deleteOnExit();
- GruntParser parser = new GruntParser(new StringReader(script));
+ GruntParser parser = new GruntParser(new StringReader(script), ps);
parser.setInteractive(false);
- parser.setParams(ps);
try {
parser.parseStopOnError();
} catch (org.apache.pig.tools.pigscript.parser.ParseException e) {
Modified: pig/trunk/test/org/apache/pig/test/pigunit/pig/TestGruntParser.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/pigunit/pig/TestGruntParser.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/pigunit/pig/TestGruntParser.java (original)
+++ pig/trunk/test/org/apache/pig/test/pigunit/pig/TestGruntParser.java Fri Mar 7 07:12:59 2014
@@ -19,7 +19,10 @@ import java.util.Map;
import junit.framework.Assert;
+import org.apache.pig.ExecType;
+import org.apache.pig.backend.executionengine.ExecException;
import org.apache.pig.pigunit.pig.GruntParser;
+import org.apache.pig.pigunit.pig.PigServer;
import org.junit.Before;
import org.junit.Test;
@@ -30,12 +33,13 @@ public class TestGruntParser {
@SuppressWarnings("serial")
@Before
- public void setUp() {
+ public void setUp() throws ExecException {
override = new HashMap<String, String>() {{
put("STORE", "");
put("DUMP", "");
}};
- parser = new GruntParser(new StringReader(""), override);
+ PigServer pigServer = new PigServer(ExecType.LOCAL);
+ parser = new GruntParser(new StringReader(""), pigServer, override);
}
@Test
Modified: pig/trunk/test/org/apache/pig/tools/grunt/TestGruntParser.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/tools/grunt/TestGruntParser.java?rev=1575200&r1=1575199&r2=1575200&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/tools/grunt/TestGruntParser.java (original)
+++ pig/trunk/test/org/apache/pig/tools/grunt/TestGruntParser.java Fri Mar 7 07:12:59 2014
@@ -39,18 +39,17 @@ public class TestGruntParser {
public void setup() throws IOException {
pig = new PigServer(ExecType.LOCAL);
}
-
+
@Test
public void testProcessRemove() throws IOException {
File tmpFile = File.createTempFile("TestGruntParser", "testProcessRemove");
String dummyScript = "";
- parser = new GruntParser(new ByteArrayInputStream(dummyScript.getBytes()));
- parser.setParams(pig);
-
+ parser = new GruntParser(new ByteArrayInputStream(dummyScript.getBytes()), pig);
+
//Delete existing file and check that it doesn't exist
parser.processRemove(tmpFile.getAbsolutePath(), "");
assertFalse(tmpFile.exists());
-
+
//Delete non-existing file and check for exception
try {
parser.processRemove(tmpFile.getAbsolutePath(), "");
|