You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by na...@apache.org on 2017/04/21 21:52:29 UTC

[1/2] incubator-systemml git commit: Use Apache Commons CLI to parse command line arguments in DMLScript

Repository: incubator-systemml
Updated Branches:
  refs/heads/master f73673d59 -> 32924dc60


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/java/org/apache/sysml/api/MLContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/MLContext.java b/src/main/java/org/apache/sysml/api/MLContext.java
index 809776a..b3102e9 100644
--- a/src/main/java/org/apache/sysml/api/MLContext.java
+++ b/src/main/java/org/apache/sysml/api/MLContext.java
@@ -771,7 +771,7 @@ public class MLContext {
 				args[i] = entry.getKey() + "=" + entry.getValue();
 			i++;
 		}
-		return compileAndExecuteScript(dmlScriptFilePath, args, true, parsePyDML, configFilePath);
+		return compileAndExecuteScript(dmlScriptFilePath, args, true, parsePyDML ? ScriptType.PYDML : ScriptType.DML, configFilePath);
 	}
 	
 	/**
@@ -785,17 +785,7 @@ public class MLContext {
 	 * @throws ParseException if ParseException occurs
 	 */
 	public MLOutput execute(String dmlScriptFilePath, Map<String, String> namedArgs, String configFilePath) throws IOException, DMLException, ParseException {
-		String [] args = new String[namedArgs.size()];
-		int i = 0;
-		for(Entry<String, String> entry : namedArgs.entrySet()) {
-			if(entry.getValue().trim().isEmpty())
-				args[i] = entry.getKey() + "=\"" + entry.getValue() + "\"";
-			else
-				args[i] = entry.getKey() + "=" + entry.getValue();
-			i++;
-		}
-		
-		return compileAndExecuteScript(dmlScriptFilePath, args, true, false, configFilePath);
+		return execute(dmlScriptFilePath, namedArgs, false, configFilePath);
 	}
 	
 	/**
@@ -1014,7 +1004,7 @@ public class MLContext {
 	 * @throws ParseException if ParseException occurs
 	 */
 	public MLOutput execute(String dmlScriptFilePath, String [] args, boolean parsePyDML, String configFilePath) throws IOException, DMLException, ParseException {
-		return compileAndExecuteScript(dmlScriptFilePath, args, false, parsePyDML, configFilePath);
+		return compileAndExecuteScript(dmlScriptFilePath, args, false, parsePyDML ? ScriptType.PYDML : ScriptType.DML, configFilePath);
 	}
 	
 	/**
@@ -1067,7 +1057,7 @@ public class MLContext {
 	 * @throws ParseException if ParseException occurs
 	 */
 	public MLOutput execute(String dmlScriptFilePath, boolean parsePyDML, String configFilePath) throws IOException, DMLException, ParseException {
-		return compileAndExecuteScript(dmlScriptFilePath, null, false, parsePyDML, configFilePath);
+		return compileAndExecuteScript(dmlScriptFilePath, null, false, parsePyDML ? ScriptType.PYDML : ScriptType.DML, configFilePath);
 	}
 	
 	/**
@@ -1314,7 +1304,7 @@ public class MLContext {
 
 	public MLOutput executeScript(String dmlScript, boolean isPyDML, String configFilePath)
 			throws IOException, DMLException {
-		return compileAndExecuteScript(dmlScript, null, false, false, isPyDML, configFilePath);
+		return compileAndExecuteScript(dmlScript, null, false, false, isPyDML ? ScriptType.PYDML : ScriptType.DML, configFilePath);
 	}
 
 	/*
@@ -1391,7 +1381,7 @@ public class MLContext {
 				args[i] = entry.getKey() + "=" + entry.getValue();
 			i++;
 		}
-		return compileAndExecuteScript(dmlScript, args, false, true, isPyDML, configFilePath);
+		return compileAndExecuteScript(dmlScript, args, false, true, isPyDML ? ScriptType.PYDML : ScriptType.DML, configFilePath);
 	}
 
 	private void checkIfRegisteringInputAllowed() throws DMLRuntimeException {
@@ -1400,26 +1390,29 @@ public class MLContext {
 		}
 	}
 	
-	private MLOutput compileAndExecuteScript(String dmlScriptFilePath, String [] args, boolean isNamedArgument, boolean isPyDML, String configFilePath) throws IOException, DMLException {
-		return compileAndExecuteScript(dmlScriptFilePath, args, true, isNamedArgument, isPyDML, configFilePath);
+	private MLOutput compileAndExecuteScript(String dmlScriptFilePath, String [] args, boolean isNamedArgument, ScriptType scriptType, String configFilePath) throws IOException, DMLException {
+		return compileAndExecuteScript(dmlScriptFilePath, args, true, isNamedArgument, scriptType, configFilePath);
 	}
-	
+
 	/**
 	 * All the execute() methods call this, which  after setting appropriate input/output variables
 	 * calls _compileAndExecuteScript
 	 * We have explicitly synchronized this function because MLContext/SystemML does not yet support multi-threading.
+	 * @throws ParseException if ParseException occurs
 	 * @param dmlScriptFilePath script file path
 	 * @param args arguments
-	 * @param isNamedArgument is named argument
+	 * @param isFile whether the string is a path
+	 * @param isNamedArgument  is named argument
+	 * @param scriptType type of script (DML or PyDML)
+	 * @param configFilePath path to config file
 	 * @return output as MLOutput
 	 * @throws IOException if IOException occurs
 	 * @throws DMLException if DMLException occurs
-	 * @throws ParseException if ParseException occurs
 	 */
