You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by mb...@apache.org on 2018/05/11 03:31:35 UTC

[1/3] systemml git commit: [SYSTEMML-2309] Fix length over lists w/ unknown dims initial compile

Repository: systemml
Updated Branches:
  refs/heads/master 6f2c885e8 -> e2875cae4


[SYSTEMML-2309] Fix length over lists w/ unknown dims initial compile

This patch fixes the runtime instruction for length operations over
lists with unknown size during initial compilation. Initially known
sizes lead to constant propagation and replacement which has hidden the
existing issue of the runtime instruction.


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

Branch: refs/heads/master
Commit: 4e4586881a8086b40f99c184d8875931e8836a76
Parents: 6f2c885
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu May 10 13:47:06 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu May 10 13:47:06 2018 -0700

----------------------------------------------------------------------
 .../cp/AggregateUnaryCPInstruction.java         | 60 +++++++++++---------
 1 file changed, 33 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/4e458688/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java
index c9d9dc7..ed95f7d 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java
@@ -87,34 +87,40 @@ public class AggregateUnaryCPInstruction extends UnaryCPInstruction
 				throw new DMLRuntimeException("Variable '"+input1.getName()+"' does not exist.");
 			
 			//get meta data information
-			MatrixCharacteristics mc = ec.getMatrixCharacteristics(input1.getName());
-			long rval = getSizeMetaData(_type, mc);
-
-			//check for valid output, and acquire read if necessary
-			//(Use case: In case of forced exec type singlenode, there are no reblocks. For csv
-			//we however, support unspecified input sizes, which requires a read to obtain the
-			//required meta data)
-			//Note: check on matrix characteristics to cover incorrect length (-1*-1 -> 1)
-			if( !mc.dimsKnown() ) //invalid nrow/ncol/length
-			{
-				if( DMLScript.rtplatform == RUNTIME_PLATFORM.SINGLE_NODE 
-					|| (input1.getDataType() == DataType.FRAME && OptimizerUtils.isHadoopExecutionMode()) )
+			long rval = -1;
+			if (input1.getDataType() == DataType.LIST && _type == AUType.LENGTH ) {
+				rval = ((ListObject)ec.getVariable(input1.getName())).getLength();
+			}
+			else if( input1.getDataType().isMatrix() || input1.getDataType().isFrame() ) {
+				MatrixCharacteristics mc = ec.getMatrixCharacteristics(input1.getName());
+				rval = getSizeMetaData(_type, mc);
+	
+				//check for valid output, and acquire read if necessary
+				//(Use case: In case of forced exec type singlenode, there are no reblocks. For csv
+				//we however, support unspecified input sizes, which requires a read to obtain the
+				//required meta data)
+				//Note: check on matrix characteristics to cover incorrect length (-1*-1 -> 1)
+				if( !mc.dimsKnown() ) //invalid nrow/ncol/length
 				{
-					if( OptimizerUtils.isHadoopExecutionMode() )
-						LOG.warn("Reading csv input frame of unkown size into memory for '"+opcode+"'.");
-					
-					//read the input matrix/frame and explicitly refresh meta data
-					CacheableData<?> obj = ec.getCacheableData(input1.getName());
-					obj.acquireRead();
-					obj.refreshMetaData();
-					obj.release();
-					
-					//update meta data information
-					mc = ec.getMatrixCharacteristics(input1.getName());
-					rval = getSizeMetaData(_type, mc);
-				}
-				else {
-					throw new DMLRuntimeException("Invalid meta data returned by '"+opcode+"': "+rval + ":" + instString);
+					if( DMLScript.rtplatform == RUNTIME_PLATFORM.SINGLE_NODE 
+						|| (input1.getDataType() == DataType.FRAME && OptimizerUtils.isHadoopExecutionMode()) )
+					{
+						if( OptimizerUtils.isHadoopExecutionMode() )
+							LOG.warn("Reading csv input frame of unkown size into memory for '"+opcode+"'.");
+						
+						//read the input matrix/frame and explicitly refresh meta data
+						CacheableData<?> obj = ec.getCacheableData(input1.getName());
+						obj.acquireRead();
+						obj.refreshMetaData();
+						obj.release();
+						
+						//update meta data information
+						mc = ec.getMatrixCharacteristics(input1.getName());
+						rval = getSizeMetaData(_type, mc);
+					}
+					else {
+						throw new DMLRuntimeException("Invalid meta data returned by '"+opcode+"': "+rval + ":" + instString);
+					}
 				}
 			}
 			


[2/3] systemml git commit: [SYSTEMML-2311] Support for lists and named-lists in function calls

Posted by mb...@apache.org.
[SYSTEMML-2311] Support for lists and named-lists in function calls

This patch makes all necessary compiler and runtime extensions to
support lists with unknown value type as function inputs and outputs.
Beside parser extensions, this also includes additional tests and a
cleanup of redundant code in the dml and pydml syntactic parsers.


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

Branch: refs/heads/master
Commit: ae86c3f767f12e7be0572e4c1308ce1e0a02d024
Parents: 4e45868
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu May 10 14:07:28 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu May 10 20:32:45 2018 -0700

----------------------------------------------------------------------
 .../sysml/parser/FunctionStatementBlock.java    |  8 ---
 .../parser/common/CommonSyntacticValidator.java | 54 +++++++++++++++++---
 .../java/org/apache/sysml/parser/dml/Dml.g4     |  4 +-
 .../sysml/parser/dml/DmlSyntacticValidator.java | 48 +++--------------
 .../parser/pydml/PydmlSyntacticValidator.java   | 42 +++------------
 .../functions/misc/ListAndStructTest.java       | 25 +++++++++
 src/test/scripts/functions/misc/ListNamed.dml   |  2 +-
 src/test/scripts/functions/misc/ListNamedFun.R  | 42 +++++++++++++++
 .../scripts/functions/misc/ListNamedFun.dml     | 40 +++++++++++++++
 .../scripts/functions/misc/ListUnnamedFun.R     | 42 +++++++++++++++
 .../scripts/functions/misc/ListUnnamedFun.dml   | 40 +++++++++++++++
 11 files changed, 252 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java b/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java
index 2d7543f..198a914 100644
--- a/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java
@@ -85,14 +85,6 @@ public class FunctionStatementBlock extends StatementBlock
 					raiseValidateError("for function " + fstmt.getName() + ", return variable " + returnValue.getName() + " must be defined in function ", conditional);
 				}
 				
