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

incubator-systemml git commit: [SYSTEMML-1476][SYSTEMML-1477] Source import from jar

Repository: incubator-systemml
Updated Branches:
  refs/heads/master 4a941e44e -> eef5abf94


[SYSTEMML-1476][SYSTEMML-1477] Source import from jar

Rename AParserWrapper to ParserWrapper.
Create ParserFactory class for createParser method.
Update ParserWrapper to allow source statement to import scripts from the
SystemML jar file if the scripts are not found in the file system.

Closes #458.


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

Branch: refs/heads/master
Commit: eef5abf949ddcf749d4eecc62f8e2189e05b4ef4
Parents: 4a941e4
Author: Deron Eriksson <de...@us.ibm.com>
Authored: Wed Apr 12 14:53:53 2017 -0700
Committer: Deron Eriksson <de...@us.ibm.com>
Committed: Wed Apr 12 14:53:53 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/sysml/api/DMLScript.java    |  11 +-
 .../java/org/apache/sysml/api/MLContext.java    |   5 +-
 .../org/apache/sysml/api/jmlc/Connection.java   |   5 +-
 .../sysml/api/mlcontext/ScriptExecutor.java     |   5 +-
 .../org/apache/sysml/parser/AParserWrapper.java | 171 ------------------
 .../org/apache/sysml/parser/ParserFactory.java  |  49 +++++
 .../org/apache/sysml/parser/ParserWrapper.java  | 180 +++++++++++++++++++
 .../sysml/parser/dml/DMLParserWrapper.java      |   6 +-
 .../sysml/parser/pydml/PyDMLParserWrapper.java  |   9 +-
 .../functions/misc/DataTypeChangeTest.java      |  10 +-
 .../parfor/ParForDependencyAnalysisTest.java    |  10 +-
 11 files changed, 261 insertions(+), 200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/eef5abf9/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 e6a183a..dce9053 100644
--- a/src/main/java/org/apache/sysml/api/DMLScript.java
+++ b/src/main/java/org/apache/sysml/api/DMLScript.java
@@ -61,11 +61,12 @@ import org.apache.sysml.hops.codegen.SpoofCompiler.PlanCachePolicy;
 import org.apache.sysml.hops.globalopt.GlobalOptimizerWrapper;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.lops.LopsException;
-import org.apache.sysml.parser.AParserWrapper;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DMLTranslator;
 import org.apache.sysml.parser.LanguageException;
 import org.apache.sysml.parser.ParseException;
+import org.apache.sysml.parser.ParserFactory;
+import org.apache.sysml.parser.ParserWrapper;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLScriptException;
 import org.apache.sysml.runtime.controlprogram.Program;
@@ -73,12 +74,12 @@ import org.apache.sysml.runtime.controlprogram.caching.CacheStatistics;
 import org.apache.sysml.runtime.controlprogram.caching.CacheableData;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContextFactory;
-import org.apache.sysml.runtime.instructions.gpu.context.GPUContext;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
 import org.apache.sysml.runtime.controlprogram.parfor.ProgramConverter;
 import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysml.runtime.controlprogram.parfor.util.IDHandler;
+import org.apache.sysml.runtime.instructions.gpu.context.GPUContext;
+import org.apache.sysml.runtime.io.IOUtilFunctions;
 import org.apache.sysml.runtime.matrix.CleanupMR;
 import org.apache.sysml.runtime.matrix.data.LibMatrixDNN;
 import org.apache.sysml.runtime.matrix.mapred.MRConfigurationNames;
@@ -580,7 +581,7 @@ public class DMLScript
 		
 		//Step 3: parse dml script
 		Statistics.startCompileTimer();
-		AParserWrapper parser = AParserWrapper.createParser(parsePyDML);
+		ParserWrapper parser = ParserFactory.createParser(parsePyDML);
 		DMLProgram prog = parser.parse(DML_FILE_PATH_ANTLR_PARSER, dmlScriptStr, argVals);
 		
 		//Step 4: construct HOP DAGs (incl LVA, validate, and setup)
@@ -710,7 +711,7 @@ public class DMLScript
 		ConfigurationManager.setGlobalConfig(conf);
 
 		//Step 2: parse dml script
-		AParserWrapper parser = AParserWrapper.createParser(parsePyDML);
+		ParserWrapper parser = ParserFactory.createParser(parsePyDML);
 		DMLProgram prog = parser.parse(DML_FILE_PATH_ANTLR_PARSER, dmlScriptStr, argVals);
 		
 		//Step 3: construct HOP DAGs (incl LVA and validate)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/eef5abf9/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 6382832..85827da 100644