-	private synchronized MLOutput compileAndExecuteScript(String dmlScriptFilePath, String [] args,  boolean isFile, boolean isNamedArgument, boolean isPyDML, String configFilePath) throws IOException, DMLException {
+	private synchronized MLOutput compileAndExecuteScript(String dmlScriptFilePath, String [] args,  boolean isFile, boolean isNamedArgument, ScriptType scriptType, String configFilePath) throws IOException, DMLException {
 		try {
 
-			DMLScript.SCRIPT_TYPE = isPyDML ? ScriptType.PYDML : ScriptType.DML;
+			DMLScript.SCRIPT_TYPE = scriptType;
 
 			if(getActiveMLContext() != null) {
 				throw new DMLRuntimeException("SystemML (and hence by definition MLContext) doesnot support parallel execute() calls from same or different MLContexts. "
@@ -1439,7 +1432,7 @@ public class MLContext {
 				Map<String, String> argVals = DMLScript.createArgumentsMap(isNamedArgument, args);
 				
 				// Run the DML script
-				ExecutionContext ec = executeUsingSimplifiedCompilationChain(dmlScriptFilePath, isFile, argVals, isPyDML, inputs, outputs, _variables, configFilePath);
+				ExecutionContext ec = executeUsingSimplifiedCompilationChain(dmlScriptFilePath, isFile, argVals, scriptType, inputs, outputs, _variables, configFilePath);
 				SparkExecutionContext sec = (SparkExecutionContext) ec;
 				
 				// Now collect the output
@@ -1482,7 +1475,7 @@ public class MLContext {
 	 * @param dmlScriptFilePath script file path
 	 * @param isFile true if file, false otherwise
 	 * @param argVals map of args
-	 * @param parsePyDML  true if pydml, false otherwise
+	 * @param scriptType  type of script (DML or PyDML)
 	 * @param inputs the inputs
 	 * @param outputs the outputs
 	 * @param inputSymbolTable the input symbol table
@@ -1492,7 +1485,7 @@ public class MLContext {
 	 * @throws DMLException if DMLException occurs
 	 * @throws ParseException if ParseException occurs
 	 */
-	private ExecutionContext executeUsingSimplifiedCompilationChain(String dmlScriptFilePath, boolean isFile, Map<String, String> argVals, boolean parsePyDML, 
+	private ExecutionContext executeUsingSimplifiedCompilationChain(String dmlScriptFilePath, boolean isFile, Map<String, String> argVals, ScriptType scriptType,
 			String[] inputs, String[] outputs, LocalVariableMap inputSymbolTable, String configFilePath) 
 		throws IOException, DMLException
 	{
@@ -1511,13 +1504,13 @@ public class MLContext {
 		ConfigurationManager.setGlobalConfig(cconf);
 		
 		//read dml script string
-		String dmlScriptStr = DMLScript.readDMLScript( isFile?"-f":"-s", dmlScriptFilePath);
+		String dmlScriptStr = DMLScript.readDMLScript( isFile, dmlScriptFilePath);
 		
 		//simplified compilation chain
 		_rtprog = null;
 		
 		//parsing
-		ParserWrapper parser = ParserFactory.createParser(parsePyDML);
+		ParserWrapper parser = ParserFactory.createParser(scriptType);
 		DMLProgram prog;
 		if (isFile) {
 			prog = parser.parse(dmlScriptFilePath, null, argVals);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/java/org/apache/sysml/api/jmlc/Connection.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/jmlc/Connection.java b/src/main/java/org/apache/sysml/api/jmlc/Connection.java
index fe37a6c..0e26c62 100644
--- a/src/main/java/org/apache/sysml/api/jmlc/Connection.java
+++ b/src/main/java/org/apache/sysml/api/jmlc/Connection.java
@@ -166,7 +166,7 @@ public class Connection implements Closeable
 		try
 		{
 			//parsing
-			ParserWrapper parser = ParserFactory.createParser(parsePyDML);
+			ParserWrapper parser = ParserFactory.createParser(parsePyDML ? ScriptType.PYDML : ScriptType.DML);
 			DMLProgram prog = parser.parse(null, script, args);
 			
 			//language validate

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java b/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
index e9270c5..ac2b92c 100644
--- a/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
+++ b/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
@@ -432,7 +432,7 @@ public class ScriptExecutor {
 	 */
 	protected void parseScript() {
 		try {
-			ParserWrapper parser = ParserFactory.createParser(script.getScriptType().isPYDML());
+			ParserWrapper parser = ParserFactory.createParser(script.getScriptType());
 			Map<String, Object> inputParameters = script.getInputParameters();
 			Map<String, String> inputParametersStringMaps = MLContextUtil.convertInputParametersForParser(
 					inputParameters, script.getScriptType());

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/java/org/apache/sysml/parser/ParserFactory.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/ParserFactory.java b/src/main/java/org/apache/sysml/parser/ParserFactory.java
index 37c62df..2d4e273 100644
--- a/src/main/java/org/apache/sysml/parser/ParserFactory.java
+++ b/src/main/java/org/apache/sysml/parser/ParserFactory.java
@@ -19,6 +19,7 @@
 
 package org.apache.sysml.parser;
 
+import org.apache.sysml.api.mlcontext.ScriptType;
 import org.apache.sysml.parser.common.CommonSyntacticValidator;
 import org.apache.sysml.parser.dml.DMLParserWrapper;
 import org.apache.sysml.parser.pydml.PyDMLParserWrapper;
@@ -28,19 +29,18 @@ public class ParserFactory {
 	/**
 	 * Factory method for creating parser wrappers
 	 * 
-	 * @param pydml
-	 *            true if a PyDMLParserWrapper is needed, false if a DMLParserWrapper is needed
+	 * @param scriptType
+	 *            type of script
 	 * @return parser wrapper (DMLParserWrapper or PyDMLParserWrapper)
 	 */
-	public static ParserWrapper createParser(boolean pydml) {
+	public static ParserWrapper createParser(ScriptType scriptType) {
 		ParserWrapper ret = null;
 
 		// create the parser instance
-		if (pydml)
-			ret = new PyDMLParserWrapper();
-		else
-			ret = new DMLParserWrapper();
-
+		switch (scriptType) {
+			case DML: ret = new DMLParserWrapper(); break;
+			case PYDML: ret = new PyDMLParserWrapper(); break;
+		}
 		CommonSyntacticValidator.init();
 
 		return ret;

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/java/org/apache/sysml/utils/Explain.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/Explain.java b/src/main/java/org/apache/sysml/utils/Explain.java
index b6e7b6f..af7102b 100644
--- a/src/main/java/org/apache/sysml/utils/Explain.java
+++ b/src/main/java/org/apache/sysml/utils/Explain.java
@@ -26,7 +26,6 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.sysml.api.DMLException;
 import org.apache.sysml.hops.Hop;
 import org.apache.sysml.hops.HopsException;
 import org.apache.sysml.hops.LiteralOp;
@@ -40,18 +39,18 @@ import org.apache.sysml.hops.globalopt.gdfgraph.GDFNode.NodeType;
 import org.apache.sysml.hops.ipa.FunctionCallGraph;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.parser.DMLProgram;
-import org.apache.sysml.parser.ForStatement;
 import org.apache.sysml.parser.ExternalFunctionStatement;
+import org.apache.sysml.parser.ForStatement;
 import org.apache.sysml.parser.ForStatementBlock;
 import org.apache.sysml.parser.FunctionStatement;
 import org.apache.sysml.parser.FunctionStatementBlock;
 import org.apache.sysml.parser.IfStatement;
 import org.apache.sysml.parser.IfStatementBlock;
+import org.apache.sysml.parser.LanguageException;
 import org.apache.sysml.parser.ParForStatementBlock;
+import org.apache.sysml.parser.StatementBlock;
 import org.apache.sysml.parser.WhileStatement;
 import org.apache.sysml.parser.WhileStatementBlock;
-import org.apache.sysml.parser.LanguageException;
-import org.apache.sysml.parser.StatementBlock;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.ExternalFunctionProgramBlock;
 import org.apache.sysml.runtime.controlprogram.ForProgramBlock;
@@ -436,30 +435,7 @@ public class Explain
 				
 		return counts;		
 	}
-
-	public static ExplainType parseExplainType( String arg ) 
-		throws DMLException
-	{
-		ExplainType ret = ExplainType.NONE;
-		
-		if( arg !=null )
-		{
-			if( arg.equalsIgnoreCase("hops") )
-				ret = ExplainType.HOPS;
-			else if( arg.equalsIgnoreCase("runtime") )
-				ret = ExplainType.RUNTIME;
-			else if( arg.equalsIgnoreCase("recompile_hops") )
-				ret = ExplainType.RECOMPILE_HOPS;
-			else if( arg.equalsIgnoreCase("recompile_runtime") )
-				ret = ExplainType.RECOMPILE_RUNTIME;
-			else 
-				throw new DMLException("Failed to parse explain type: "+arg+" " +
-						               "(valid types: hops, runtime, recompile_hops, recompile_runtime).");
-		}
-		
-		return ret;
-	}
-
+	
 	public static String getIdentation( int level ) {
 		return createOffset(level);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/java/org/apache/sysml/yarn/DMLYarnClient.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/yarn/DMLYarnClient.java b/src/main/java/org/apache/sysml/yarn/DMLYarnClient.java
index bd9116d..c4da603 100644
--- a/src/main/java/org/apache/sysml/yarn/DMLYarnClient.java
+++ b/src/main/java/org/apache/sysml/yarn/DMLYarnClient.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.yarn.client.api.YarnClientApplication;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
-
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.parser.ParseException;
 import org.apache.sysml.runtime.DMLRuntimeException;
@@ -429,7 +428,7 @@ public class DMLYarnClient
 			command.append(' ');
 			if( i>0 && _args[i-1].equals("-f") ){
 				command.append(_hdfsDMLScript);
-				command.append(" -config=" + _hdfsDMLConfig);
+				command.append(" -config " + _hdfsDMLConfig);
 			}
 			else if( _args[i].startsWith("-config") ){
 				//ignore because config added with -f

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/resources/scripts/sparkDML.sh
----------------------------------------------------------------------
diff --git a/src/main/resources/scripts/sparkDML.sh b/src/main/resources/scripts/sparkDML.sh
index cd57ae0..a68d34a 100644
--- a/src/main/resources/scripts/sparkDML.sh
+++ b/src/main/resources/scripts/sparkDML.sh
@@ -116,7 +116,7 @@ $SPARK_HOME/bin/spark-submit \
      ${conf} \
      ${SYSTEMML_HOME}/${project.artifactId}-${project.version}.jar \
          -f ${f} \
-         -config=${SYSTEMML_HOME}/SystemML-config.xml \
+         -config ${SYSTEMML_HOME}/SystemML-config.xml \
          -exec hybrid_spark \
          $explain \
          $stats \

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/standalone/runStandaloneSystemML.bat
----------------------------------------------------------------------
diff --git a/src/main/standalone/runStandaloneSystemML.bat b/src/main/standalone/runStandaloneSystemML.bat
index 9275d9e..35ebbc7 100644
--- a/src/main/standalone/runStandaloneSystemML.bat
+++ b/src/main/standalone/runStandaloneSystemML.bat
@@ -46,7 +46,7 @@ set CMD=java %SYSTEMML_STANDALONE_OPTS% ^
      org.apache.sysml.api.DMLScript ^
      -f %1 ^
      -exec singlenode ^
-     -config=SystemML-config.xml ^
+     -config SystemML-config.xml ^
      %ALLBUTFIRST%
 
 :: execute the java command
@@ -72,6 +72,6 @@ GOTO Msg
 :Msg
 ECHO Usage: runStandaloneSystemML.bat ^<dml-filename^> [arguments] [-help]
 ECHO Default Java options (-Xmx4g -Xms4g -Xmn400m) can be overridden by setting SYSTEMML_STANDALONE_OPTS.
-ECHO Script internally invokes 'java [SYSTEMML_STANDALONE_OPTS] -cp ./lib/* -Dlog4j.configuration=file:log4j.properties org.apache.sysml.api.DMLScript -f ^<dml-filename^> -exec singlenode -config=SystemML-config.xml [arguments]'
+ECHO Script internally invokes 'java [SYSTEMML_STANDALONE_OPTS] -cp ./lib/* -Dlog4j.configuration=file:log4j.properties org.apache.sysml.api.DMLScript -f ^<dml-filename^> -exec singlenode -config SystemML-config.xml [arguments]'
 
 :End

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/standalone/runStandaloneSystemML.sh
----------------------------------------------------------------------
diff --git a/src/main/standalone/runStandaloneSystemML.sh b/src/main/standalone/runStandaloneSystemML.sh
index 28000b4..2980080 100644
--- a/src/main/standalone/runStandaloneSystemML.sh
+++ b/src/main/standalone/runStandaloneSystemML.sh
@@ -76,7 +76,7 @@ java ${SYSTEMML_STANDALONE_OPTS} \
 org.apache.sysml.api.DMLScript \
 -f ${SCRIPT_FILE} \
 -exec singlenode \
--config=$CURRENT_PATH"/SystemML-config.xml" \
+-config $CURRENT_PATH"/SystemML-config.xml" \
 $@"
 
 $CMD

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java b/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java
index f3ede65..f7071ba 100644
--- a/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java
+++ b/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java
@@ -32,7 +32,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 
-import org.apache.sysml.lops.Lop;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.spark.sql.SparkSession;
@@ -46,6 +45,7 @@ import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
 import org.apache.sysml.api.MLContext;
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.OptimizerUtils;
+import org.apache.sysml.lops.Lop;
 import org.apache.sysml.parser.DataExpression;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
@@ -56,15 +56,19 @@ import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
 import org.apache.sysml.runtime.io.FrameReader;
 import org.apache.sysml.runtime.io.FrameReaderFactory;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
-import org.apache.sysml.runtime.matrix.data.InputInfo;
-import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
 import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
 import org.apache.sysml.runtime.matrix.data.FrameBlock;
+import org.apache.sysml.runtime.matrix.data.InputInfo;
+import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
 import org.apache.sysml.runtime.matrix.data.OutputInfo;
 import org.apache.sysml.runtime.util.MapReduceTool;
 import org.apache.sysml.test.utils.TestUtils;
 import org.apache.sysml.utils.ParameterBuilder;
 import org.apache.sysml.utils.Statistics;
+import org.apache.wink.json4j.JSONObject;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
 
 
 /**
@@ -826,7 +830,7 @@ public abstract class AutomatedTestBase
 	 * comparison files.
 	 * </p>
 	 * 
-	 * @param configurationName
+	 * @param config
 	 *            test configuration name
 	 * 
 	 */
@@ -841,7 +845,7 @@ public abstract class AutomatedTestBase
 	 * comparison files.
 	 * </p>
 	 * 
-	 * @param configurationName
+	 * @param config
 	 *            test configuration name
 	 * @param cacheDirectory
 	 *            subdirectory for reusing R script expected results
@@ -1219,7 +1223,8 @@ public abstract class AutomatedTestBase
 			throw new RuntimeException("Unknown runtime platform: " + rtplatform);
 		}
 		//use optional config file since default under SystemML/DML
-		args.add("-config="+ getCurConfigFile().getPath());
+		args.add("-config");
+		args.add(getCurConfigFile().getPath());
 		
 		if(TEST_GPU)
 			args.add("-gpu");
@@ -1704,7 +1709,7 @@ public abstract class AutomatedTestBase
 	 * 
 	 * @param name
 	 *            directory name
-	 * @param matrix
+	 * @param data
 	 *            two dimensional frame data
 	 * @param bIncludeR
 	 *            generates also the corresponding R frame data

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/test/java/org/apache/sysml/test/integration/functions/dmlscript/DMLScriptTest1.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/dmlscript/DMLScriptTest1.java b/src/test/java/org/apache/sysml/test/integration/functions/dmlscript/DMLScriptTest1.java
deleted file mode 100644
index c7006df..0000000
--- a/src/test/java/org/apache/sysml/test/integration/functions/dmlscript/DMLScriptTest1.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.sysml.test.integration.functions.dmlscript;
-
-import org.junit.Test;
-
-import org.apache.sysml.test.integration.AutomatedTestBase;
-import org.apache.sysml.test.integration.TestConfiguration;
-
-
-/**
- * <p>
- * <b>Positive tests:</b>
- * </p>
- * <ul>
- * <li>text format</li>
- * <li>binary format</li>
- * </ul>
- * <p>
- * <b>Negative tests:</b>
- * </p>
- * <ul>
- * </ul>
- * 
- * 
- */
-public class DMLScriptTest1 extends AutomatedTestBase 
-{
-	
-	private final static String TEST_DIR = "functions/dmlscript/";
-	private final static String TEST_CLASS_DIR = TEST_DIR + DMLScriptTest1.class.getSimpleName() + "/";
-	private final static String TEST_NAME = "DMLScriptTest";
-	
-	@Override
-	public void setUp() {
-		// positive tests
-		TestConfiguration config = new TestConfiguration(TEST_CLASS_DIR, TEST_NAME, new String[] { "a" });
-		addTestConfiguration(TEST_NAME, config);
-		
-		// negative tests		
-	}
-
-	@Test
-	public void testWithFile() {
-		int rows = 10;
-		int cols = 10;
-
-		TestConfiguration config = getTestConfiguration(TEST_NAME);
-		config.addVariable("rows", rows);
-		config.addVariable("cols", cols);
-		config.addVariable("format", "text");
-		loadTestConfiguration(config);
-		
-		String HOME = SCRIPT_DIR + TEST_DIR;
-		fullDMLScriptName = HOME + "DMLScriptTest.dml";
-		
-		programArgs = new String[]{"-args", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-
-		double[][] a = getRandomMatrix(rows, cols, -1, 1, 0.5, -1);
-		writeInputMatrix("a", a, true);
-
-		runTest(true, false, null, -1);
-
-		programArgs = new String[]{"-args", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-		
-		programArgs = new String[]{"-exec", "hybrid", "-args", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-		
-		programArgs = new String[]{"-exec", "hybrid", "-config=" + HOME + "SystemML-config.xml",
-			"-args", input("a"), Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-	}
-
-	@Test
-	public void testWithString() {
-		String s = " A = read($1, rows=$2, cols=$3, format=$4); \n " + 
-				  "write(A, $5, format=$4); \n";
-		int rows = 10;
-		int cols = 10;
-		String HOME = SCRIPT_DIR + TEST_DIR;
-
-		TestConfiguration config = getTestConfiguration(TEST_NAME);
-		config.addVariable("rows", rows);
-		config.addVariable("cols", cols);
-		config.addVariable("format", "text");
-		loadTestConfiguration(config);
-		
-		programArgs = new String[]{"-s", s, "-args", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-
-		double[][] a = getRandomMatrix(rows, cols, -1, 1, 0.5, -1);
-		writeInputMatrix("a", a, true);
-
-		runTest(true, false, null, -1);
-		
-		programArgs = new String[]{"-s", s, "-args", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-		
-		programArgs = new String[]{"-s", s, "-config=" + HOME + "SystemML-config.xml", "-exec", "hybrid",
-			"-args", input("a"), Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/test/java/org/apache/sysml/test/integration/functions/dmlscript/DMLScriptTest2.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/dmlscript/DMLScriptTest2.java b/src/test/java/org/apache/sysml/test/integration/functions/dmlscript/DMLScriptTest2.java
deleted file mode 100644
index 123146c..0000000
--- a/src/test/java/org/apache/sysml/test/integration/functions/dmlscript/DMLScriptTest2.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * 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.sysml.test.integration.functions.dmlscript;
-
-import org.junit.Test;
-
-import org.apache.sysml.api.DMLException;
-import org.apache.sysml.test.integration.AutomatedTestBase;
-import org.apache.sysml.test.integration.TestConfiguration;
-
-
-/**
- * <p>
- * <b>Positive tests:</b>
- * </p>
- * <ul>
- * <li>text format</li>
- * <li>binary format</li>
- * </ul>
- * <p>
- * <b>Negative tests:</b>
- * </p>
- * <ul>
- * </ul>
- * 
- * 
- */
-public class DMLScriptTest2 extends AutomatedTestBase 
-{
-	
-	private final static String TEST_DIR = "functions/dmlscript/";
-	private final static String TEST_CLASS_DIR = TEST_DIR + DMLScriptTest2.class.getSimpleName() + "/";
-	private final static String TEST_NAME = "DMLScriptTest2";
-	
-	/**
-	 * Main method for running one test at a time.
-	 */
-	public static void main(String[] args) {
-		long startMsec = System.currentTimeMillis();
-
-		DMLScriptTest2 t = new DMLScriptTest2();
-		t.setUpBase();
-		t.setUp();
-		t.testWithString();
-		t.tearDown();
-
-		long elapsedMsec = System.currentTimeMillis() - startMsec;
-		System.err.printf("Finished in %1.3f sec\n", elapsedMsec / 1000.0);
-	}
-	
-	@Override
-	public void setUp() {
-		// positive tests
-		
-		// negative tests
-		TestConfiguration config = new TestConfiguration(TEST_CLASS_DIR, TEST_NAME, new String[] { "a" });
-		addTestConfiguration(TEST_NAME, config);
-	}
-
-	@Test
-	public void testWithFile() {
-		int rows = 10;
-		int cols = 10;
-		String HOME = SCRIPT_DIR + TEST_DIR;
-
-		TestConfiguration config = getTestConfiguration(TEST_NAME);
-		config.addVariable("rows", rows);
-		config.addVariable("cols", cols);
-		config.addVariable("format", "text");
-		loadTestConfiguration(config);
-
-		double[][] a = getRandomMatrix(rows, cols, -1, 1, 0.5, -1);
-		writeInputMatrix("a", a, true);
-		
-		//Expect to print out an ERROR message. -f or -s must be the first argument.
-		fullDMLScriptName = HOME + "DMLScriptTest.dml";
-		programArgs = new String[]{ "-exec", "hybrid", "-args", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-
-		//Expect to print out an ERROR message. -args should be the last argument.
-		programArgs = new String[]{"-args", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a"), "-exec", "hybrid"};
-		runTest(true, true, DMLException.class, -1);
-		
-		//Expect to print out an ERROR message, -de is an unknown argument
-		programArgs = new String[]{"-de", "-exec", "hybrid", "-config=" + HOME + "SystemML-config.xml",
-			"-args", input("a"), Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-		
-		//Expect to print out an ERROR message, -config syntax is -config=<config file>
-		programArgs = new String[]{"-exec", "hybrid", "-config", HOME + "SystemML-config.xml",
-			"-args", input("a"), Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-	}
-
-	@Test
-	public void testWithString() {
-		String s = " A = read($1, rows=$2, cols=$3, format=$4); \n " + 
-				  "write(A, $5, format=$4); \n";
-		int rows = 10;
-		int cols = 10;
-		String HOME = SCRIPT_DIR + TEST_DIR;
-
-		TestConfiguration config = availableTestConfigurations.get("DMLScriptTest2");
-		config.addVariable("rows", rows);
-		config.addVariable("cols", cols);
-		config.addVariable("format", "text");
-		loadTestConfiguration(config);
-
-		double[][] a = getRandomMatrix(rows, cols, -1, 1, 0.5, -1);
-		writeInputMatrix("a", a, true);
-		
-		//Expect to print out an ERROR message. -f or -s must be the first argument.
-		programArgs = new String[]{ "-v", "-s", s, "-args", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-		
-		//Expect to print out an ERROR message. -args should be the last argument.
-		programArgs = new String[]{"-s", s, "-args", "-v", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-		
-		//Expect to print out an ERROR message, -de is an unknown argument
-		programArgs = new String[]{"-s", s, "-de", "-args", input("a"),
-			Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-		
-		//Expect to print out an ERROR message, -config syntax is -config=<config file>
-		programArgs = new String[]{"-s", s, "-config", HOME + "SystemML-config.xml", "-exec", "hybrid",
-			"-args", input("a"), Integer.toString(rows), Integer.toString(cols), "text", output("a")};
-		runTest(true, false, null, -1);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/test/java/org/apache/sysml/test/integration/functions/misc/DataTypeChangeTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/misc/DataTypeChangeTest.java b/src/test/java/org/apache/sysml/test/integration/functions/misc/DataTypeChangeTest.java
index 899037c..6462969 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/misc/DataTypeChangeTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/misc/DataTypeChangeTest.java
@@ -148,11 +148,7 @@ public class DataTypeChangeTest extends AutomatedTestBase
 	public void testDataTypeChangeValidate4f() { runTest("dt_change_4f", false); }
 	
 	
-	/**
-	 * 
-	 * @param cfc
-	 * @param vt
-	 */
+
 	private void runTest( String testName, boolean exceptionExpected ) 
 	{
         String RI_HOME = SCRIPT_DIR + TEST_DIR;
@@ -166,11 +162,7 @@ public class DataTypeChangeTest extends AutomatedTestBase
 		runTest(true, exceptionExpected, DMLException.class, -1);
 	}
 	
-	/**
-	 * 
-	 * @param scriptFilename
-	 * @param expectedException
-	 */
+
 	private void runValidateTest( String fullTestName, boolean expectedException )
 	{
 		boolean raisedException = false;
@@ -197,7 +189,7 @@ public class DataTypeChangeTest extends AutomatedTestBase
 			}	
 			
 			//parsing and dependency analysis
-			ParserWrapper parser = ParserFactory.createParser(false);
+			ParserWrapper parser = ParserFactory.createParser(org.apache.sysml.api.mlcontext.ScriptType.DML);
 			DMLProgram prog = parser.parse(DMLScript.DML_FILE_PATH_ANTLR_PARSER, dmlScriptString, argVals);
 			DMLTranslator dmlt = new DMLTranslator(prog);
 			dmlt.liveVariableAnalysis(prog);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/test/java/org/apache/sysml/test/integration/functions/parfor/ParForDependencyAnalysisTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/parfor/ParForDependencyAnalysisTest.java b/src/test/java/org/apache/sysml/test/integration/functions/parfor/ParForDependencyAnalysisTest.java
index 76d7ffa..049b704 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/parfor/ParForDependencyAnalysisTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/parfor/ParForDependencyAnalysisTest.java
@@ -351,7 +351,7 @@ public class ParForDependencyAnalysisTest extends AutomatedTestBase
 			}	
 			
 			//parsing and dependency analysis
-			ParserWrapper parser = ParserFactory.createParser(false);
+			ParserWrapper parser = ParserFactory.createParser(org.apache.sysml.api.mlcontext.ScriptType.DML);
 			DMLProgram prog = parser.parse(DMLScript.DML_FILE_PATH_ANTLR_PARSER, dmlScriptString, argVals);
 			DMLTranslator dmlt = new DMLTranslator(prog);
 			dmlt.validateParseTree(prog);	

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java
index 68b434b..a879356 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java
@@ -19,8 +19,6 @@
 
 package org.apache.sysml.test.integration.functions.transform;
 
-import org.junit.Assert;
-import org.junit.Test;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
 import org.apache.sysml.hops.OptimizerUtils;
@@ -34,6 +32,8 @@ import org.apache.sysml.test.integration.AutomatedTestBase;
 import org.apache.sysml.test.integration.TestConfiguration;
 import org.apache.sysml.test.utils.TestUtils;
 import org.apache.sysml.utils.Statistics;
+import org.junit.Assert;
+import org.junit.Test;
 
 public class TransformFrameEncodeDecodeTest extends AutomatedTestBase 
 {
@@ -161,8 +161,15 @@ public class TransformFrameEncodeDecodeTest extends AutomatedTestBase
 				"DATA=" + HOME + "input/" + DATASET,
 				"TFSPEC=" + HOME + "input/" + SPEC,
 				"TFDATA=" + output("tfout"), "SEP=,",
-				"OFMT=" + ofmt, "OSEP=\",\"" };
-	
+				"OFMT=" + ofmt, "OSEP=," };
+
+			// Originally OSEP was set to
+			// OSEP=","
+			// Apache Commons CLI strips away the leading and trailing quotes, leaving us with
+			// OSEP=",
+			// This is just a feature/bug and is reported in CLI-262,
+			// though even a fix is unlikely to be backported to 1.2
+
 			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
 			runTest(true, false, null, -1); 
 			

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/test/java/org/apache/sysml/test/unit/CLIOptionsParserTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/unit/CLIOptionsParserTest.java b/src/test/java/org/apache/sysml/test/unit/CLIOptionsParserTest.java
new file mode 100644
index 0000000..8018bcf
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/unit/CLIOptionsParserTest.java
@@ -0,0 +1,415 @@
+/*
+ * 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.sysml.test.unit;
+
+import java.util.Map;
+
+import org.apache.commons.cli.AlreadySelectedException;
+import org.apache.commons.cli.MissingOptionException;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.api.mlcontext.ScriptType;
+import org.apache.sysml.utils.Explain;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class CLIOptionsParserTest {
+
+  @Test(expected = MissingOptionException.class)
+  public void testNoOptions() throws Exception {
+    String cl = "systemml";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  @Test
+  public void testFile() throws Exception {
+    String cl = "systemml -f test.dml";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals("test.dml", o.filePath);
+    Assert.assertEquals(ScriptType.DML, o.scriptType);
+
+  }
+
+  @Test
+  public void testScript() throws Exception {
+    String cl = "systemml -s \"print('hello')\"";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals("print('hello')", o.script);
+  }
+
+  @Test
+  public void testConfig() throws Exception {
+    String cl = "systemml -s \"print('hello')\" -config SystemML-config.xml";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals("print('hello')", o.script);
+    Assert.assertEquals("SystemML-config.xml", o.configFile);
+  }
+
+  @Test
+  public void testDebug() throws Exception {
+    String cl = "systemml -s \"print('hello')\" -debug";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals("print('hello')", o.script);
+    Assert.assertEquals(true, o.debug);
+  }
+
+  @Test
+  public void testClean() throws Exception {
+    String cl = "systemml -clean";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(true, o.clean);
+  }
+
+  @Test(expected = AlreadySelectedException.class)
+  public void testBadClean() throws Exception {
+    String cl = "systemml -clean -f test.dml";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  @Test(expected = AlreadySelectedException.class)
+  public void testBadScript() throws Exception {
+    String cl = "systemml -f test.dml -s \"print('hello')\"";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  @Test
+  public void testStats() throws Exception {
+    String cl = "systemml -f test.dml -stats";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(true, o.stats);
+    Assert.assertEquals(10, o.statsCount);
+  }
+
+  @Test
+  public void testStatsCount() throws Exception {
+    String cl = "systemml -f test.dml -stats 9123";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(true, o.stats);
+    Assert.assertEquals(9123, o.statsCount);
+  }
+
+  @Test(expected = ParseException.class)
+  public void testBadStats() throws Exception {
+    String cl = "systemml -f test.dml -stats help";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(true, o.stats);
+  }
+
+  @Test
+  public void testGPUForce() throws Exception {
+    String cl = "systemml -f test.dml -gpu force";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(true, o.gpu);
+    Assert.assertEquals(true, o.forceGPU);
+  }
+
+  @Test(expected = ParseException.class)
+  public void testBadGPUOption() throws Exception {
+    String cl = "systemml -f test.dml -gpu f2orce";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  @Test
+  public void testPython() throws Exception {
+    String cl = "systemml -f test.dml -python";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(ScriptType.PYDML, o.scriptType);
+  }
+
+  @Test
+  public void testHelp() throws Exception {
+    String cl = "systemml -help";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(true, o.help);
+  }
+
+  @Test(expected = AlreadySelectedException.class)
+  public void testBadHelp() throws Exception {
+    String cl = "systemml -help -clean";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(true, o.help);
+  }
+
+  @Test
+  public void testExplain1() throws Exception {
+    String cl = "systemml -f test.dml -explain";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(Explain.ExplainType.RUNTIME, o.explainType);
+  }
+
+  @Test
+  public void testExplain2() throws Exception {
+    String cl = "systemml -f test.dml -explain hops";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(Explain.ExplainType.HOPS, o.explainType);
+  }
+
+  @Test
+  public void testExplain3() throws Exception {
+    String cl = "systemml -f test.dml -explain runtime";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(Explain.ExplainType.RUNTIME, o.explainType);
+  }
+
+  @Test
+  public void testExplain4() throws Exception {
+    String cl = "systemml -f test.dml -explain recompile_hops";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(Explain.ExplainType.RECOMPILE_HOPS, o.explainType);
+  }
+
+  @Test
+  public void testExplain5() throws Exception {
+    String cl = "systemml -f test.dml -explain recompile_runtime";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(Explain.ExplainType.RECOMPILE_RUNTIME, o.explainType);
+  }
+
+  @Test
+  public void testExec1() throws Exception {
+    String cl = "systemml -f test.dml -exec hadoop";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(DMLScript.RUNTIME_PLATFORM.HADOOP, o.execMode);
+  }
+
+  @Test
+  public void testExec2() throws Exception {
+    String cl = "systemml -f test.dml -exec spark";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(DMLScript.RUNTIME_PLATFORM.SPARK, o.execMode);
+  }
+
+  @Test
+  public void testExec3() throws Exception {
+    String cl = "systemml -f test.dml -exec singlenode";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(DMLScript.RUNTIME_PLATFORM.SINGLE_NODE, o.execMode);
+  }
+
+  @Test
+  public void testExec4() throws Exception {
+    String cl = "systemml -f test.dml -exec hybrid";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(DMLScript.RUNTIME_PLATFORM.HYBRID, o.execMode);
+  }
+
+  @Test
+  public void testExec5() throws Exception {
+    String cl = "systemml -f test.dml -exec hybrid_spark";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Assert.assertEquals(DMLScript.RUNTIME_PLATFORM.HYBRID_SPARK, o.execMode);
+  }
+
+  @Test(expected = ParseException.class)
+  public void testBadExec() throws Exception {
+    String cl = "systemml -f test.dml -exec new_system";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  @Test
+  public void testArgs1() throws Exception {
+    String cl = "systemml -f test.dml -args 10 \"x.csv\"";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Map<String, String> m = o.argVals;
+    Assert.assertEquals(2, m.size());
+    Assert.assertEquals("10", m.get("$1"));
+    Assert.assertEquals("x.csv", m.get("$2"));
+  }
+
+  @Test
+  public void testArgs2() throws Exception {
+    String cl = "systemml -f test.dml -args 10 \"x.csv\" 1234.2 systemml.conf -config systemml.conf";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Map<String, String> m = o.argVals;
+    Assert.assertEquals(4, m.size());
+    Assert.assertEquals("10", m.get("$1"));
+    Assert.assertEquals("x.csv", m.get("$2"));
+    Assert.assertEquals("1234.2", m.get("$3"));
+    Assert.assertEquals("systemml.conf", m.get("$4"));
+  }
+
+  @Test(expected = ParseException.class)
+  public void testBadArgs1() throws Exception {
+    String cl = "systemml -f test.dml -args -config systemml.conf";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  @Test
+  public void testNVArgs1() throws Exception {
+    String cl = "systemml -f test.dml -nvargs A=12 B=x.csv my123=12.2";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Map<String, String> m = o.argVals;
+    Assert.assertEquals(3, m.size());
+    Assert.assertEquals("12", m.get("$A"));
+    Assert.assertEquals("x.csv", m.get("$B"));
+    Assert.assertEquals("12.2", m.get("$my123"));
+  }
+
+  @Test(expected = ParseException.class)
+  public void testBadNVArgs1() throws Exception {
+    String cl = "systemml -f test.dml -nvargs";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  @Test(expected = ParseException.class)
+  public void testBadNVArgs2() throws Exception {
+    String cl = "systemml -f test.dml -nvargs asd qwe";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  @Test(expected = ParseException.class)
+  public void testBadNVArgs3() throws Exception {
+    String cl = "systemml -f test.dml -nvargs $X=12";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  @Test(expected = ParseException.class)
+  public void testBadNVArgs4() throws Exception {
+    String cl = "systemml -f test.dml -nvargs 123=123";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.parseCLArguments(args, options);
+  }
+
+  /**
+   * For Apache Commons CLI, if an argument to an option is enclosed in quotes,
+   * the leading and trailing quotes are stripped away. For instance, if the options is -arg and the
+   * argument is "foo"
+   *  -args "foo"
+   * Commons CLI will strip the quotes from "foo". This becomes troublesome when you really do
+   * want to pass in "foo" and not just foo.
+   * A way around this is to use 'foo` as done in {@link CLIOptionsParserTest#testNVArgs3()}
+   */
+  @Test
+  public void testNVArgs2() throws Exception {
+    String cl = "systemml -f test.dml -args \"def\"";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Map<String, String> m = o.argVals;
+    Assert.assertEquals("def", m.get("$1"));
+  }
+
+
+  /**
+   * See comment in {@link CLIOptionsParserTest#testNVArgs2()}
+   */
+  @Test
+  public void testNVArgs3() throws Exception {
+    String cl = "systemml -f test.dml -args 'def'";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Map<String, String> m = o.argVals;
+    Assert.assertEquals("'def'", m.get("$1"));
+  }
+
+  /**
+   * See comment in {@link CLIOptionsParserTest#testNVArgs2()}
+   * Additionally, if we try to pass something like
+   * -nvargs X="foo"
+   * Commons CLI will strip the leading and trailing quotes (viz. double quotes), which
+   * causes it to return
+   * X="foo
+   * The way to overcome this is to enclose the <value> of the <key=value> pair in single quotes
+   * and strip them away in the parsing code ourselves.
+   * TODO: Read the javadoc for this method, we can add in this logic if required
+   */
+  @Test
+  public void testNVArgs4() throws Exception {
+    String cl = "systemml -f test.dml -nvargs abc='def'";
+    String[] args = cl.split(" ");
+    Options options = DMLScript.createCLIOptions();
+    DMLScript.DMLOptions o = DMLScript.parseCLArguments(args, options);
+    Map<String, String> m = o.argVals;
+    Assert.assertEquals("'def'", m.get("$abc"));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/test_suites/java/org/apache/sysml/test/integration/functions/dmlscript/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/dmlscript/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/dmlscript/ZPackageSuite.java
deleted file mode 100644
index 713c46d..0000000
--- a/src/test_suites/java/org/apache/sysml/test/integration/functions/dmlscript/ZPackageSuite.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.sysml.test.integration.functions.dmlscript;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
-/** Group together the tests in this package into a single suite so that the Maven build
- *  won't run two of them at once. */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-	DMLScriptTest1.class,
-	DMLScriptTest2.class
-})
-
-
-/** This class is just a holder for the above JUnit annotations. */
-public class ZPackageSuite {
-
-}


[2/2] incubator-systemml git commit: Use Apache Commons CLI to parse command line arguments in DMLScript

Posted by na...@apache.org.
Use Apache Commons CLI to parse command line arguments in DMLScript

- Added unit tests
- changed scripts to accept "-config " instead of "-config="
- Removed DMLScriptTest{1,2}
- Modified bin/systemml script to print a better help message
- Removed extraneous ZPackageSuite for DMLScriptTest{1,2}

Closes #440


Project: http://git-wip-us.apache.org/repos/asf/incubator-systemml/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-systemml/commit/32924dc6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-systemml/tree/32924dc6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-systemml/diff/32924dc6

Branch: refs/heads/master
Commit: 32924dc6027df1973b37a8688b7d0cacbdefd4bf
Parents: f73673d
Author: Nakul Jindal <na...@gmail.com>
Authored: Fri Apr 21 14:50:56 2017 -0700
Committer: Nakul Jindal <na...@gmail.com>
Committed: Fri Apr 21 14:50:57 2017 -0700

----------------------------------------------------------------------
 bin/systemml                                    |  84 +--
 bin/systemml.bat                                |   4 +-
 docs/algorithms-classification.md               |  44 +-
 docs/algorithms-clustering.md                   |  14 +-
 docs/algorithms-descriptive-statistics.md       |  14 +-
 docs/algorithms-matrix-factorization.md         |  18 +-
 docs/algorithms-regression.md                   |  36 +-
 docs/algorithms-survival-analysis.md            |  16 +-
 docs/hadoop-batch-mode.md                       |  16 +-
 docs/spark-batch-mode.md                        |   4 +-
 docs/standalone-guide.md                        |   2 +-
 docs/troubleshooting-guide.md                   |   2 +-
 scripts/sparkDML.sh                             |   2 +-
 .../java/org/apache/sysml/api/DMLScript.java    | 576 ++++++++++++-------
 .../java/org/apache/sysml/api/MLContext.java    |  49 +-
 .../org/apache/sysml/api/jmlc/Connection.java   |   2 +-
 .../sysml/api/mlcontext/ScriptExecutor.java     |   2 +-
 .../org/apache/sysml/parser/ParserFactory.java  |  16 +-
 .../java/org/apache/sysml/utils/Explain.java    |  32 +-
 .../org/apache/sysml/yarn/DMLYarnClient.java    |   3 +-
 src/main/resources/scripts/sparkDML.sh          |   2 +-
 src/main/standalone/runStandaloneSystemML.bat   |   4 +-
 src/main/standalone/runStandaloneSystemML.sh    |   2 +-
 .../test/integration/AutomatedTestBase.java     |  19 +-
 .../functions/dmlscript/DMLScriptTest1.java     | 125 ----
 .../functions/dmlscript/DMLScriptTest2.java     | 151 -----
 .../functions/misc/DataTypeChangeTest.java      |  14 +-
 .../parfor/ParForDependencyAnalysisTest.java    |   2 +-
 .../TransformFrameEncodeDecodeTest.java         |  15 +-
 .../sysml/test/unit/CLIOptionsParserTest.java   | 415 +++++++++++++
 .../functions/dmlscript/ZPackageSuite.java      |  37 --
 31 files changed, 992 insertions(+), 730 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/bin/systemml
----------------------------------------------------------------------
diff --git a/bin/systemml b/bin/systemml
index 0ccee2d..44ab45e 100755
--- a/bin/systemml
+++ b/bin/systemml
@@ -20,32 +20,22 @@
 #
 #-------------------------------------------------------------
 
+
 # error help print
-printUsageExit()
+printSimpleUsage()
 {
 cat << EOF
 Usage: $0 <dml-filename> [arguments] [-help]
-    -help     - Print this usage message and exit
+    -help     - Print detailed help message
 EOF
   exit 1
 }
-#    Script internally invokes 'java -Xmx4g -Xms4g -Xmn400m [Custom-Java-Options] -jar StandaloneSystemML.jar -f <dml-filename> -exec singlenode -config=SystemML-config.xml [Optional-Arguments]'
 
-while getopts "h:" options; do
-  case $options in
-    h ) echo Warning: Help requested. Will exit after usage message
-        printUsageExit
-        ;;
-    \? ) echo Warning: Help requested. Will exit after usage message
-        printUsageExit
-        ;;
-    * ) echo Error: Unexpected error while processing options
-  esac
-done
+#    Script internally invokes 'java -Xmx4g -Xms4g -Xmn400m [Custom-Java-Options] -jar StandaloneSystemML.jar -f <dml-filename> -exec singlenode -config=SystemML-config.xml [Optional-Arguments]'
 
 if [ -z "$1" ] ; then
     echo "Wrong Usage.";
-    printUsageExit;
+    printSimpleUsage
 fi
 
 
@@ -98,24 +88,6 @@ then
 fi
 
 
-# Peel off first argument so that $@ contains arguments to DML script
-SCRIPT_FILE=$1
-shift
-
-# if the script file path was omitted, try to complete the script path
-if [ ! -f "$SCRIPT_FILE" ]
-then
-  SCRIPT_FILE_NAME=$(basename $SCRIPT_FILE)
-  SCRIPT_FILE_FOUND=$(find "$PROJECT_ROOT_DIR/scripts" -name "$SCRIPT_FILE_NAME")
-  if [ ! "$SCRIPT_FILE_FOUND" ]
-  then
-    echo "Could not find DML script: $SCRIPT_FILE"
-    printUsageExit;
-  else
-    SCRIPT_FILE=$SCRIPT_FILE_FOUND
-    echo "DML script: $SCRIPT_FILE"
-  fi
-fi
 
 
 # add hadoop libraries which were generated by the build to the classpath
@@ -149,13 +121,57 @@ if [ -f "${PROJECT_ROOT_DIR}/conf/systemml-env.sh" ]; then
     fi
 fi
 
+
+printUsageExit()
+{
+CMD="\
+java ${SYSTEMML_DEFAULT_JAVA_OPTS} \
+org.apache.sysml.api.DMLScript \
+-help"
+# echo ${CMD}
+eval ${CMD}
+exit 0
+}
+
+while getopts "h:" options; do
+  case $options in
+    h ) echo Warning: Help requested. Will exit after usage message
+        printUsageExit
+        ;;
+    \? ) echo Warning: Help requested. Will exit after usage message
+        printUsageExit
+        ;;
+    * ) echo Error: Unexpected error while processing options
+  esac
+done
+
+# Peel off first argument so that $@ contains arguments to DML script
+SCRIPT_FILE=$1
+shift
+
+# if the script file path was omitted, try to complete the script path
+if [ ! -f "$SCRIPT_FILE" ]
+then
+  SCRIPT_FILE_NAME=$(basename $SCRIPT_FILE)
+  SCRIPT_FILE_FOUND=$(find "$PROJECT_ROOT_DIR/scripts" -name "$SCRIPT_FILE_NAME")
+  if [ ! "$SCRIPT_FILE_FOUND" ]
+  then
+    echo "Could not find DML script: $SCRIPT_FILE"
+    printSimpleUsage
+  else
+    SCRIPT_FILE=$SCRIPT_FILE_FOUND
+    echo "DML script: $SCRIPT_FILE"
+  fi
+fi
+
+
 # Invoke the jar with options and arguments
 CMD="\
 java ${SYSTEMML_DEFAULT_JAVA_OPTS} \
 org.apache.sysml.api.DMLScript \
 -f '$SCRIPT_FILE' \
 -exec singlenode \
--config='$PROJECT_ROOT_DIR/conf/SystemML-config.xml' \
+-config '$PROJECT_ROOT_DIR/conf/SystemML-config.xml' \
 $@"
 
 eval ${CMD}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/bin/systemml.bat
----------------------------------------------------------------------
diff --git a/bin/systemml.bat b/bin/systemml.bat
index 3fc86a8..e16a2a1 100755
--- a/bin/systemml.bat
+++ b/bin/systemml.bat
@@ -117,7 +117,7 @@ set CMD=java -Xmx4g -Xms2g -Xmn400m ^
      org.apache.sysml.api.DMLScript ^
      -f %SCRIPT_FILE% ^
      -exec singlenode ^
-     -config="%PROJECT_ROOT_DIR%\conf\SystemML-config.xml" ^
+     -config "%PROJECT_ROOT_DIR%\conf\SystemML-config.xml" ^
      %DML_OPT_ARGS%
 
 :: execute the java command
@@ -141,7 +141,7 @@ GOTO Msg
 
 :Msg
 ECHO Usage: runStandaloneSystemML.bat ^<dml-filename^> [arguments] [-help]
-ECHO Script internally invokes 'java -Xmx4g -Xms4g -Xmn400m -jar jSystemML.jar -f ^<dml-filename^> -exec singlenode -config=SystemML-config.xml [Optional-Arguments]'
+ECHO Script internally invokes 'java -Xmx4g -Xms4g -Xmn400m -jar jSystemML.jar -f ^<dml-filename^> -exec singlenode -config SystemML-config.xml [Optional-Arguments]'
 GOTO ExitErr
 
 :ExitErr

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/algorithms-classification.md
----------------------------------------------------------------------
diff --git a/docs/algorithms-classification.md b/docs/algorithms-classification.md
index b029e0a..ed56c34 100644
--- a/docs/algorithms-classification.md
+++ b/docs/algorithms-classification.md
@@ -165,7 +165,7 @@ val prediction = model.transform(X_test_df)
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f MultiLogReg.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -336,7 +336,7 @@ prediction.show()
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f MultiLogReg.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx
@@ -532,7 +532,7 @@ val model = svm.fit(X_train_df)
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f l2-svm.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -579,7 +579,7 @@ val prediction = model.transform(X_test_df)
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f l2-svm-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=[file]
@@ -661,7 +661,7 @@ using a held-out test set. Note that this is an optional argument.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f l2-svm.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/y.mtx
@@ -695,7 +695,7 @@ using a held-out test set. Note that this is an optional argument.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f l2-svm-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/y.mtx
@@ -800,7 +800,7 @@ val model = svm.fit(X_train_df)
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f m-svm.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -847,7 +847,7 @@ val prediction = model.transform(X_test_df)
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f m-svm-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=[file]
@@ -1014,7 +1014,7 @@ prediction.show()
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f m-svm.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/y.mtx
@@ -1048,7 +1048,7 @@ prediction.show()
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f m-svm-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/y.mtx
@@ -1153,7 +1153,7 @@ val model = nb.fit(X_train_df)
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f naive-bayes.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -1198,7 +1198,7 @@ val prediction = model.transform(X_test_df)
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f naive-bayes-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=[file]
@@ -1289,7 +1289,7 @@ metrics.f1_score(newsgroups_test.target, pred, average='weighted')
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f naive-bayes.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/y.mtx
@@ -1321,7 +1321,7 @@ metrics.f1_score(newsgroups_test.target, pred, average='weighted')
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f naive-bayes-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/y.mtx
@@ -1420,7 +1420,7 @@ implementation is well-suited to handle large-scale data and builds a
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f decision-tree.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -1458,7 +1458,7 @@ implementation is well-suited to handle large-scale data and builds a
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f decision-tree-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=[file]
@@ -1558,7 +1558,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f decision-tree.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx
@@ -1593,7 +1593,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f decision-tree-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx
@@ -1828,7 +1828,7 @@ for classification in parallel.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f random-forest.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -1871,7 +1871,7 @@ for classification in parallel.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f random-forest-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=[file]
@@ -1994,7 +1994,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f random-forest.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx
@@ -2032,7 +2032,7 @@ To compute predictions:
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f random-forest-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/algorithms-clustering.md
----------------------------------------------------------------------
diff --git a/docs/algorithms-clustering.md b/docs/algorithms-clustering.md
index 023a7f5..0c91fa1 100644
--- a/docs/algorithms-clustering.md
+++ b/docs/algorithms-clustering.md
@@ -134,7 +134,7 @@ apart is a "false negative"�etc.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Kmeans.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          C=[file]
@@ -168,7 +168,7 @@ apart is a "false negative"�etc.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Kmeans-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=[file]
                                          C=[file]
@@ -260,7 +260,7 @@ standard output
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Kmeans.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          k=5
@@ -289,7 +289,7 @@ standard output
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Kmeans.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          k=5
@@ -322,7 +322,7 @@ To predict Y given X and C:
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Kmeans-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          C=/user/ml/C.mtx
@@ -348,7 +348,7 @@ given X and C:
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Kmeans-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          C=/user/ml/C.mtx
@@ -373,7 +373,7 @@ labels prY:
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Kmeans-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs spY=/user/ml/Y.mtx
                                          prY=/user/ml/PredY.mtx

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/algorithms-descriptive-statistics.md
----------------------------------------------------------------------
diff --git a/docs/algorithms-descriptive-statistics.md b/docs/algorithms-descriptive-statistics.md
index 1ddf01a..f45ffae 100644
--- a/docs/algorithms-descriptive-statistics.md
+++ b/docs/algorithms-descriptive-statistics.md
@@ -130,7 +130,7 @@ to compute the mean of a categorical attribute like \u2018Hair Color\u2019.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Univar-Stats.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          TYPES=<file>
@@ -169,7 +169,7 @@ be stored. The format of the output matrix is defined by
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Univar-Stats.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          TYPES=/user/ml/types.mtx
@@ -590,7 +590,7 @@ attributes like \u2018Hair Color\u2019.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f bivar-stats.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          index1=<file>
@@ -659,7 +659,7 @@ are defined in [**Table�2**](algorithms-descriptive-statistics.html#table2).
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f bivar-stats.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          index1=/user/ml/S1.mtx
@@ -1152,7 +1152,7 @@ becomes reversed and amplified (from $+0.1$ to $-0.5$) if we ignore the months.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f stratstats.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Xcid=[file]
@@ -1360,7 +1360,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f stratstats.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Xcid=/user/ml/Xcid.mtx
@@ -1388,7 +1388,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f stratstats.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/Data.mtx
                                          Xcid=/user/ml/Xcid.mtx

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/algorithms-matrix-factorization.md
----------------------------------------------------------------------
diff --git a/docs/algorithms-matrix-factorization.md b/docs/algorithms-matrix-factorization.md
index 51eb614..9af8c19 100644
--- a/docs/algorithms-matrix-factorization.md
+++ b/docs/algorithms-matrix-factorization.md
@@ -61,7 +61,7 @@ top-$K$ (for a given value of $K$) principal components.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f PCA.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs INPUT=<file>
                                          K=<int>
@@ -124,7 +124,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f PCA.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs INPUT=/user/ml/input.mtx
                                          K=10
@@ -154,7 +154,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f PCA.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs INPUT=/user/ml/test_input.mtx
                                          K=10
@@ -262,7 +262,7 @@ problems.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f ALS.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs V=<file>
                                          L=<file>
@@ -296,7 +296,7 @@ problems.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f ALS_predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -327,7 +327,7 @@ problems.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f ALS_topk_predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -436,7 +436,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f ALS.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs V=/user/ml/V
                                          L=/user/ml/L
@@ -472,7 +472,7 @@ To compute predicted ratings for a given list of users and items:
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f ALS_predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X
                                          Y=/user/ml/Y
@@ -506,7 +506,7 @@ predicted ratings for a given list of users:
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f ALS_topk_predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X
                                          Y=/user/ml/Y

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/algorithms-regression.md
----------------------------------------------------------------------
diff --git a/docs/algorithms-regression.md b/docs/algorithms-regression.md
index 31f7ec2..494693c 100644
--- a/docs/algorithms-regression.md
+++ b/docs/algorithms-regression.md
@@ -107,7 +107,7 @@ y_test = lr.fit(df_train)
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f LinearRegDS.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -152,7 +152,7 @@ y_test = lr.fit(df_train)
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f LinearRegCG.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -258,7 +258,7 @@ print("Residual sum of squares: %.2f" % np.mean((regr.predict(diabetes_X_test) -
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f LinearRegDS.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx
@@ -316,7 +316,7 @@ print("Residual sum of squares: %.2f" % np.mean((regr.predict(diabetes_X_test) -
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f LinearRegCG.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx
@@ -557,7 +557,7 @@ lowest AIC is computed.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f StepLinearRegDS.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -628,7 +628,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f StepLinearRegDS.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx
@@ -760,7 +760,7 @@ distributions and link functions, see below for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -898,7 +898,7 @@ if no maximum limit provided
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx
@@ -1235,7 +1235,7 @@ distribution family is supported (see below for details).
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f StepGLM.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=<file>
@@ -1340,7 +1340,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f StepGLM.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          Y=/user/ml/Y.mtx
@@ -1486,7 +1486,7 @@ this step outside the scope of `GLM-predict.dml` for now.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          Y=[file]
@@ -1625,7 +1625,7 @@ unknown (which sets it to�`1.0`).
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs dfam=1
                                          vpow=0.0
@@ -1661,7 +1661,7 @@ unknown (which sets it to�`1.0`).
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs dfam=1
                                          vpow=0.0
@@ -1695,7 +1695,7 @@ unknown (which sets it to�`1.0`).
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs dfam=2
                                          link=2
@@ -1730,7 +1730,7 @@ unknown (which sets it to�`1.0`).
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs dfam=2
                                          link=3
@@ -1763,7 +1763,7 @@ unknown (which sets it to�`1.0`).
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs dfam=3
                                          X=/user/ml/X.mtx
@@ -1798,7 +1798,7 @@ unknown (which sets it to�`1.0`).
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs dfam=1
                                          vpow=1.0
@@ -1837,7 +1837,7 @@ unknown (which sets it to�`1.0`).
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f GLM-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs dfam=1
                                          vpow=2.0

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/algorithms-survival-analysis.md
----------------------------------------------------------------------
diff --git a/docs/algorithms-survival-analysis.md b/docs/algorithms-survival-analysis.md
index a5e641e..239ab08 100644
--- a/docs/algorithms-survival-analysis.md
+++ b/docs/algorithms-survival-analysis.md
@@ -62,7 +62,7 @@ censored and uncensored survival times.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f KM.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          TE=<file>
@@ -157,7 +157,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f KM.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          TE=/user/ml/TE
@@ -194,7 +194,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f KM.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          TE=/user/ml/TE
@@ -466,7 +466,7 @@ may be categorical (ordinal or nominal) as well as continuous-valued.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Cox.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          TE=<file>
@@ -508,7 +508,7 @@ may be categorical (ordinal or nominal) as well as continuous-valued.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Cox-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=<file>
                                          RT=<file>
@@ -617,7 +617,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Cox.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          TE=/user/ml/TE
@@ -656,7 +656,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Cox.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X.mtx
                                          TE=/user/ml/TE
@@ -696,7 +696,7 @@ SystemML Language Reference for details.
                                  --conf spark.akka.frameSize=128
                                  SystemML.jar
                                  -f Cox-predict.dml
-                                 -config=SystemML-config.xml
+                                 -config SystemML-config.xml
                                  -exec hybrid_spark
                                  -nvargs X=/user/ml/X-sorted.mtx
                                          RT=/user/ml/recoded-timestamps.csv

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/hadoop-batch-mode.md
----------------------------------------------------------------------
diff --git a/docs/hadoop-batch-mode.md b/docs/hadoop-batch-mode.md
index ddc1c1f..3af7c0c 100644
--- a/docs/hadoop-batch-mode.md
+++ b/docs/hadoop-batch-mode.md
@@ -49,11 +49,11 @@ refer to the Hadoop documentation.
 
 SystemML can be invoked in Hadoop Batch mode using the following syntax:
 
-    hadoop jar SystemML.jar [-? | -help | -f <filename>] (-config=<config_filename>) ([-args | -nvargs] <args-list>)
+    hadoop jar SystemML.jar [-? | -help | -f <filename>] (-config <config_filename>) ([-args | -nvargs] <args-list>)
 
 The `SystemML.jar` file is specified to Hadoop using the `jar` option.
 The DML script to invoke is specified after the `-f` argument. Configuration settings can be passed to SystemML
-using the optional `-config=` argument. DML scripts can optionally take named arguments (`-nvargs`) or positional
+using the optional `-config ` argument. DML scripts can optionally take named arguments (`-nvargs`) or positional
 arguments (`-args`). Named arguments are preferred over positional arguments. Positional arguments are considered
 to be deprecated. All the primary algorithm scripts included with SystemML use named arguments.
 
@@ -69,11 +69,11 @@ to be deprecated. All the primary algorithm scripts included with SystemML use n
 
 In a clustered environment, it is *highly* recommended that SystemML configuration settings are specified
 in a `SystemML-config.xml` file. By default, SystemML will look for this file in the current working
-directory (`./SystemML-config.xml`). This location can be overridden by the `-config=` argument.
+directory (`./SystemML-config.xml`). This location can be overridden by the `-config ` argument.
 
 **Example #3: DML Invocation with Configuration File Explicitly Specified and Named Arguments**
 
-	hadoop jar systemml/SystemML.jar -f systemml/algorithms/Kmeans.dml -config=/conf/SystemML-config.xml -nvargs X=X.mtx k=5
+	hadoop jar systemml/SystemML.jar -f systemml/algorithms/Kmeans.dml -config /conf/SystemML-config.xml -nvargs X=X.mtx k=5
 
 For recommended SystemML configuration settings in a clustered environment, please see
 [Recommended Hadoop Cluster Configuration Settings](hadoop-batch-mode.html#recommended-hadoop-cluster-configuration-settings).
@@ -170,7 +170,7 @@ arguments to the DML script were specified following the `-nvargs` option.
 
 In the console output, we see a warning that no default SystemML config file was found in the current working directory.
 In a distributed environment on a large data set, it is highly advisable to specify configuration settings in a SystemML config file for
-optimal performance. The location of the SystemML config file can be explicitly specified using the `-config=` argument.
+optimal performance. The location of the SystemML config file can be explicitly specified using the `-config ` argument.
 
 The OptimizerUtils warning occurs because parallel multi-threaded text reads in Java versions less than 1.8 result
 in thread contention issues, so only a single thread reads matrix data in text formats.
@@ -859,7 +859,7 @@ A description of the named arguments that can be passed in to this script can be
 `genRandData4Kmeans.dml` file. For data, I'll generate a matrix `X.mtx` consisting of 1 million rows and 100 features. I'll explicitly reference my `SystemML-config.xml` file, since I'm
 executing SystemML in Hadoop from my home directory rather than from the SystemML project root directory.
 
-	[hadoop@host1 ~]$ hadoop jar systemml-{{site.SYSTEMML_VERSION}}/SystemML.jar -f genRandData4Kmeans.dml -config=systemml-{{site.SYSTEMML_VERSION}}/SystemML-config.xml -nvargs nr=1000000 nf=100 nc=10 dc=10.0 dr=1.0 fbf=100.0 cbf=100.0 X=X.mtx C=C.mtx Y=Y.mtx YbyC=YbyC.mtx
+	[hadoop@host1 ~]$ hadoop jar systemml-{{site.SYSTEMML_VERSION}}/SystemML.jar -f genRandData4Kmeans.dml -config systemml-{{site.SYSTEMML_VERSION}}/SystemML-config.xml -nvargs nr=1000000 nf=100 nc=10 dc=10.0 dr=1.0 fbf=100.0 cbf=100.0 X=X.mtx C=C.mtx Y=Y.mtx YbyC=YbyC.mtx
 
 After the data generation has finished, I'll check HDFS for the amount of space used. The 1M-row matrix `X.mtx`
 requires about 2.8GB of space.
@@ -895,7 +895,7 @@ Here we can see the `X.mtx` data files.
 
 Next, I'll run the `Kmeans.dml` algorithm on the 1M-row matrix `X.mtx`.
 
-	[hadoop@host1 ~]$ hadoop jar systemml-{{site.SYSTEMML_VERSION}}/SystemML.jar -f systemml-{{site.SYSTEMML_VERSION}}/algorithms/Kmeans.dml -config=/systemml-{{site.SYSTEMML_VERSION}}/SystemML-config.xml -nvargs X=X.mtx k=5 C=Centroids.mtx
+	[hadoop@host1 ~]$ hadoop jar systemml-{{site.SYSTEMML_VERSION}}/SystemML.jar -f systemml-{{site.SYSTEMML_VERSION}}/algorithms/Kmeans.dml -config /systemml-{{site.SYSTEMML_VERSION}}/SystemML-config.xml -nvargs X=X.mtx k=5 C=Centroids.mtx
 
 We can see the `Centroids.mtx` data file has been written to HDFS.
 
@@ -916,7 +916,7 @@ We can see the `Centroids.mtx` data file has been written to HDFS.
 Now that we have trained our model, next we will test our model. We can do this with
 the `Kmeans-predict.dml` script.
 
-	[hadoop@host1 ~]$ hadoop jar systemml-{{site.SYSTEMML_VERSION}}/SystemML.jar -f systemml-{{site.SYSTEMML_VERSION}}/algorithms/Kmeans-predict.dml -config=systemml-{{site.SYSTEMML_VERSION}}/SystemML-config.xml -nvargs X=X.mtx C=Centroids.mtx prY=PredY.mtx O=stats.txt
+	[hadoop@host1 ~]$ hadoop jar systemml-{{site.SYSTEMML_VERSION}}/SystemML.jar -f systemml-{{site.SYSTEMML_VERSION}}/algorithms/Kmeans-predict.dml -config systemml-{{site.SYSTEMML_VERSION}}/SystemML-config.xml -nvargs X=X.mtx C=Centroids.mtx prY=PredY.mtx O=stats.txt
 
 In the file system, we can see that the `PredY.mtx` matrix was created.
 The `stats.txt` file lists statistics about the results.

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/spark-batch-mode.md
----------------------------------------------------------------------
diff --git a/docs/spark-batch-mode.md b/docs/spark-batch-mode.md
index c199b1f..39bcd3e 100644
--- a/docs/spark-batch-mode.md
+++ b/docs/spark-batch-mode.md
@@ -43,10 +43,10 @@ mode in more depth.
 
 SystemML can be invoked in Hadoop Batch mode using the following syntax:
 
-    spark-submit SystemML.jar [-? | -help | -f <filename>] (-config=<config_filename>) ([-args | -nvargs] <args-list>)
+    spark-submit SystemML.jar [-? | -help | -f <filename>] (-config <config_filename>) ([-args | -nvargs] <args-list>)
 
 The DML script to invoke is specified after the `-f` argument. Configuration settings can be passed to SystemML
-using the optional `-config=` argument. DML scripts can optionally take named arguments (`-nvargs`) or positional
+using the optional `-config ` argument. DML scripts can optionally take named arguments (`-nvargs`) or positional
 arguments (`-args`). Named arguments are preferred over positional arguments. Positional arguments are considered
 to be deprecated. All the primary algorithm scripts included with SystemML use named arguments.
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/standalone-guide.md
----------------------------------------------------------------------
diff --git a/docs/standalone-guide.md b/docs/standalone-guide.md
index 2c2092d..586e56e 100644
--- a/docs/standalone-guide.md
+++ b/docs/standalone-guide.md
@@ -605,5 +605,5 @@ script (`runStandaloneSystemML.sh` or `runStandaloneSystemML.bat`) to increase
 the memory available to the JVM, i.e:
 
     java -Xmx16g -Xms4g -Xmn1g -cp ${CLASSPATH} org.apache.sysml.api.DMLScript \
-         -f ${SCRIPT_FILE} -exec singlenode -config=SystemML-config.xml \
+         -f ${SCRIPT_FILE} -exec singlenode -config SystemML-config.xml \
          $@

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/docs/troubleshooting-guide.md
----------------------------------------------------------------------
diff --git a/docs/troubleshooting-guide.md b/docs/troubleshooting-guide.md
index 629bcf5..4731f51 100644
--- a/docs/troubleshooting-guide.md
+++ b/docs/troubleshooting-guide.md
@@ -91,7 +91,7 @@ They can also be configured on a **per SystemML-task basis** by inserting the fo
 
 Note: The default `SystemML-config.xml` is located in `<path to SystemML root>/conf/`. It is passed to SystemML using the `-config` argument:
 
-    hadoop jar SystemML.jar [-? | -help | -f <filename>] (-config=<config_filename>) ([-args | -nvargs] <args-list>)
+    hadoop jar SystemML.jar [-? | -help | -f <filename>] (-config <config_filename>) ([-args | -nvargs] <args-list>)
     
 See [Invoking SystemML in Hadoop Batch Mode](hadoop-batch-mode.html) for details of the syntax. 
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/scripts/sparkDML.sh
----------------------------------------------------------------------
diff --git a/scripts/sparkDML.sh b/scripts/sparkDML.sh
index 5548859..7bea639 100755
--- a/scripts/sparkDML.sh
+++ b/scripts/sparkDML.sh
@@ -116,7 +116,7 @@ $SPARK_HOME/bin/spark-submit \
      ${conf} \
      ${SYSTEMML_HOME}/SystemML.jar \
          -f ${f} \
-         -config=${SYSTEMML_HOME}/SystemML-config.xml \
+         -config ${SYSTEMML_HOME}/SystemML-config.xml \
          -exec hybrid_spark \
          $explain \
          $stats \

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/32924dc6/src/main/java/org/apache/sysml/api/DMLScript.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/DMLScript.java b/src/main/java/org/apache/sysml/api/DMLScript.java
index 84bbf77..ce60d55 100644
--- a/src/main/java/org/apache/sysml/api/DMLScript.java
+++ b/src/main/java/org/apache/sysml/api/DMLScript.java
@@ -35,6 +35,15 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Scanner;
 
+import org.apache.commons.cli.AlreadySelectedException;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.OptionGroup;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.PosixParser;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -105,66 +114,77 @@ public class DMLScript
 		HYBRID_SPARK,   // execute matrix operations in CP or Spark   
 		SPARK			// execute matrix operations in Spark
 	}
-	
-	public static RUNTIME_PLATFORM rtplatform = OptimizerUtils.getDefaultExecutionMode();
-	public static boolean STATISTICS = false; //default statistics
-	public static int STATISTICS_COUNT = 10;	//default statistics maximum heavy hitter count
-	public static boolean ENABLE_DEBUG_MODE = false; //default debug mode
-	public static boolean USE_LOCAL_SPARK_CONFIG = false; //set default local spark configuration - used for local testing
-	public static String DML_FILE_PATH_ANTLR_PARSER = null;
-	public static ExplainType EXPLAIN = ExplainType.NONE; //default explain
+
+	/**
+	 * Set of DMLOptions that can be set through the command line
+	 * and {@link org.apache.sysml.api.mlcontext.MLContext}
+	 * The values have been initialized with the default values
+	 * Despite there being a DML and PyDML, this class is named DMLOptions
+	 * to keep it consistent with {@link DMLScript} and {@link DMLOptions}
+	 */
+	public static class DMLOptions {
+		public Map<String, String>  argVals       = new HashMap<>();  // Arguments map containing either named arguments or arguments by position for a DML program
+		public String               configFile    = null;             // Path to config file if default config and default config is to be overriden
+		public boolean              clean         = false;            // Whether to clean up all SystemML working directories (FS, DFS)
+		public boolean              stats         = false;            // Whether to record and print the statistics
+		public int                  statsCount    = 10;	              // Default statistics count
+		public Explain.ExplainType  explainType   = Explain.ExplainType.NONE;  // Whether to print the "Explain" and if so, what type
+		public DMLScript.RUNTIME_PLATFORM execMode = OptimizerUtils.getDefaultExecutionMode();  // Execution mode standalone, MR, Spark or a hybrid
+		public boolean              gpu           = false;            // Whether to use the GPU
+		public boolean              forceGPU      = false;            // Whether to ignore memory & estimates and always use the GPU
+		public boolean              debug         = false;            // to go into debug mode to be able to step through a program
+		public ScriptType           scriptType    = ScriptType.DML;   // whether the script is a DML or PyDML script
+		public String               filePath      = null;             // path to script
+		public String               script        = null;             // the script itself
+		public boolean              help          = false;            // whether to print the usage option
+
+		public final static DMLOptions defaultOptions = new DMLOptions();
+
+		@Override
+		public String toString() {
+			return "DMLOptions{" +
+							"argVals=" + argVals +
+							", configFile='" + configFile + '\'' +
+							", clean=" + clean +
+							", stats=" + stats +
+							", statsCount=" + statsCount +
+							", explainType=" + explainType +
+							", execMode=" + execMode +
+							", gpu=" + gpu +
+							", forceGPU=" + forceGPU +
+							", debug=" + debug +
+							", scriptType=" + scriptType +
+							", filePath='" + filePath + '\'' +
+							", script='" + script + '\'' +
+							", help=" + help +
+							'}';
+		}
+	}
+
+	public static RUNTIME_PLATFORM  rtplatform          = DMLOptions.defaultOptions.execMode;    // the execution mode
+	public static boolean           STATISTICS          = DMLOptions.defaultOptions.stats;       // whether to print statistics
+	public static int               STATISTICS_COUNT    = DMLOptions.defaultOptions.statsCount;  // statistics maximum heavy hitter count
+	public static boolean           ENABLE_DEBUG_MODE   = DMLOptions.defaultOptions.debug;       // debug mode
+	public static ExplainType       EXPLAIN             = DMLOptions.defaultOptions.explainType; // explain type
+	public static String            DML_FILE_PATH_ANTLR_PARSER = DMLOptions.defaultOptions.filePath; // filename of dml/pydml script
+
 	/**
 	 * Global variable indicating the script type (DML or PYDML). Can be used
 	 * for DML/PYDML-specific tasks, such as outputting booleans in the correct
 	 * case (TRUE/FALSE for DML and True/False for PYDML).
 	 */
-	public static ScriptType SCRIPT_TYPE = ScriptType.DML;
-	
-	public static boolean USE_ACCELERATOR = false;
-	public static boolean FORCE_ACCELERATOR = false;
-	
-	// flag that indicates whether or not to suppress any prints to stdout
-	public static boolean _suppressPrint2Stdout = false;
-	
-	public static String _uuid = IDHandler.createDistributedUniqueID(); 
+	public static ScriptType        SCRIPT_TYPE         = DMLOptions.defaultOptions.scriptType;
+	public static boolean           USE_ACCELERATOR     = DMLOptions.defaultOptions.gpu;
+	public static boolean           FORCE_ACCELERATOR   = DMLOptions.defaultOptions.forceGPU;
+
+
+	public static boolean _suppressPrint2Stdout = false;  // flag that indicates whether or not to suppress any prints to stdout
+	public static boolean USE_LOCAL_SPARK_CONFIG = false; //set default local spark configuration - used for local testing
 	public static boolean _activeAM = false;
-	
+
+	public static String _uuid = IDHandler.createDistributedUniqueID();
 	private static final Log LOG = LogFactory.getLog(DMLScript.class.getName());
 	
-	public static String USAGE = 
-			"Usage is " + DMLScript.class.getCanonicalName() + " -f <filename>" 
-	        //+ " (-exec <mode>)?" + " (-explain <type>)?" + " (-stats)?" + " (-clean)?" + " (-config=<config_filename>)? 
-			+ " [-options] ([-args | -nvargs] <args-list>)? \n" 
-			+ "   -f: <filename> will be interpreted as a filename path (if <filename> is prefixed\n"
-			+ "         with hdfs or gpfs it is read from DFS, otherwise from local file system)\n" 
-			//undocumented feature in beta 08/2014 release
-			//+ "   -s: <filename> will be interpreted as a DML script string \n"
-			+ "   -python: (optional) parses Python-like DML\n"
-			+ "   -debug: (optional) run in debug mode\n"
-			+ "   -gpu: <flags> (optional) use acceleration whenever possible. Current version only supports CUDA.\n"
-			+ "			Supported <flags> for this mode is force=(true|false)\n"
-			// Later add optional flags to indicate optimizations turned on or off. Currently they are turned off.
-			//+ "   -debug: <flags> (optional) run in debug mode\n"
-			//+ "			Optional <flags> that is supported for this mode is optimize=(on|off)\n"
-			+ "   -exec: <mode> (optional) execution mode (hadoop, singlenode, [hybrid], hybrid_spark)\n"
-			+ "   -explain: <type> (optional) explain plan (hops, [runtime], recompile_hops, recompile_runtime)\n"
-			+ "   -stats: <count> (optional) monitor and report caching/recompilation statistics, default heavy hitter count is 10\n"
-			+ "   -clean: (optional) cleanup all SystemML working directories (FS, DFS).\n"
-			+ "         All other flags are ignored in this mode. \n"
-			+ "   -config: (optional) use config file <config_filename> (default: use parameter\n"
-			+ "         values in default SystemML-config.xml config file; if <config_filename> is\n" 
-			+ "         prefixed with hdfs or gpfs it is read from DFS, otherwise from local file system)\n"
-			+ "   -args: (optional) parameterize DML script with contents of [args list], ALL args\n"
-			+ "         after -args flag, each argument must be an unnamed-argument, where 1st value\n"
-			+ "         after -args will replace $1 in DML script, 2nd value will replace $2, etc.\n"
-			+ "   -nvargs: (optional) parameterize DML script with contents of [args list], ALL args\n"
-			+ "         after -nvargs flag, each argument must be be named-argument of form argName=argValue,\n"
-			+ "         where value will replace $argName in DML script, argName must be a valid DML variable\n"
-			+ "         name (start with letter, contain only letters, numbers, or underscores).\n"
-			+ "   <args-list>: (optional) args to DML script \n" 
-			+ "   -? | -help: (optional) show this help message \n";
-	
-	
 	///////////////////////////////
 	// public external interface
 	////////
@@ -195,21 +215,19 @@ public class DMLScript
 	public static boolean isActiveAM(){
 		return _activeAM;
 	}
-	
-	
+
 	/**
-	 * Default DML script invocation (e.g., via 'hadoop jar SystemML.jar -f Test.dml')
-	 * 
+	 *
 	 * @param args command-line arguments
 	 * @throws IOException if an IOException occurs
 	 * @throws DMLException if a DMLException occurs
 	 */
-	public static void main(String[] args) 
+	public static void main(String[] args)
 		throws IOException, DMLException
 	{
 		Configuration conf = new Configuration(ConfigurationManager.getCachedJobConf());
 		String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
-		
+
 		try {
 			DMLScript.executeScript(conf, otherArgs);
 		} catch (ParseException pe) {
@@ -218,7 +236,199 @@ public class DMLScript
 			// In case of DMLScriptException, simply print the error message.
 			System.err.println(e.getMessage());
 		}
-	} 
+	}
+
+	/**
+	 * Parses command line arguments to create a {@link DMLOptions} instance with the correct options
+	 * @param args	arguments from the command line
+	 * @param options	an {@link Options} instance containing the options that need to be parsed
+	 * @return an instance of {@link Options} that contain the correct {@link Option}s.
+	 * @throws org.apache.commons.cli.ParseException if there is an incorrect option specified in the CLI
+	 */
+	public static DMLOptions parseCLArguments(String[] args, Options options) throws org.apache.commons.cli.ParseException {
+
+		CommandLineParser clParser = new PosixParser();
+		CommandLine line = clParser.parse(options, args);
+
+		DMLOptions dmlOptions = new DMLOptions();
+		dmlOptions.help = line.hasOption("help");
+		dmlOptions.scriptType = line.hasOption("python") ? ScriptType.PYDML : ScriptType.DML;
+		dmlOptions.debug = line.hasOption("debug");
+		dmlOptions.gpu = line.hasOption("gpu");
+		if (dmlOptions.gpu) {
+			String force = line.getOptionValue("gpu");
+			if (force != null) {
+				if (force.equalsIgnoreCase("force")) {
+					dmlOptions.forceGPU = true;
+				} else {
+					throw new org.apache.commons.cli.ParseException("Invalid argument specified for -gpu option");
+				}
+			}
+		}
+		if (line.hasOption("exec")){
+			String execMode = line.getOptionValue("exec");
+			if (execMode != null){
+				if (execMode.equalsIgnoreCase("hadoop")) dmlOptions.execMode = RUNTIME_PLATFORM.HADOOP;
+				else if (execMode.equalsIgnoreCase("singlenode")) dmlOptions.execMode = RUNTIME_PLATFORM.SINGLE_NODE;
+				else if (execMode.equalsIgnoreCase("hybrid")) dmlOptions.execMode = RUNTIME_PLATFORM.HYBRID;
+				else if (execMode.equalsIgnoreCase("hybrid_spark")) dmlOptions.execMode = RUNTIME_PLATFORM.HYBRID_SPARK;
+				else if (execMode.equalsIgnoreCase("spark")) dmlOptions.execMode = RUNTIME_PLATFORM.SPARK;
+				else throw new org.apache.commons.cli.ParseException("Invalid argument specified for -exec option, must be one of [hadoop, singlenode, hybrid, hybrid_spark, spark]");
+			}
+		}
+		if (line.hasOption("explain")) {
+			dmlOptions.explainType = ExplainType.RUNTIME;
+			String explainType = line.getOptionValue("explain");
+			if (explainType != null){
+				if (explainType.equalsIgnoreCase("hops")) dmlOptions.explainType = ExplainType.HOPS;
+				else if (explainType.equalsIgnoreCase("runtime")) dmlOptions.explainType = ExplainType.RUNTIME;
+				else if (explainType.equalsIgnoreCase("recompile_hops")) dmlOptions.explainType = ExplainType.RECOMPILE_HOPS;
+				else if (explainType.equalsIgnoreCase("recompile_runtime")) dmlOptions.explainType = ExplainType.RECOMPILE_RUNTIME;
+				else throw new org.apache.commons.cli.ParseException("Invalid argument specified for -hops option, must be one of [hops, runtime, recompile_hops, recompile_runtime]");
+			}
+		}
+		dmlOptions.stats = line.hasOption("stats");
+		if (dmlOptions.stats){
+			String statsCount = line.getOptionValue("stats");
+			if (statsCount != null) {
+				try {
+					dmlOptions.statsCount = Integer.parseInt(statsCount);
+				} catch (NumberFormatException e) {
+					throw new org.apache.commons.cli.ParseException("Invalid argument specified for -stats option, must be a valid integer");
+				}
+			}
+		}
+
+		dmlOptions.clean = line.hasOption("clean");
+
+		if (line.hasOption("config")){
+			dmlOptions.configFile = line.getOptionValue("config");
+		}
+
+		if (line.hasOption("f")){
+			dmlOptions.filePath = line.getOptionValue("f");
+		}
+
+		if (line.hasOption("s")){
+			dmlOptions.script = line.getOptionValue("s");
+		}
+
+		// Positional arguments map is created as ("$1", "a"), ("$2", 123), ....
+		if (line.hasOption("args")){
+			String[] argValues = line.getOptionValues("args");
+			for (int k=0; k<argValues.length; k++){
+				String str = argValues[k];
+				if (!str.isEmpty()) {
+					dmlOptions.argVals.put("$" + (k+1), str);
+				}
+			}
+		}
+
+		// Named arguments map is created as ("$K, 123), ("$X", "X.csv"), ....
+		if (line.hasOption("nvargs")){
+			String varNameRegex = "^[a-zA-Z]([a-zA-Z0-9_])*$";
+			String[] nvargValues = line.getOptionValues("nvargs");
+			for (String str : nvargValues){
+				if (!str.isEmpty()){
+					String[] kv = str.split("=");
+					if (kv.length != 2){
+						throw new org.apache.commons.cli.ParseException("Invalid argument specified for -nvargs option, must be a list of space separated K=V pairs, where K is a valid name of a variable in the DML/PyDML program");
+					}
+					if (!kv[0].matches(varNameRegex)) {
+						throw new org.apache.commons.cli.ParseException("Invalid argument specified for -nvargs option, " + kv[0] + " does not seem like a valid variable name in DML. Valid variable names in DML start with upper-case or lower-case letter, and contain only letters, digits, or underscores");
+					}
+					dmlOptions.argVals.put("$" + kv[0], kv[1]);
+				}
+			}
+		}
+
+		return dmlOptions;
+
+	}
+
+	/**
+	 * Creates an {@link Options} instance for the command line parameters
+	 *  As of SystemML 0.13, Apache Commons CLI 1.2 is transitively in the classpath
+	 *  However the most recent version of Apache Commons CLI is 1.4
+	 *  Creating CLI options is done using Static methods. This obviously makes it
+	 *  thread unsafe. Instead of {@link OptionBuilder}, CLI 1.4 uses Option.Builder which
+	 *  has non-static methods.
+	 * @return an appropriate instance of {@link Options}
+	 */
+	@SuppressWarnings("static-access")
+	public static Options createCLIOptions() {
+		Options options = new Options();
+		Option nvargsOpt = OptionBuilder.withArgName("key=value")
+						.withDescription("parameterizes DML script with named parameters of the form <key=value>; <key> should be a valid identifier in DML/PyDML")
+						.hasArgs()
+						.create("nvargs");
+		Option argsOpt = OptionBuilder.withArgName("argN")
+						.withDescription("specifies positional parameters; first value will replace $1 in DML program; $2 will replace 2nd and so on")
+						.hasArgs()
+						.create("args");
+		Option configOpt = OptionBuilder.withArgName("filename")
+						.withDescription("uses a given configuration file (can be on local/hdfs/gpfs; default values in SystemML-config.xml")
+						.hasArg()
+						.create("config");
+		Option cleanOpt = OptionBuilder.withDescription("cleans up all SystemML working directories (FS, DFS); all other flags are ignored in this mode. \n")
+						.create("clean");
+		Option statsOpt = OptionBuilder.withArgName("count")
+						.withDescription("monitors and reports caching/recompilation statistics; heavy hitter <count> is 10 unless overridden; default off")
+						.hasOptionalArg()
+						.create("stats");
+		Option explainOpt = OptionBuilder.withArgName("level")
+						.withDescription("explains plan levels; can be 'hops' / 'runtime'[default] / 'recompile_hops' / 'recompile_runtime'")
+						.hasOptionalArg()
+						.create("explain");
+		Option execOpt = OptionBuilder.withArgName("mode")
+						.withDescription("sets execution mode; can be 'hadoop' / 'singlenode' / 'hybrid'[default] / 'hybrid_spark' / 'spark'")
+						.hasArg()
+						.create("exec");
+		Option gpuOpt = OptionBuilder.withArgName("force")
+						.withDescription("uses CUDA instructions when reasonable; set <force> option to skip conservative memory estimates and use GPU wherever possible; default off")
+						.hasOptionalArg()
+						.create("gpu");
+		Option debugOpt = OptionBuilder.withDescription("runs in debug mode; default off")
+						.create("debug");
+		Option pythonOpt = OptionBuilder.withDescription("parses Python-like DML")
+						.create("python");
+		Option fileOpt = OptionBuilder.withArgName("filename")
+						.withDescription("specifies dml/pydml file to execute; path can be local/hdfs/gpfs (prefixed with appropriate URI)")
+						.isRequired()
+						.hasArg()
+						.create("f");
+		Option scriptOpt = OptionBuilder.withArgName("script_contents")
+						.withDescription("specified script string to execute directly")
+						.isRequired()
+						.hasArg()
+						.create("s");
+		Option helpOpt = OptionBuilder.withDescription("shows usage message")
+						.create("help");
+
+		OptionGroup fileOrScriptOpt = new OptionGroup();
+		// Either a clean(-clean), a file(-f), a script(-s) or help(-help) needs to be specified
+		fileOrScriptOpt.addOption(scriptOpt);
+		fileOrScriptOpt.addOption(fileOpt);
+		fileOrScriptOpt.addOption(cleanOpt);
+		fileOrScriptOpt.addOption(helpOpt);
+		fileOrScriptOpt.setRequired(true);
+
+		OptionGroup argsOrNVArgsOpt = new OptionGroup();
+		argsOrNVArgsOpt.addOption(nvargsOpt).addOption(argsOpt);	// Either -args or -nvargs
+
+		options.addOption(configOpt);
+		options.addOption(cleanOpt);
+		options.addOption(statsOpt);
+		options.addOption(explainOpt);
+		options.addOption(execOpt);
+		options.addOption(gpuOpt);
+		options.addOption(debugOpt);
+		options.addOption(pythonOpt);
+		options.addOptionGroup(fileOrScriptOpt);
+		options.addOptionGroup(argsOrNVArgsOpt);
+		options.addOption(helpOpt);
+		return options;
+	}
 
 	/**
 	 * Single entry point for all public invocation alternatives (e.g.,
@@ -233,127 +443,86 @@ public class DMLScript
 	public static boolean executeScript( Configuration conf, String[] args ) 
 		throws DMLException
 	{
-		//Step 1: parse arguments 
-		//check for help 
-		if( args.length==0 || (args.length==1 && (args[0].equalsIgnoreCase("-help")|| args[0].equalsIgnoreCase("-?"))) ){
-			System.err.println( USAGE );
-			return true;
-		}
-		
-		//check for clean
-		else if( args.length==1 && args[0].equalsIgnoreCase("-clean") ){
-			cleanSystemMLWorkspace();
-			return true;
-		}
-			
-		//check number of args - print usage if incorrect
-		if( args.length < 2 ){
-			System.err.println( "ERROR: Unrecognized invocation arguments." );
-			System.err.println( USAGE );
-			return false;
-		}
-				
-		//check script arg - print usage if incorrect
-		if (!(args[0].equals("-f") || args[0].equals("-s"))){
-			System.err.println("ERROR: First argument must be either -f or -s");
-			System.err.println( USAGE );
-			return false;
-		}
-		
 		//parse arguments and set execution properties
-		RUNTIME_PLATFORM oldrtplatform = rtplatform; //keep old rtplatform
-		ExplainType oldexplain = EXPLAIN; //keep old explain
-		
-		// Reset global flags to avoid errors in test suite
-		ENABLE_DEBUG_MODE = false;
-		
-		boolean parsePyDML = false;
+		RUNTIME_PLATFORM oldrtplatform  = rtplatform;  //keep old rtplatform
+		ExplainType oldexplain          = EXPLAIN;     //keep old explain
+
+		Options options = createCLIOptions();
 		try
 		{
-			String fnameOptConfig = null; //optional config filename
-			String[] scriptArgs = null; //optional script arguments
-			boolean namedScriptArgs = false;
-			
-			for( int i=2; i<args.length; i++ )
-			{
-				if( args[i].equalsIgnoreCase("-explain") ) { 
-					EXPLAIN = ExplainType.RUNTIME;
-					if( args.length > (i+1) && !args[i+1].startsWith("-") )
-						EXPLAIN = Explain.parseExplainType(args[++i]);
-				}
-				else if( args[i].equalsIgnoreCase("-stats") ) {
-					STATISTICS = true;
-					if (args.length > (i + 1) && !args[i + 1].startsWith("-"))
-						STATISTICS_COUNT = Integer.parseInt(args[++i]);
-				}
-				else if ( args[i].equalsIgnoreCase("-exec")) {
-					rtplatform = parseRuntimePlatform(args[++i]);
-					if( rtplatform==null ) 
-						return false;
-				}
-				else if (args[i].startsWith("-config=")) // legacy
-					fnameOptConfig = args[i].substring(8).replaceAll("\"", "");
-				else if (args[i].equalsIgnoreCase("-config"))
-					fnameOptConfig = args[++i];
-				else if( args[i].equalsIgnoreCase("-debug") ) {					
-					ENABLE_DEBUG_MODE = true;
-				}
-				else if( args[i].equalsIgnoreCase("-gpu") ) {	
-					USE_ACCELERATOR = true;
-					if( args.length > (i+1) && !args[i+1].startsWith("-") ) {
-						String flag = args[++i];
-						if(flag.startsWith("force=")) {
-							String [] flagOptions = flag.split("=");
-							if(flagOptions.length == 2)
-								FORCE_ACCELERATOR = Boolean.parseBoolean(flagOptions[1]);
-							else
-								throw new DMLRuntimeException("Unsupported \"force\" option for -gpu:" + flag);
-						}
-						else {
-							throw new DMLRuntimeException("Unsupported flag for -gpu:" + flag);
-						}
-					}
-					GPUContext.getGPUContext(); // creates the singleton GPU context object. Return value ignored.
-				}
-				else if( args[i].equalsIgnoreCase("-python") ) {
-					parsePyDML = true;
-				}
-				else if (args[i].startsWith("-args") || args[i].startsWith("-nvargs")) {
-					namedScriptArgs = args[i].startsWith("-nvargs"); i++;
-					scriptArgs = new String[args.length - i];
-					System.arraycopy(args, i, scriptArgs, 0, scriptArgs.length); 
-					break;
-				}
-				else{
-					System.err.println("ERROR: Unknown argument: " + args[i]);
-					return false;
-				}
+			DMLOptions dmlOptions = parseCLArguments(args, options);
+
+			// String[] scriptArgs = null; //optional script arguments
+			// boolean namedScriptArgs = false;
+
+			STATISTICS        = dmlOptions.stats;
+			STATISTICS_COUNT  = dmlOptions.statsCount;
+			USE_ACCELERATOR   = dmlOptions.gpu;
+			FORCE_ACCELERATOR = dmlOptions.forceGPU;
+			EXPLAIN           = dmlOptions.explainType;
+			ENABLE_DEBUG_MODE = dmlOptions.debug;
+			SCRIPT_TYPE       = dmlOptions.scriptType;
+			rtplatform        = dmlOptions.execMode;
+
+			String fnameOptConfig = dmlOptions.configFile;
+			boolean isFile = dmlOptions.filePath != null;
+			String fileOrScript = isFile ? dmlOptions.filePath : dmlOptions.script;
+
+			boolean help = dmlOptions.help;
+
+			if (help) {
+				HelpFormatter formatter = new HelpFormatter();
+				formatter.printHelp( "systemml", options );
+				return true;
 			}
-			
+
+			if (USE_ACCELERATOR){
+				GPUContext.getGPUContext();
+			}
+
+			if (dmlOptions.clean) {
+				cleanSystemMLWorkspace();
+				return true;
+			}
+
 			//set log level
 			if (!ENABLE_DEBUG_MODE)
 				setLoggingProperties( conf );
 		
 			//Step 2: prepare script invocation
-			if (StringUtils.endsWithIgnoreCase(args[1], ".pydml")) {
-				parsePyDML = true;
+			if (isFile && StringUtils.endsWithIgnoreCase(fileOrScript, ".pydml")) {
+				SCRIPT_TYPE = ScriptType.PYDML;
 			}
-			String dmlScriptStr = readDMLScript(args[0], args[1]);
-			Map<String, String> argVals = createArgumentsMap(namedScriptArgs, scriptArgs);
-			
-			DML_FILE_PATH_ANTLR_PARSER = args[1];
+
+			String dmlScriptStr = readDMLScript(isFile, fileOrScript);
+			Map<String, String> argVals = dmlOptions.argVals;
+
+			DML_FILE_PATH_ANTLR_PARSER = dmlOptions.filePath;
 			
 			//Step 3: invoke dml script
-			printInvocationInfo(args[1], fnameOptConfig, argVals);
+			printInvocationInfo(fileOrScript, fnameOptConfig, argVals);
 			if (ENABLE_DEBUG_MODE) {
 				// inner try loop is just to isolate the debug exception, which will allow to manage the bugs from debugger v/s runtime
-				launchDebugger(dmlScriptStr, fnameOptConfig, argVals, parsePyDML);
+				launchDebugger(dmlScriptStr, fnameOptConfig, argVals, SCRIPT_TYPE);
 			}
 			else {
-				execute(dmlScriptStr, fnameOptConfig, argVals, args, parsePyDML);
+				execute(dmlScriptStr, fnameOptConfig, argVals, args, SCRIPT_TYPE);
 			}
 
 		}
+		catch(AlreadySelectedException e)
+		{
+			System.err.println("Mutually exclusive options were selected. " + e.getMessage());
+			HelpFormatter formatter = new HelpFormatter();
+			formatter.printHelp( "systemml", options );
+			return false;
+		}
+		catch(org.apache.commons.cli.ParseException e)
+		{
+			System.err.println(e.getMessage());
+			HelpFormatter formatter = new HelpFormatter();
+			formatter.printHelp( "systemml", options );
+		}
 		catch (ParseException pe) {
 			throw pe;
 		}
@@ -380,6 +549,17 @@ public class DMLScript
 	// private internal utils (argument parsing)
 	////////
 
+	@Deprecated()
+	/**
+	 * Creates an argument map appropriate for consumption by the backend
+	 * The only method using this is the legacy {@link MLContext} api.
+	 * Once that is removed, this function should be removed as well.
+	 * This method uses a fragile position based argument for -args & -nvargs
+	 * @param hasNamedArgs true for named arguments, false for positional arguments
+	 * @param args in "k=v" format for named arguments and "v" for positional arguments
+	 * @return	a map containing either ($K,V) or ($1,V) for named and positional arguments respectively
+	 * @throws LanguageException when a named argument is an invalid identifier for DML/PyDML
+	 */
 	protected static Map<String,String> createArgumentsMap(boolean hasNamedArgs, String[] args)
 		throws LanguageException
 	{			
@@ -399,7 +579,7 @@ public class DMLScript
 				arg.equalsIgnoreCase("-stats") || 
 				arg.equalsIgnoreCase("-exec") ||
 				arg.equalsIgnoreCase("-debug") ||
-				arg.startsWith("-config="))
+				arg.startsWith("-config"))
 			{
 					throw new LanguageException("-args or -nvargs must be the final argument for DMLScript!");
 			}
@@ -432,17 +612,26 @@ public class DMLScript
 		
 		return argMap;
 	}
-	
-	protected static String readDMLScript( String argname, String script ) 
+
+
+	/**
+	 * Reads the DML/PyDML script into a String
+	 * @param isFile	Whether the string argument is a path to a file or the script itself
+	 * @param scriptOrFilename script or filename
+	 * @return a string representation of the script
+	 * @throws IOException	if error
+	 * @throws LanguageException	if error
+	 */
+	protected static String readDMLScript( boolean isFile, String scriptOrFilename )
 		throws IOException, LanguageException
 	{
-		boolean fromFile = argname.equals("-f");
 		String dmlScriptStr;
 		
-		if( fromFile )
+		if( isFile )
 		{
+			String fileName = scriptOrFilename;
 			//read DML script from file
-			if(script == null)
+			if(fileName == null)
 				throw new LanguageException("DML script path was not specified!");
 			
 			StringBuilder sb = new StringBuilder();
@@ -450,21 +639,21 @@ public class DMLScript
 			try 
 			{
 				//read from hdfs or gpfs file system
-				if(    script.startsWith("hdfs:") 
-					|| script.startsWith("gpfs:") ) 
+				if(    fileName.startsWith("hdfs:")
+					|| fileName.startsWith("gpfs:") )
 				{ 
-					if( !LocalFileUtils.validateExternalFilename(script, true) )
+					if( !LocalFileUtils.validateExternalFilename(fileName, true) )
 						throw new LanguageException("Invalid (non-trustworthy) hdfs filename.");
 					FileSystem fs = FileSystem.get(ConfigurationManager.getCachedJobConf());
-					Path scriptPath = new Path(script);
+					Path scriptPath = new Path(fileName);
 					in = new BufferedReader(new InputStreamReader(fs.open(scriptPath)));
 				}
 				// from local file system
 				else 
 				{ 
-					if( !LocalFileUtils.validateExternalFilename(script, false) )
+					if( !LocalFileUtils.validateExternalFilename(fileName, false) )
 						throw new LanguageException("Invalid (non-trustworthy) local filename.");
-					in = new BufferedReader(new FileReader(script));
+					in = new BufferedReader(new FileReader(fileName));
 				}
 				
 				//core script reading
@@ -488,11 +677,12 @@ public class DMLScript
 		}
 		else
 		{
+			String scriptString = scriptOrFilename;
 			//parse given script string 
-			if(script == null)
+			if(scriptString == null)
 				throw new LanguageException("DML script was not specified!");
 			
-			InputStream is = new ByteArrayInputStream(script.getBytes());
+			InputStream is = new ByteArrayInputStream(scriptString.getBytes());
 			Scanner scan = new Scanner(is);
 			dmlScriptStr = scan.useDelimiter("\\A").next();	
 			scan.close();
@@ -500,26 +690,7 @@ public class DMLScript
 		
 		return dmlScriptStr;
 	}
-	
-	private static RUNTIME_PLATFORM parseRuntimePlatform( String platform )
-	{
-		RUNTIME_PLATFORM lrtplatform = null;
-		
-		if ( platform.equalsIgnoreCase("hadoop")) 
-			lrtplatform = RUNTIME_PLATFORM.HADOOP;
-		else if ( platform.equalsIgnoreCase("singlenode"))
-			lrtplatform = RUNTIME_PLATFORM.SINGLE_NODE;
-		else if ( platform.equalsIgnoreCase("hybrid"))
-			lrtplatform = RUNTIME_PLATFORM.HYBRID;
-		else if ( platform.equalsIgnoreCase("spark"))
-			lrtplatform = RUNTIME_PLATFORM.SPARK;
-		else if ( platform.equalsIgnoreCase("hybrid_spark"))
-			lrtplatform = RUNTIME_PLATFORM.HYBRID_SPARK;
-		else 
-			System.err.println("ERROR: Unknown runtime platform: " + platform);
-		
-		return lrtplatform;
-	}
+
 	
 	private static void setLoggingProperties( Configuration conf )
 	{
@@ -552,7 +723,7 @@ public class DMLScript
 	 * @param fnameOptConfig configuration file
 	 * @param argVals map of argument values
 	 * @param allArgs arguments
-	 * @param parsePyDML true if PYDML, false if DML
+	 * @param scriptType type of script (DML or PyDML)
 	 * @throws ParseException if ParseException occurs
 	 * @throws IOException if IOException occurs
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
@@ -560,10 +731,10 @@ public class DMLScript
 	 * @throws HopsException if HopsException occurs
 	 * @throws LopsException if LopsException occurs
 	 */
-	private static void execute(String dmlScriptStr, String fnameOptConfig, Map<String,String> argVals, String[] allArgs, boolean parsePyDML)
+	private static void execute(String dmlScriptStr, String fnameOptConfig, Map<String,String> argVals, String[] allArgs, ScriptType scriptType)
 		throws ParseException, IOException, DMLRuntimeException, LanguageException, HopsException, LopsException 
 	{	
-		SCRIPT_TYPE = parsePyDML ? ScriptType.PYDML : ScriptType.DML;
+		SCRIPT_TYPE = scriptType;
 
 		//print basic time and environment info
 		printStartExecInfo( dmlScriptStr );
@@ -582,7 +753,7 @@ public class DMLScript
 		
 		//Step 3: parse dml script
 		Statistics.startCompileTimer();
-		ParserWrapper parser = ParserFactory.createParser(parsePyDML);
+		ParserWrapper parser = ParserFactory.createParser(scriptType);
 		DMLProgram prog = parser.parse(DML_FILE_PATH_ANTLR_PARSER, dmlScriptStr, argVals);
 		
 		//Step 4: construct HOP DAGs (incl LVA, validate, and setup)
@@ -700,7 +871,7 @@ public class DMLScript
 	 * @param dmlScriptStr DML script contents (including new lines)
 	 * @param fnameOptConfig Full path of configuration file for SystemML
 	 * @param argVals Key-value pairs defining arguments of DML script
-	 * @param parsePyDML true if PYDML, false if DML
+	 * @param scriptType type of script (DML or PyDML)
 	 * @throws ParseException if ParseException occurs
 	 * @throws IOException if IOException occurs
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
@@ -709,7 +880,7 @@ public class DMLScript
 	 * @throws HopsException if HopsException occurs
 	 * @throws LopsException if LopsException occurs
 	 */
-	private static void launchDebugger(String dmlScriptStr, String fnameOptConfig, Map<String,String> argVals, boolean parsePyDML)
+	private static void launchDebugger(String dmlScriptStr, String fnameOptConfig, Map<String,String> argVals, ScriptType scriptType)
 		throws ParseException, IOException, DMLRuntimeException, DMLDebuggerException, LanguageException, HopsException, LopsException 
 	{		
 		DMLDebuggerProgramInfo dbprog = new DMLDebuggerProgramInfo();
@@ -719,7 +890,8 @@ public class DMLScript
 		ConfigurationManager.setGlobalConfig(conf);
 
 		//Step 2: parse dml script
-		ParserWrapper parser = ParserFactory.createParser(parsePyDML);
+
+		ParserWrapper parser = ParserFactory.createParser(scriptType);
 		DMLProgram prog = parser.parse(DML_FILE_PATH_ANTLR_PARSER, dmlScriptStr, argVals);
 		
 		//Step 3: construct HOP DAGs (incl LVA and validate)