-				if (curr.getDataType() == DataType.UNKNOWN){
-					raiseValidateError("for function " + fstmt.getName() + ", return variable " + curr.getName() + " data type of " + curr.getDataType() + " may not match data type in function signature of " + returnValue.getDataType(), true);
-				}
-				
-				if (curr.getValueType() == ValueType.UNKNOWN){
-					raiseValidateError("for function " + fstmt.getName() + ", return variable " + curr.getName() + " data type of " + curr.getValueType() + " may not match data type in function signature of " + returnValue.getValueType(), true);
-				}
-				
 				if (curr.getDataType() != DataType.UNKNOWN && !curr.getDataType().equals(returnValue.getDataType()) ){
 					raiseValidateError("for function " + fstmt.getName() + ", return variable " + curr.getName() + " data type of " + curr.getDataType() + " does not match data type in function signature of " + returnValue.getDataType(), conditional);
 				}

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java b/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java
index fb25a6a..3226895 100644
--- a/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java
+++ b/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java
@@ -42,6 +42,8 @@ import org.apache.sysml.parser.DataIdentifier;
 import org.apache.sysml.parser.DoubleIdentifier;
 import org.apache.sysml.parser.Expression;
 import org.apache.sysml.parser.Expression.DataOp;
+import org.apache.sysml.parser.Expression.DataType;
+import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.parser.FunctionCallIdentifier;
 import org.apache.sysml.parser.IntIdentifier;
 import org.apache.sysml.parser.LanguageException;