--- a/src/main/java/org/apache/sysml/api/MLContext.java
+++ b/src/main/java/org/apache/sysml/api/MLContext.java
@@ -51,7 +51,6 @@ import org.apache.sysml.hops.OptimizerUtils.OptimizationLevel;
 import org.apache.sysml.hops.globalopt.GlobalOptimizerWrapper;
 import org.apache.sysml.hops.rewrite.ProgramRewriter;
 import org.apache.sysml.hops.rewrite.RewriteRemovePersistentReadWrite;
-import org.apache.sysml.parser.AParserWrapper;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DMLTranslator;
 import org.apache.sysml.parser.DataExpression;
@@ -60,6 +59,8 @@ import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.parser.IntIdentifier;
 import org.apache.sysml.parser.LanguageException;
 import org.apache.sysml.parser.ParseException;
+import org.apache.sysml.parser.ParserFactory;
+import org.apache.sysml.parser.ParserWrapper;
 import org.apache.sysml.parser.StringIdentifier;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.LocalVariableMap;
@@ -1516,7 +1517,7 @@ public class MLContext {
 		_rtprog = null;
 		
 		//parsing
-		AParserWrapper parser = AParserWrapper.createParser(parsePyDML);
+		ParserWrapper parser = ParserFactory.createParser(parsePyDML);
 		DMLProgram prog;
 		if (isFile) {
 			prog = parser.parse(dmlScriptFilePath, null, argVals);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/eef5abf9/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 5240dc4..fe37a6c 100644
--- a/src/main/java/org/apache/sysml/api/jmlc/Connection.java
+++ b/src/main/java/org/apache/sysml/api/jmlc/Connection.java
@@ -40,11 +40,12 @@ import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.rewrite.ProgramRewriter;
 import org.apache.sysml.hops.rewrite.RewriteRemovePersistentReadWrite;
-import org.apache.sysml.parser.AParserWrapper;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DMLTranslator;
 import org.apache.sysml.parser.DataExpression;
 import org.apache.sysml.parser.ParseException;
+import org.apache.sysml.parser.ParserFactory;
+import org.apache.sysml.parser.ParserWrapper;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.Program;
 import org.apache.sysml.runtime.controlprogram.caching.CacheableData;
@@ -165,7 +166,7 @@ public class Connection implements Closeable
 		try
 		{
 			//parsing
-			AParserWrapper parser = AParserWrapper.createParser(parsePyDML);
+			ParserWrapper parser = ParserFactory.createParser(parsePyDML);
 			DMLProgram prog = parser.parse(null, script, args);
 			
 			//language validate

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/eef5abf9/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 c5801d4..e9270c5 100644
--- a/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
+++ b/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
@@ -36,11 +36,12 @@ import org.apache.sysml.hops.globalopt.GlobalOptimizerWrapper;
 import org.apache.sysml.hops.rewrite.ProgramRewriter;
 import org.apache.sysml.hops.rewrite.RewriteRemovePersistentReadWrite;
 import org.apache.sysml.lops.LopsException;
-import org.apache.sysml.parser.AParserWrapper;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DMLTranslator;
 import org.apache.sysml.parser.LanguageException;
 import org.apache.sysml.parser.ParseException;
+import org.apache.sysml.parser.ParserFactory;
+import org.apache.sysml.parser.ParserWrapper;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.LocalVariableMap;
 import org.apache.sysml.runtime.controlprogram.Program;
@@ -431,7 +432,7 @@ public class ScriptExecutor {
 	 */
 	protected void parseScript() {
 		try {
-			AParserWrapper parser = AParserWrapper.createParser(script.getScriptType().isPYDML());
+			ParserWrapper parser = ParserFactory.createParser(script.getScriptType().isPYDML());
 			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/eef5abf9/src/main/java/org/apache/sysml/parser/AParserWrapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/AParserWrapper.java b/src/main/java/org/apache/sysml/parser/AParserWrapper.java
deleted file mode 100644
index a413f53..0000000
--- a/src/main/java/org/apache/sysml/parser/AParserWrapper.java
+++ /dev/null
@@ -1,171 +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.parser;
-
-import java.io.BufferedReader;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.sysml.conf.ConfigurationManager;
-import org.apache.sysml.parser.common.CommonSyntacticValidator;
-import org.apache.sysml.parser.common.CustomErrorListener.ParseIssue;
-import org.apache.sysml.parser.dml.DMLParserWrapper;
-import org.apache.sysml.parser.pydml.PyDMLParserWrapper;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.util.LocalFileUtils;
-
-/**
- * Base class for all dml parsers in order to make the various compilation chains
- * independent of the used parser.
- */
-public abstract class AParserWrapper 
-{
-	protected boolean atLeastOneError = false;
-	protected boolean atLeastOneWarning = false;
-	protected List<ParseIssue> parseIssues;
-	
-	public abstract DMLProgram parse(String fileName, String dmlScript, Map<String, String> argVals)
-		throws ParseException;
-
-	
-	/**
-	 * Factory method for creating parser wrappers
-	 * 
-	 * @param pydml true if a PyDML parser is needed
-	 * @return parser wrapper
-	 */
-	public static AParserWrapper createParser(boolean pydml)
-	{
-		AParserWrapper ret = null;
-		
-		//create the parser instance
-		if( pydml )
-			ret = new PyDMLParserWrapper();
-		else
-			ret = new DMLParserWrapper();
-		
-		CommonSyntacticValidator.init();
-		
-		return ret;
-	}
-	
-	/**
-	 * Custom wrapper to convert statement into statement blocks. Called by doParse and in DmlSyntacticValidator for for, parfor, while, ...
-	 * @param current a statement
-	 * @return corresponding statement block
-	 */
-	public static StatementBlock getStatementBlock(Statement current) {
-		StatementBlock blk = null;
-		if(current instanceof ParForStatement) {
-			blk = new ParForStatementBlock();
-			blk.addStatement(current);
-		}
-		else if(current instanceof ForStatement) {
-			blk = new ForStatementBlock();
-			blk.addStatement(current);
-		}
-		else if(current instanceof IfStatement) {
-			blk = new IfStatementBlock();
-			blk.addStatement(current);
-		}
-		else if(current instanceof WhileStatement) {
-			blk = new WhileStatementBlock();
-			blk.addStatement(current);
-		}
-		else {
-			// This includes ImportStatement
-			blk = new StatementBlock();
-			blk.addStatement(current);
-		}
-		return blk;
-	}
-	
-	
-	public static String readDMLScript( String script, Log LOG) 
-			throws IOException, LanguageException
-	{
-		String dmlScriptStr = null;
-		
-		//read DML script from file
-		if(script == null)
-			throw new LanguageException("DML script path was not specified!");
-		
-		StringBuilder sb = new StringBuilder();
-		BufferedReader in = null;
-		try 
-		{
-			//read from hdfs or gpfs file system
-			if(    script.startsWith("hdfs:") 
-				|| script.startsWith("gpfs:") ) 
-			{ 
-				if( !LocalFileUtils.validateExternalFilename(script, true) )
-					throw new LanguageException("Invalid (non-trustworthy) hdfs filename.");
-				FileSystem fs = FileSystem.get(ConfigurationManager.getCachedJobConf());
-				Path scriptPath = new Path(script);
-				in = new BufferedReader(new InputStreamReader(fs.open(scriptPath)));
-			}
-			// from local file system
-			else 
-			{ 
-				if( !LocalFileUtils.validateExternalFilename(script, false) )
-					throw new LanguageException("Invalid (non-trustworthy) local filename.");
-				in = new BufferedReader(new FileReader(script));
-			}
-			
-			//core script reading
-			String tmp = null;
-			while ((tmp = in.readLine()) != null)
-			{
-				sb.append( tmp );
-				sb.append( "\n" );
-			}
-		}
-		catch (IOException ex)
-		{
-			LOG.error("Failed to read the script from the file system", ex);
-			throw ex;
-		}
-		finally {
-			IOUtilFunctions.closeSilently(in);
-		}
-		
-		dmlScriptStr = sb.toString();
-		
-		return dmlScriptStr;
-	}
-	
-	public boolean isAtLeastOneError() {
-		return atLeastOneError;
-	}
-
-	public boolean isAtLeastOneWarning() {
-		return atLeastOneWarning;
-	}
-
-	public List<ParseIssue> getParseIssues() {
-		return parseIssues;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/eef5abf9/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
new file mode 100644
index 0000000..37c62df
--- /dev/null
+++ b/src/main/java/org/apache/sysml/parser/ParserFactory.java
@@ -0,0 +1,49 @@
+/*
+ * 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.parser;
+
+import org.apache.sysml.parser.common.CommonSyntacticValidator;
+import org.apache.sysml.parser.dml.DMLParserWrapper;
+import org.apache.sysml.parser.pydml.PyDMLParserWrapper;
+
+public class ParserFactory {
+
+	/**
+	 * Factory method for creating parser wrappers
+	 * 
+	 * @param pydml
+	 *            true if a PyDMLParserWrapper is needed, false if a DMLParserWrapper is needed
+	 * @return parser wrapper (DMLParserWrapper or PyDMLParserWrapper)
+	 */
+	public static ParserWrapper createParser(boolean pydml) {
+		ParserWrapper ret = null;
+
+		// create the parser instance
+		if (pydml)
+			ret = new PyDMLParserWrapper();
+		else
+			ret = new DMLParserWrapper();
+
+		CommonSyntacticValidator.init();
+
+		return ret;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/eef5abf9/src/main/java/org/apache/sysml/parser/ParserWrapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/ParserWrapper.java b/src/main/java/org/apache/sysml/parser/ParserWrapper.java
new file mode 100644
index 0000000..ec479e2
--- /dev/null
+++ b/src/main/java/org/apache/sysml/parser/ParserWrapper.java
@@ -0,0 +1,180 @@
+/*
+ * 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.parser;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.sysml.conf.ConfigurationManager;
+import org.apache.sysml.parser.common.CustomErrorListener.ParseIssue;
+import org.apache.sysml.runtime.io.IOUtilFunctions;
+import org.apache.sysml.runtime.util.LocalFileUtils;
+
+/**
+ * Base class for all dml parsers in order to make the various compilation chains
+ * independent of the used parser.
+ */
+public abstract class ParserWrapper {
+	protected boolean atLeastOneError = false;
+	protected boolean atLeastOneWarning = false;
+	protected List<ParseIssue> parseIssues;
+	
+	public abstract DMLProgram parse(String fileName, String dmlScript, Map<String, String> argVals)
+		throws ParseException;
+
+	/**
+	 * Custom wrapper to convert statement into statement blocks. Called by doParse and in DmlSyntacticValidator for for, parfor, while, ...
+	 * @param current a statement
+	 * @return corresponding statement block
+	 */
+	public static StatementBlock getStatementBlock(Statement current) {
+		StatementBlock blk = null;
+		if(current instanceof ParForStatement) {
+			blk = new ParForStatementBlock();
+			blk.addStatement(current);
+		}
+		else if(current instanceof ForStatement) {
+			blk = new ForStatementBlock();
+			blk.addStatement(current);
+		}
+		else if(current instanceof IfStatement) {
+			blk = new IfStatementBlock();
+			blk.addStatement(current);
+		}
+		else if(current instanceof WhileStatement) {
+			blk = new WhileStatementBlock();
+			blk.addStatement(current);
+		}
+		else {
+			// This includes ImportStatement
+			blk = new StatementBlock();
+			blk.addStatement(current);
+		}
+		return blk;
+	}
+	
+	
+	public static String readDMLScript( String script, Log LOG) 
+			throws IOException, LanguageException
+	{
+		String dmlScriptStr = null;
+		
+		//read DML script from file
+		if(script == null)
+			throw new LanguageException("DML script path was not specified!");
+		
+		StringBuilder sb = new StringBuilder();
+		BufferedReader in = null;
+		try 
+		{
+			//read from hdfs or gpfs file system
+			if(    script.startsWith("hdfs:") 
+				|| script.startsWith("gpfs:") ) 
+			{
+				LOG.debug("Looking for the following file in HDFS or GPFS: " + script);
+				if( !LocalFileUtils.validateExternalFilename(script, true) )
+					throw new LanguageException("Invalid (non-trustworthy) hdfs filename.");
+				FileSystem fs = FileSystem.get(ConfigurationManager.getCachedJobConf());
+				Path scriptPath = new Path(script);
+				in = new BufferedReader(new InputStreamReader(fs.open(scriptPath)));
+			}
+			// from local file system
+			else 
+			{
+				LOG.debug("Looking for the following file in the local file system: " + script);
+				if( !LocalFileUtils.validateExternalFilename(script, false) )
+					throw new LanguageException("Invalid (non-trustworthy) local filename.");
+				in = new BufferedReader(new FileReader(script));
+			}
+			
+			//core script reading
+			String tmp = null;
+			while ((tmp = in.readLine()) != null)
+			{
+				sb.append( tmp );
+				sb.append( "\n" );
+			}
+		}
+		catch (IOException ex)
+		{
+			String resPath = scriptPathToResourcePath(script);
+			LOG.debug("Looking for the following resource from the SystemML jar file: " + resPath);
+			InputStream is = ParserWrapper.class.getResourceAsStream(resPath);
+			if (is == null) {
+				if (resPath.startsWith("/scripts")) {
+					LOG.error("Failed to read from the file system ('" + script + "') or SystemML jar file ('" + resPath + "')");
+					throw ex;
+				} else {
+					// for accessing script packages in the scripts directory
+					String scriptsResPath = "/scripts" + resPath;
+					LOG.debug("Looking for the following resource from the SystemML jar file: " + scriptsResPath);
+					is = ParserWrapper.class.getResourceAsStream(scriptsResPath);
+					if (is == null) {
+						LOG.error("Failed to read from the file system ('" + script + "') or SystemML jar file ('" + resPath + "' or '" + scriptsResPath + "')");
+						throw ex;
+					}
+				}
+			}
+			String s = IOUtils.toString(is);
+			return s;
+		}
+		finally {
+			IOUtilFunctions.closeSilently(in);
+		}
+		
+		dmlScriptStr = sb.toString();
+		
+		return dmlScriptStr;
+	}
+
+	private static String scriptPathToResourcePath(String scriptPath) {
+		String resPath = scriptPath;
+		if (resPath.startsWith(".")) {
+			resPath = resPath.substring(1);
+		} else if (resPath.startsWith("\\")) {
+			// do nothing
+		} else if (!resPath.startsWith("/")) {
+			resPath = "/" + resPath;
+		}
+		resPath = resPath.replace("\\", "/");
+		return resPath;
+	}
+
+	public boolean isAtLeastOneError() {
+		return atLeastOneError;
+	}
+
+	public boolean isAtLeastOneWarning() {
+		return atLeastOneWarning;
+	}
+
+	public List<ParseIssue> getParseIssues() {
+		return parseIssues;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/eef5abf9/src/main/java/org/apache/sysml/parser/dml/DMLParserWrapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/dml/DMLParserWrapper.java b/src/main/java/org/apache/sysml/parser/dml/DMLParserWrapper.java
index 9c070d8..337aa18 100644
--- a/src/main/java/org/apache/sysml/parser/dml/DMLParserWrapper.java
+++ b/src/main/java/org/apache/sysml/parser/dml/DMLParserWrapper.java
@@ -36,12 +36,12 @@ import org.antlr.v4.runtime.tree.ParseTreeWalker;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.parser.AParserWrapper;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.FunctionStatementBlock;
 import org.apache.sysml.parser.ImportStatement;
 import org.apache.sysml.parser.LanguageException;
 import org.apache.sysml.parser.ParseException;
+import org.apache.sysml.parser.ParserWrapper;
 import org.apache.sysml.parser.common.CustomErrorListener;
 import org.apache.sysml.parser.dml.DmlParser.FunctionStatementContext;
 import org.apache.sysml.parser.dml.DmlParser.ProgramrootContext;
@@ -71,7 +71,7 @@ import org.apache.sysml.parser.dml.DmlParser.StatementContext;
  * If in future we intend to make it multi-threaded, look at cleanUpState method and resolve the dependency accordingly.    
  *
  */
-public class DMLParserWrapper extends AParserWrapper
+public class DMLParserWrapper extends ParserWrapper
 {
 	private static final Log LOG = LogFactory.getLog(DMLScript.class.getName());
 
@@ -111,7 +111,7 @@ public class DMLParserWrapper extends AParserWrapper
 			InputStream stream = new ByteArrayInputStream(dmlScript.getBytes());
 			in = new ANTLRInputStream(stream);
 		} catch (FileNotFoundException e) {
-			throw new ParseException("Cannot find file: " + fileName, e);
+			throw new ParseException("Cannot find file/resource: " + fileName, e);
 		} catch (IOException e) {
 			throw new ParseException("Cannot open file: " + fileName, e);
 		} catch (LanguageException e) {

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/eef5abf9/src/main/java/org/apache/sysml/parser/pydml/PyDMLParserWrapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/pydml/PyDMLParserWrapper.java b/src/main/java/org/apache/sysml/parser/pydml/PyDMLParserWrapper.java
index 2799bc4..6ba6c94 100644
--- a/src/main/java/org/apache/sysml/parser/pydml/PyDMLParserWrapper.java
+++ b/src/main/java/org/apache/sysml/parser/pydml/PyDMLParserWrapper.java
@@ -36,15 +36,14 @@ import org.antlr.v4.runtime.tree.ParseTreeWalker;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.parser.AParserWrapper;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.FunctionStatementBlock;
 import org.apache.sysml.parser.ImportStatement;
 import org.apache.sysml.parser.LanguageException;
 import org.apache.sysml.parser.ParseException;
+import org.apache.sysml.parser.ParserWrapper;
 import org.apache.sysml.parser.Statement;
 import org.apache.sysml.parser.common.CustomErrorListener;
-import org.apache.sysml.parser.dml.DMLParserWrapper;
 import org.apache.sysml.parser.pydml.PydmlParser.FunctionStatementContext;
 import org.apache.sysml.parser.pydml.PydmlParser.ProgramrootContext;
 import org.apache.sysml.parser.pydml.PydmlParser.StatementContext;
@@ -55,7 +54,7 @@ import org.apache.sysml.parser.pydml.PydmlParser.StatementContext;
  * Note: ExpressionInfo and StatementInfo are simply wrapper objects and are reused in both DML and PyDML parsers.
  *
  */
-public class PyDMLParserWrapper extends AParserWrapper
+public class PyDMLParserWrapper extends ParserWrapper
 {
 	private static final Log LOG = LogFactory.getLog(DMLScript.class.getName());
 
@@ -89,14 +88,14 @@ public class PyDMLParserWrapper extends AParserWrapper
 		ANTLRInputStream in;
 		try {
 			if(dmlScript == null) {
-				dmlScript = DMLParserWrapper.readDMLScript(fileName, LOG);
+				dmlScript = readDMLScript(fileName, LOG);
 			}
 			
 			InputStream stream = new ByteArrayInputStream(dmlScript.getBytes());
 			in = new org.antlr.v4.runtime.ANTLRInputStream(stream);
 		} 
 		catch (FileNotFoundException e) {
-			throw new ParseException("Cannot find file: " + fileName, e);
+			throw new ParseException("Cannot find file/resource: " + fileName, e);
 		} 
 		catch (IOException e) {
 			throw new ParseException("Cannot open file: " + fileName, e);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/eef5abf9/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 42e6c05..899037c 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
@@ -23,19 +23,19 @@ import java.io.BufferedReader;
 import java.io.FileReader;
 import java.util.HashMap;
 
-import org.junit.Assert;
-import org.junit.Test;
-
 import org.apache.sysml.api.DMLException;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.DMLConfig;
-import org.apache.sysml.parser.AParserWrapper;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DMLTranslator;
 import org.apache.sysml.parser.LanguageException;
+import org.apache.sysml.parser.ParserFactory;
+import org.apache.sysml.parser.ParserWrapper;
 import org.apache.sysml.test.integration.AutomatedTestBase;
 import org.apache.sysml.test.integration.TestConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
 
 /**
  * GENERAL NOTE
@@ -197,7 +197,7 @@ public class DataTypeChangeTest extends AutomatedTestBase
 			}	
 			
 			//parsing and dependency analysis
-			AParserWrapper parser = AParserWrapper.createParser(false);
+			ParserWrapper parser = ParserFactory.createParser(false);
 			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/eef5abf9/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 f58d747..76d7ffa 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
@@ -23,18 +23,18 @@ import java.io.BufferedReader;
 import java.io.FileReader;
 import java.util.HashMap;
 
-import org.junit.Assert;
-import org.junit.Test;
-
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.DMLConfig;
-import org.apache.sysml.parser.AParserWrapper;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DMLTranslator;
 import org.apache.sysml.parser.LanguageException;
+import org.apache.sysml.parser.ParserFactory;
+import org.apache.sysml.parser.ParserWrapper;
 import org.apache.sysml.test.integration.AutomatedTestBase;
 import org.apache.sysml.test.integration.TestConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
 
 /**
  * Different test cases for ParFOR loop dependency analysis:
@@ -351,7 +351,7 @@ public class ParForDependencyAnalysisTest extends AutomatedTestBase
 			}	
 			
 			//parsing and dependency analysis
-			AParserWrapper parser = AParserWrapper.createParser(false);
+			ParserWrapper parser = ParserFactory.createParser(false);
 			DMLProgram prog = parser.parse(DMLScript.DML_FILE_PATH_ANTLR_PARSER, dmlScriptString, argVals);
 			DMLTranslator dmlt = new DMLTranslator(prog);
 			dmlt.validateParseTree(prog);