@@ -705,12 +707,52 @@ public abstract class CommonSyntacticValidator {
 	 * @param start antlr token
 	 */
 	protected void checkValidDataType(String datatype, Token start) {
-		boolean validMatrixType = 
-				datatype.equals("matrix") || datatype.equals("Matrix") || 
-				datatype.equals("frame") || datatype.equals("Frame") ||
-				datatype.equals("scalar") || datatype.equals("Scalar");
-		if(!validMatrixType	) {
-			notifyErrorListeners("incorrect datatype (expected matrix, frame or scalar)", start);
+		boolean validMatrixType = datatype.equals("matrix") || datatype.equals("Matrix")
+			|| datatype.equals("frame") || datatype.equals("Frame")
+			|| datatype.equals("list") || datatype.equals("List")
+			|| datatype.equals("scalar") || datatype.equals("Scalar");
+		if( !validMatrixType )
+			notifyErrorListeners("incorrect datatype (expected matrix, frame, list, or scalar)", start);
+	}
+	
+	protected boolean setDataAndValueType(DataIdentifier dataId, String dataType, String valueType, Token start, boolean shortVt, boolean helpBool) {
+		if( dataType.equalsIgnoreCase("matrix") )
+			dataId.setDataType(DataType.MATRIX);
+		else if( dataType.equalsIgnoreCase("frame") )
+			dataId.setDataType(DataType.FRAME);
+		else if( dataType.equalsIgnoreCase("list") )
+			dataId.setDataType(DataType.LIST);
+		else if( dataType.equalsIgnoreCase("scalar") )
+			dataId.setDataType(DataType.SCALAR);
+
+		if( (shortVt && valueType.equals("int"))
+			|| valueType.equals("int") || valueType.equals("integer")
+			|| valueType.equals("Int") || valueType.equals("Integer")) {
+			dataId.setValueType(ValueType.INT);
+		}
+		else if( (shortVt && valueType.equals("str"))
+			|| valueType.equals("string") || valueType.equals("String")) {
+			dataId.setValueType(ValueType.STRING);
+		}
+		else if( (shortVt && valueType.equals("bool"))
+			|| valueType.equals("boolean") || valueType.equals("Boolean")) {
+			dataId.setValueType(ValueType.BOOLEAN);
+		}
+		else if( (shortVt && valueType.equals("float") )
+			|| valueType.equals("double") || valueType.equals("Double")) {
+			dataId.setValueType(ValueType.DOUBLE);
+		}
+		else if(valueType.equals("unknown") || (!shortVt && valueType.equals("Unknown"))) {
+			dataId.setValueType(ValueType.UNKNOWN);
+		}
+		else if(helpBool && valueType.equals("bool")) {
+			notifyErrorListeners("invalid valuetype " + valueType + " (Quickfix: use \'boolean\' instead)", start);
+			return false;
+		}
+		else {
+			notifyErrorListeners("invalid valuetype " + valueType, start);
+			return false;
 		}
+		return true;
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/main/java/org/apache/sysml/parser/dml/Dml.g4
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/dml/Dml.g4 b/src/main/java/org/apache/sysml/parser/dml/Dml.g4
index 6846f7a..e5c1f2c 100644
--- a/src/main/java/org/apache/sysml/parser/dml/Dml.g4
+++ b/src/main/java/org/apache/sysml/parser/dml/Dml.g4
@@ -190,8 +190,8 @@ ID : (ALPHABET (ALPHABET|DIGIT|'_')*  '::')? ALPHABET (ALPHABET|DIGIT|'_')*
 ml_type :  valueType | dataType '[' valueType ']';
 // Note to reduce number of keywords, these are case-sensitive,
 // To allow case-insenstive,  'int' becomes: ('i' | 'I') ('n' | 'N') ('t' | 'T')
-valueType: 'int' | 'integer' | 'string' | 'boolean' | 'double'
-            | 'Int' | 'Integer' | 'String' | 'Boolean' | 'Double';
+valueType: 'int' | 'integer' | 'string' | 'boolean' | 'double' | 'unknown'
+            | 'Int' | 'Integer' | 'String' | 'Boolean' | 'Double' | 'Unknown';
 dataType:
         // 'scalar' # ScalarDataTypeDummyCheck
         // |

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java b/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java
index c440be4..ba5b2db 100644
--- a/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java
+++ b/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java
@@ -38,8 +38,6 @@ import org.apache.sysml.parser.ConditionalPredicate;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DataIdentifier;
 import org.apache.sysml.parser.Expression;
-import org.apache.sysml.parser.Expression.DataType;
-import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.parser.ExpressionList;
 import org.apache.sysml.parser.ExternalFunctionStatement;
 import org.apache.sysml.parser.ForStatement;
@@ -702,49 +700,15 @@ public class DmlSyntacticValidator extends CommonSyntacticValidator implements D
 		ArrayList<DataIdentifier> retVal = new ArrayList<>();
 		for(TypedArgNoAssignContext paramCtx : ctx) {
 			DataIdentifier dataId = new DataIdentifier(paramCtx.paramName.getText());
-			String dataType = null;
-			String valueType = null;
-
-			if(paramCtx.paramType == null || paramCtx.paramType.dataType() == null
-					|| paramCtx.paramType.dataType().getText() == null || paramCtx.paramType.dataType().getText().isEmpty()) {
-				dataType = "scalar";
-			}
-			else {
-				dataType = paramCtx.paramType.dataType().getText();
-			}
-
-
+			String dataType = (paramCtx.paramType == null || paramCtx.paramType.dataType() == null
+				|| paramCtx.paramType.dataType().getText() == null || paramCtx.paramType.dataType().getText().isEmpty()) ?
+				"scalar" : paramCtx.paramType.dataType().getText();
+			String valueType = paramCtx.paramType.valueType().getText();
+			
 			//check and assign data type
 			checkValidDataType(dataType, paramCtx.start);
-			if( dataType.equalsIgnoreCase("matrix") )
-				dataId.setDataType(DataType.MATRIX);
-			else if( dataType.equalsIgnoreCase("frame") )
-				dataId.setDataType(DataType.FRAME);
-			else if( dataType.equalsIgnoreCase("scalar") )
-				dataId.setDataType(DataType.SCALAR);
-
-			valueType = paramCtx.paramType.valueType().getText();
-			if(valueType.equals("int") || valueType.equals("integer")
-				|| valueType.equals("Int") || valueType.equals("Integer")) {
-				dataId.setValueType(ValueType.INT);
-			}
-			else if(valueType.equals("string") || valueType.equals("String")) {
-				dataId.setValueType(ValueType.STRING);
-			}
-			else if(valueType.equals("boolean") || valueType.equals("Boolean")) {
-				dataId.setValueType(ValueType.BOOLEAN);
-			}
-			else if(valueType.equals("double") || valueType.equals("Double")) {
-				dataId.setValueType(ValueType.DOUBLE);
-			}
-			else if(valueType.equals("bool")) {
-				notifyErrorListeners("invalid valuetype " + valueType + " (Quickfix: use \'boolean\' instead)", paramCtx.start);
+			if( !setDataAndValueType(dataId, dataType, valueType, paramCtx.start, false, true) )
 				return null;
-			}
-			else {
-				notifyErrorListeners("invalid valuetype " + valueType, paramCtx.start);
-				return null;
-			}
 			retVal.add(dataId);
 		}
 		return retVal;

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java b/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
index 5060ebd..117dff1 100644
--- a/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
+++ b/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
@@ -42,8 +42,6 @@ import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DataIdentifier;
 import org.apache.sysml.parser.DoubleIdentifier;
 import org.apache.sysml.parser.Expression;
-import org.apache.sysml.parser.Expression.DataType;
-import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.parser.ExternalFunctionStatement;
 import org.apache.sysml.parser.ForStatement;
 import org.apache.sysml.parser.FunctionCallIdentifier;
@@ -1371,43 +1369,15 @@ public class PydmlSyntacticValidator extends CommonSyntacticValidator implements
 		ArrayList<DataIdentifier> retVal = new ArrayList<>();
 		for(TypedArgNoAssignContext paramCtx : ctx) {
 			DataIdentifier dataId = new DataIdentifier(paramCtx.paramName.getText());
-			String dataType = null;
-			String valueType = null;
-
-			if(paramCtx.paramType == null || paramCtx.paramType.dataType() == null
-					|| paramCtx.paramType.dataType().getText() == null || paramCtx.paramType.dataType().getText().isEmpty()) {
-				dataType = "scalar";
-			}
-			else {
-				dataType = paramCtx.paramType.dataType().getText();
-			}
-
+			String dataType = (paramCtx.paramType == null || paramCtx.paramType.dataType() == null
+				|| paramCtx.paramType.dataType().getText() == null || paramCtx.paramType.dataType().getText().isEmpty()) ?
+				"scalar" : paramCtx.paramType.dataType().getText();
+			String valueType = paramCtx.paramType.valueType().getText();
+			
 			//check and assign data type
 			checkValidDataType(dataType, paramCtx.start);
-			if( dataType.equals("matrix") )
-				dataId.setDataType(DataType.MATRIX);
-			else if( dataType.equals("frame") )
-				dataId.setDataType(DataType.FRAME);
-			else if( dataType.equals("scalar") )
-				dataId.setDataType(DataType.SCALAR);
-
-			valueType = paramCtx.paramType.valueType().getText();
-			if(valueType.equals("int")) {
-				dataId.setValueType(ValueType.INT);
-			}
-			else if(valueType.equals("str")) {
-				dataId.setValueType(ValueType.STRING);
-			}
-			else if(valueType.equals("bool")) {
-				dataId.setValueType(ValueType.BOOLEAN);
-			}
-			else if(valueType.equals("float")) {
-				dataId.setValueType(ValueType.DOUBLE);
-			}
-			else {
-				notifyErrorListeners("invalid valuetype " + valueType, paramCtx.start);
+			if( !setDataAndValueType(dataId, dataType, valueType, paramCtx.start, true, false) )
 				return null;
-			}
 			retVal.add(dataId);
 		}
 		return retVal;

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java b/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java
index a282b09..0d3707c 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java
@@ -34,6 +34,9 @@ public class ListAndStructTest extends AutomatedTestBase
 {
 	private static final String TEST_NAME1 = "ListUnnamed";
 	private static final String TEST_NAME2 = "ListNamed";
+	private static final String TEST_NAME3 = "ListUnnamedFun";
+	private static final String TEST_NAME4 = "ListNamedFun";
+	
 	
 	private static final String TEST_DIR = "functions/misc/";
 	private static final String TEST_CLASS_DIR = TEST_DIR + ListAndStructTest.class.getSimpleName() + "/";
@@ -43,6 +46,8 @@ public class ListAndStructTest extends AutomatedTestBase
 		TestUtils.clearAssertionInformation();
 		addTestConfiguration( TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1, new String[] { "R" }) );
 		addTestConfiguration( TEST_NAME2, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME2, new String[] { "R" }) );
+		addTestConfiguration( TEST_NAME3, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME3, new String[] { "R" }) );
+		addTestConfiguration( TEST_NAME4, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME4, new String[] { "R" }) );
 	}
 	
 	@Test
@@ -65,6 +70,26 @@ public class ListAndStructTest extends AutomatedTestBase
 		runListStructTest(TEST_NAME2, true);
 	}
 	
+	@Test
+	public void testListUnnamedFun() {
+		runListStructTest(TEST_NAME3, false);
+	}
+	
+	@Test
+	public void testListUnnamedFunRewrites() {
+		runListStructTest(TEST_NAME3, true);
+	}
+	
+	@Test
+	public void testListNamedFun() {
+		runListStructTest(TEST_NAME4, false);
+	}
+	
+	@Test
+	public void testListNamedFunRewrites() {
+		runListStructTest(TEST_NAME4, true);
+	}
+	
 	private void runListStructTest(String testname, boolean rewrites)
 	{
 		boolean oldFlag = OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION;

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/test/scripts/functions/misc/ListNamed.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ListNamed.dml b/src/test/scripts/functions/misc/ListNamed.dml
index 5b9b4e8..6158413 100644
--- a/src/test/scripts/functions/misc/ListNamed.dml
+++ b/src/test/scripts/functions/misc/ListNamed.dml
@@ -18,7 +18,7 @@
 # under the License.
 #
 #-------------------------------------------------------------
-
+          
 
 A = matrix(1, 10, 10);
 B = matrix(2, 10, 10);

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/test/scripts/functions/misc/ListNamedFun.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ListNamedFun.R b/src/test/scripts/functions/misc/ListNamedFun.R
new file mode 100644
index 0000000..abf01be
--- /dev/null
+++ b/src/test/scripts/functions/misc/ListNamedFun.R
@@ -0,0 +1,42 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+
+args <- commandArgs(TRUE)
+options(digits=22)
+library("Matrix")
+
+A = matrix(1, 10, 10);
+B = matrix(2, 10, 10);
+c = 3;
+D = matrix(4, 10, 10);
+
+X = list(a=A, b=B, c=c, d=D, e=matrix(5, 3, 3), f=6);
+
+for( i in 1:length(X) ) {
+  tmp = X[i]
+  if( !exists("tmp") )
+     print("ERROR: non-existing entry "+i );
+}
+
+R = as.matrix(sum(as.matrix(X[['e']])));
+
+writeMM(as(R, "CsparseMatrix"), paste(args[1], "R", sep=""));

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/test/scripts/functions/misc/ListNamedFun.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ListNamedFun.dml b/src/test/scripts/functions/misc/ListNamedFun.dml
new file mode 100644
index 0000000..b02065e
--- /dev/null
+++ b/src/test/scripts/functions/misc/ListNamedFun.dml
@@ -0,0 +1,40 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+foo = function(List[unknown] X) return (List[unknown] R) {
+  for( i in 1:length(X) ) {
+    tmp = X[i];
+    if( !exists(tmp) )
+      print("ERROR: non-existing entry "+i );
+  }
+  R = X['e'];   
+}          
+
+A = matrix(1, 10, 10);
+B = matrix(2, 10, 10);
+c = 3;
+D = matrix(4, 10, 10);
+
+X = list(a=A, b=B, c=c, d=D, e=matrix(5, 3, 3), f=6);
+R = as.matrix(foo(X));
+R = as.matrix(sum(R));
+
+write(R, $1);

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/test/scripts/functions/misc/ListUnnamedFun.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ListUnnamedFun.R b/src/test/scripts/functions/misc/ListUnnamedFun.R
new file mode 100644
index 0000000..ede3c2b
--- /dev/null
+++ b/src/test/scripts/functions/misc/ListUnnamedFun.R
@@ -0,0 +1,42 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+
+args <- commandArgs(TRUE)
+options(digits=22)
+library("Matrix")
+
+A = matrix(1, 10, 10);
+B = matrix(2, 10, 10);
+c = 3;
+D = matrix(4, 10, 10);
+
+X = list(A, B, c, D, matrix(5, 3, 3), 6);
+
+for( i in 1:length(X) ) {
+  tmp = X[i]
+  if( !exists("tmp") )
+     print("ERROR: non-existing entry "+i );
+}
+
+R = as.matrix(sum(as.matrix(X[[5]])));
+
+writeMM(as(R, "CsparseMatrix"), paste(args[1], "R", sep=""));

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae86c3f7/src/test/scripts/functions/misc/ListUnnamedFun.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ListUnnamedFun.dml b/src/test/scripts/functions/misc/ListUnnamedFun.dml
new file mode 100644
index 0000000..010afc6
--- /dev/null
+++ b/src/test/scripts/functions/misc/ListUnnamedFun.dml
@@ -0,0 +1,40 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+foo = function(List[unknown] X) return (List[unknown] R) {
+  for( i in 1:length(X) ) {
+    tmp = X[i];
+    if( !exists(tmp) )
+      print("ERROR: non-existing entry "+i );
+  }
+  R = X[5];   
+}
+
+A = matrix(1, 10, 10);
+B = matrix(2, 10, 10);
+c = 3;
+D = matrix(4, 10, 10);
+
+X = list(A, B, c, D, matrix(5, 3, 3), 6);
+R = as.matrix(foo(X));
+R = as.matrix(sum(R));
+
+write(R, $1);


[3/3] systemml git commit: [SYSTEMML-2314] Recompiler extensions for improved list-matrix lookups

Posted by mb...@apache.org.
[SYSTEMML-2314] Recompiler extensions for improved list-matrix lookups

This patch introduces additional rewrites to better handle lists during
dynamic recompilation. When compiling DAGs with as.matrix(X) or
as.matrix(X[5]), where X is a list, the matrix dimensions of the output
are unknown because they reflect the selected data object. However,
often during dynamic recompilation of individual DAGs we have 1-entry
lists or known index expressions, which allows us to replace these
patterns with the direct reference to the selected matrix, which allows
compiling the subsequent operations in the same DAG with unknown sizes
and thus, avoid unnecessary distributed operations.


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

Branch: refs/heads/master
Commit: e2875cae4fe9e367113f80b25f579489819d04fc
Parents: ae86c3f
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu May 10 20:32:21 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu May 10 20:32:48 2018 -0700

----------------------------------------------------------------------
 .../hops/recompile/LiteralReplacement.java      | 49 +++++++++++++++++++-
 .../java/org/apache/sysml/lops/compile/Dag.java |  5 +-
 .../functions/misc/ListAndStructTest.java       |  5 ++
 3 files changed, 55 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/e2875cae/src/main/java/org/apache/sysml/hops/recompile/LiteralReplacement.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/recompile/LiteralReplacement.java b/src/main/java/org/apache/sysml/hops/recompile/LiteralReplacement.java
index 575e187..7c5014c 100644
--- a/src/main/java/org/apache/sysml/hops/recompile/LiteralReplacement.java
+++ b/src/main/java/org/apache/sysml/hops/recompile/LiteralReplacement.java
@@ -34,11 +34,14 @@ import org.apache.sysml.hops.Hop.DataOpTypes;
 import org.apache.sysml.hops.Hop.Direction;
 import org.apache.sysml.hops.Hop.OpOp1;
 import org.apache.sysml.hops.rewrite.HopRewriteUtils;
+import org.apache.sysml.lops.compile.Dag;
 import org.apache.sysml.parser.Expression.DataType;
+import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.LocalVariableMap;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.instructions.cp.Data;
+import org.apache.sysml.runtime.instructions.cp.ListObject;
 import org.apache.sysml.runtime.instructions.cp.ScalarObject;
 import org.apache.sysml.runtime.instructions.cp.ScalarObjectFactory;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -49,7 +52,7 @@ public class LiteralReplacement
 	
 	//internal configuration parameters
 	private static final long REPLACE_LITERALS_MAX_MATRIX_SIZE = 1000000; //10^6 cells (8MB)
-	private static final boolean REPORT_LITERAL_REPLACE_OPS_STATS = true; 	
+	private static final boolean REPORT_LITERAL_REPLACE_OPS_STATS = true;
 	
 	protected static void rReplaceLiterals( Hop hop, LocalVariableMap vars, boolean scalarsOnly )
 	{
@@ -73,6 +76,8 @@ public class LiteralReplacement
 					lit = (lit==null) ? replaceLiteralValueTypeCastRightIndexing(c, vars) : lit;
 					lit = (lit==null) ? replaceLiteralFullUnaryAggregate(c, vars) : lit;
 					lit = (lit==null) ? replaceLiteralFullUnaryAggregateRightIndexing(c, vars) : lit;
+					lit = (lit==null) ? replaceTReadMatrixFromList(c, vars) : lit;
+					lit = (lit==null) ? replaceTReadMatrixLookupFromList(c, vars) : lit;
 				}
 				
 				//replace hop w/ literal on demand
@@ -342,7 +347,47 @@ public class LiteralReplacement
 		
 		return ret;
 	}
-
+	
+	private static DataOp replaceTReadMatrixFromList( Hop c, LocalVariableMap vars ) {
+		//pattern: as.matrix(X) or as.matrix(X) with X being a list
+		DataOp ret = null;
+		if( HopRewriteUtils.isUnary(c, OpOp1.CAST_AS_MATRIX) ) {
+			Hop in = c.getInput().get(0);
+			if( in.getDataType() == DataType.LIST
+				&& HopRewriteUtils.isData(in, DataOpTypes.TRANSIENTREAD) ) {
+				ListObject list = (ListObject)vars.get(in.getName());
+				String varname = Dag.getNextUniqueVarname(DataType.MATRIX);
+				MatrixObject mo = (MatrixObject) list.slice(0);
+				vars.put(varname, mo);
+				ret = HopRewriteUtils.createTransientRead(varname, c);
+			}
+		}
+		return ret;
+	}
+	
+	private static DataOp replaceTReadMatrixLookupFromList( Hop c, LocalVariableMap vars ) {
+		//pattern: as.matrix(X[i:i]) or as.matrix(X['a','a']) with X being a list
+		DataOp ret = null;
+		if( HopRewriteUtils.isUnary(c, OpOp1.CAST_AS_MATRIX)
+			&& c.getInput().get(0) instanceof IndexingOp ) {
+			Hop ix = c.getInput().get(0);
+			Hop ixIn = c.getInput().get(0).getInput().get(0);
+			if( ixIn.getDataType() == DataType.LIST
+				&& HopRewriteUtils.isData(ixIn, DataOpTypes.TRANSIENTREAD)
+				&& ix.getInput().get(1) instanceof LiteralOp 
+				&& ix.getInput().get(2) instanceof LiteralOp
+				&& ix.getInput().get(1) == ix.getInput().get(2) ) {
+				ListObject list = (ListObject)vars.get(ixIn.getName());
+				String varname = Dag.getNextUniqueVarname(DataType.MATRIX);
+				LiteralOp lit = (LiteralOp) ix.getInput().get(1);
+				MatrixObject mo = (MatrixObject) ((lit.getValueType() == ValueType.STRING) ?
+					list.slice(lit.getName()) : list.slice((int)lit.getLongValue()-1));
+				vars.put(varname, mo);
+				ret = HopRewriteUtils.createTransientRead(varname, c);
+			}
+		}
+		return ret;
+	}
 	
 	///////////////////////////////
 	// Utility functions

http://git-wip-us.apache.org/repos/asf/systemml/blob/e2875cae/src/main/java/org/apache/sysml/lops/compile/Dag.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/compile/Dag.java b/src/main/java/org/apache/sysml/lops/compile/Dag.java
index 3244a3b..bc080d3 100644
--- a/src/main/java/org/apache/sysml/lops/compile/Dag.java
+++ b/src/main/java/org/apache/sysml/lops/compile/Dag.java
@@ -212,8 +212,9 @@ public class Dag<N extends Lop>
 	}
 	
 	public static String getNextUniqueVarname(DataType dt) {
-		return (dt==DataType.MATRIX ? Lop.MATRIX_VAR_NAME_PREFIX :
-			Lop.FRAME_VAR_NAME_PREFIX) + var_index.getNextID();
+		return (dt.isMatrix() ? Lop.MATRIX_VAR_NAME_PREFIX :
+			dt.isFrame() ? Lop.FRAME_VAR_NAME_PREFIX :
+			Lop.SCALAR_VAR_NAME_PREFIX) + var_index.getNextID();
 	}
 	
 	///////

http://git-wip-us.apache.org/repos/asf/systemml/blob/e2875cae/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java b/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java
index 0d3707c..4c33847 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/misc/ListAndStructTest.java
@@ -29,6 +29,7 @@ import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
 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;
 
 public class ListAndStructTest extends AutomatedTestBase 
 {
@@ -114,6 +115,10 @@ public class ListAndStructTest extends AutomatedTestBase
 			HashMap<CellIndex, Double> dmlfile = readDMLMatrixFromHDFS("R");
 			HashMap<CellIndex, Double> rfile  = readRMatrixFromFS("R");
 			Assert.assertEquals(dmlfile.get(new CellIndex(1,1)), rfile.get(new CellIndex(1,1)));
+			
+			//check for properly compiled CP operations
+			Assert.assertTrue(Statistics.getNoOfExecutedMRJobs()==0);
+			Assert.assertTrue(Statistics.getNoOfExecutedSPInst()==0);
 		}
 		finally {
 			OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION = oldFlag;