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/07/19 05:06:06 UTC

[1/2] systemml git commit: [SYSTEMML-2455] Fix list indexing size propagation on parser-validate

Repository: systemml
Updated Branches:
  refs/heads/master 1049f5e56 -> 9593b7fbe


[SYSTEMML-2455] Fix list indexing size propagation on parser-validate

So far only the compiler from hops downward correctly propagated sizes
for list indexing. However, this led to validation issues on parsing for
operations like as.scalar which fails with incorrect input sizes. This
patch does a best effort size propagation for list and otherwise
indicates unknowns which is correct in all cases. Furthermore, this
patch also includes a refactoring of indexed identifier validation to
avoid code duplication.


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

Branch: refs/heads/master
Commit: 78c5c22884c77808913963a5eaf9d6236365d31d
Parents: 1049f5e
Author: Matthias Boehm <mb...@gmail.com>
Authored: Wed Jul 18 21:00:40 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Wed Jul 18 22:07:09 2018 -0700

----------------------------------------------------------------------
 .../org/apache/sysml/parser/Identifier.java     | 73 +++++++-------------
 .../functions/misc/ListAndStructTest.java       | 12 ++++
 .../scripts/functions/misc/ListIxAndCasts.R     | 35 ++++++++++
 .../scripts/functions/misc/ListIxAndCasts.dml   | 30 ++++++++
 4 files changed, 101 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/78c5c228/src/main/java/org/apache/sysml/parser/Identifier.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/Identifier.java b/src/main/java/org/apache/sysml/parser/Identifier.java
index bf36a2f..0a5f79d 100644
--- a/src/main/java/org/apache/sysml/parser/Identifier.java
+++ b/src/main/java/org/apache/sysml/parser/Identifier.java
@@ -141,59 +141,40 @@ public abstract class Identifier extends Expression
 	@Override
 	public void validateExpression(HashMap<String,DataIdentifier> ids, HashMap<String,ConstIdentifier> constVars, boolean conditional) 
 	{
-		
 		if( getOutput() instanceof DataIdentifier ) {
-			
 			// set properties for Data identifier
-			String name = ((DataIdentifier)this.getOutput()).getName();
+			String name = ((DataIdentifier)getOutput()).getName();
 			Identifier id = ids.get(name);
 			if ( id == null ){
 				//undefined variables are always treated unconditionally as error in order to prevent common script-level bugs
 				raiseValidateError("Undefined Variable (" + name + ") used in statement", false, LanguageErrorCodes.INVALID_PARAMETERS);
 			}
-			this.getOutput().setProperties(id);
+			getOutput().setProperties(id);
 			
 			// validate IndexedIdentifier -- which is substype of DataIdentifer with index
-			if (this.getOutput() instanceof IndexedIdentifier){
-				
+			if( getOutput() instanceof IndexedIdentifier ){
 				// validate the row / col index bounds (if defined)
-				IndexedIdentifier indexedIdentiferOut = (IndexedIdentifier)this.getOutput();
-				
-				if (indexedIdentiferOut.getRowLowerBound() != null) {
-					indexedIdentiferOut.getRowLowerBound().validateExpression(ids, constVars, conditional);
-					Expression tempExpr = indexedIdentiferOut.getRowLowerBound(); 
-					if (tempExpr.getOutput().getDataType() == Expression.DataType.MATRIX){
-						raiseValidateError("Matrix values for row lower index bound are not supported, which includes indexed identifiers.", conditional);
+				IndexedIdentifier ixId = (IndexedIdentifier)getOutput();
+				Expression[] exp = new Expression[]{ixId.getRowLowerBound(),
+					ixId.getRowUpperBound(), ixId.getColLowerBound(), ixId.getColUpperBound()};
+				String[] msg = new String[]{"row lower", "row upper", "column lower", "column upper"};
+				for( int i=0; i<4; i++ ) {
+					if( exp[i] != null ) {
+						exp[i].validateExpression(ids, constVars, conditional);
+						if (exp[i].getOutput().getDataType() == Expression.DataType.MATRIX){
+							raiseValidateError("Matrix values for "+msg[i]+" index bound are "
+								+ "not supported, which includes indexed identifiers.", conditional);
+						}
 					}
 				}
 				
-				if (indexedIdentiferOut.getRowUpperBound() != null) {
-					indexedIdentiferOut.getRowUpperBound().validateExpression(ids, constVars, conditional);
-					Expression tempExpr = indexedIdentiferOut.getRowUpperBound(); 
-					if (tempExpr.getOutput().getDataType() == Expression.DataType.MATRIX){
-						raiseValidateError("Matrix values for row upper index bound are not supported, which includes indexed identifiers.", conditional);
-					}
-				}
-				
-				if (indexedIdentiferOut.getColLowerBound() != null) {
-					indexedIdentiferOut.getColLowerBound().validateExpression(ids,constVars, conditional);
-					Expression tempExpr = indexedIdentiferOut.getColLowerBound(); 
-					if (tempExpr.getOutput().getDataType() == Expression.DataType.MATRIX){
-						raiseValidateError("Matrix values for column lower index bound are not supported, which includes indexed identifiers.", conditional);
-					}
-				}
-				
-				if (indexedIdentiferOut.getColUpperBound() != null) {
-					indexedIdentiferOut.getColUpperBound().validateExpression(ids, constVars, conditional);
-					Expression tempExpr = indexedIdentiferOut.getColUpperBound();
-					if (tempExpr.getOutput().getDataType() == Expression.DataType.MATRIX){
-						raiseValidateError("Matrix values for column upper index bound are not supported, which includes indexed identifiers.", conditional);
-					}
-				}
-				
-				if( this.getOutput().getDataType() != DataType.LIST ) {
-					IndexPair updatedIndices = ((IndexedIdentifier)this.getOutput()).calculateIndexedDimensions(ids, constVars, conditional);
-					((IndexedIdentifier)this.getOutput()).setDimensions(updatedIndices._row, updatedIndices._col);
+				if( getOutput().getDataType() == DataType.LIST ) {
+					int dim1 = (((IndexedIdentifier)getOutput()).getRowUpperBound() == null) ? 1 : - 1;
+					((IndexedIdentifier)getOutput()).setDimensions(dim1, 1);
+				} 
+				else { //default
+					IndexPair updatedIndices = ((IndexedIdentifier)getOutput()).calculateIndexedDimensions(ids, constVars, conditional);
+					((IndexedIdentifier)getOutput()).setDimensions(updatedIndices._row, updatedIndices._col);
 				}
 			}
 		}
@@ -203,15 +184,9 @@ public abstract class Identifier extends Expression
 	}
 	
 	public void computeDataType() {
-				
-		if ((_dim1 == 0) && (_dim2 == 0)) {
-			_dataType = DataType.SCALAR;
-		} else if ((_dim1 >= 1) || (_dim2 >= 1)){
-			// Vector also set as matrix
-			// Data type is set as matrix, if either of dimensions is -1
-			_dataType = DataType.MATRIX;
-		} else _dataType = DataType.UNKNOWN;	 
-		
+		_dataType = ((_dim1 == 0) && (_dim2 == 0)) ?
+			DataType.SCALAR : ((_dim1 >= 1) || (_dim2 >= 1)) ?
+			DataType.MATRIX : DataType.UNKNOWN;
 	}
 	
 	public void setBooleanProperties(){

http://git-wip-us.apache.org/repos/asf/systemml/blob/78c5c228/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 e8e3d1a..45eeca6 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
@@ -42,6 +42,7 @@ public class ListAndStructTest extends AutomatedTestBase
 	private static final String TEST_NAME7 = "ListAsMatrix";
 	private static final String TEST_NAME8 = "ListUnnamedRix";
 	private static final String TEST_NAME9 = "ListNamedRix";
+	private static final String TEST_NAME10 = "ListIxAndCasts";
 	
 	private static final String TEST_DIR = "functions/misc/";
 	private static final String TEST_CLASS_DIR = TEST_DIR + ListAndStructTest.class.getSimpleName() + "/";
@@ -58,6 +59,7 @@ public class ListAndStructTest extends AutomatedTestBase
 		addTestConfiguration( TEST_NAME7, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME7, new String[] { "R" }) );
 		addTestConfiguration( TEST_NAME8, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME8, new String[] { "R" }) );
 		addTestConfiguration( TEST_NAME9, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME9, new String[] { "R" }) );
+		addTestConfiguration( TEST_NAME10, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME10, new String[] { "R" }) );
 	}
 	
 	@Test
@@ -150,6 +152,16 @@ public class ListAndStructTest extends AutomatedTestBase
 		runListStructTest(TEST_NAME9, true);
 	}
 	
+	@Test
+	public void testListIndexingAndCasts() {
+		runListStructTest(TEST_NAME10, false);
+	}
+	
+	@Test
+	public void testListIndexingAndCastsRewrites() {
+		runListStructTest(TEST_NAME10, true);
+	}
+	
 	private void runListStructTest(String testname, boolean rewrites)
 	{
 		boolean oldFlag = OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION;

http://git-wip-us.apache.org/repos/asf/systemml/blob/78c5c228/src/test/scripts/functions/misc/ListIxAndCasts.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ListIxAndCasts.R b/src/test/scripts/functions/misc/ListIxAndCasts.R
new file mode 100644
index 0000000..35f92a5
--- /dev/null
+++ b/src/test/scripts/functions/misc/ListIxAndCasts.R
@@ -0,0 +1,35 @@
+#-------------------------------------------------------------
+#
+# 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")
+
+X = list(1.1, 2.2, 3.3, 4.4);
+Y = list(a=1.1, b=2.2, c=3.3, d=4.4);
+while(FALSE){}
+
+R = as.double(unlist(X[1])) + as.integer(unlist(X[3]));
+R = R + as.double(unlist(Y["a"])) + as.integer(unlist(Y["c"]));
+R = as.matrix(R);
+
+writeMM(as(R, "CsparseMatrix"), paste(args[1], "R", sep=""));

http://git-wip-us.apache.org/repos/asf/systemml/blob/78c5c228/src/test/scripts/functions/misc/ListIxAndCasts.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ListIxAndCasts.dml b/src/test/scripts/functions/misc/ListIxAndCasts.dml
new file mode 100644
index 0000000..b61d866
--- /dev/null
+++ b/src/test/scripts/functions/misc/ListIxAndCasts.dml
@@ -0,0 +1,30 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+X = list(1.1, 2.2, 3.3, 4.4);
+Y = list(a=1.1, b=2.2, c=3.3, d=4.4);
+while(FALSE){}
+
+R = as.double(as.scalar(X[1])) + as.integer(as.scalar(X[3]));
+R = R + as.double(as.scalar(Y["a"])) + as.integer(as.scalar(Y["c"]));
+R = as.matrix(R);
+
+write(R, $1);


[2/2] systemml git commit: [SYSTEMML-2440] Fix robustness value type casts to double/long

Posted by mb...@apache.org.
[SYSTEMML-2440] Fix robustness value type casts to double/long

This patch fixes special cases of value type casting to double and long.
Under special circumstances, scalars after list indexing can be of type
string but still hold the correct values. We now use a more robust
casting mechanism that works for all combinations of value types,
including strings and safe casts for double to long (according to
machine precision).

Furthermore, this also includes a fix of the parser to correctly
propagate unknown value types after list indexing to prevent rewrite
anomalies that incorrectly removed casts to double which again created
incorrect results for string inputs.


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

Branch: refs/heads/master
Commit: 9593b7fbef093c5976b21dec5c2dd4c33acebe00
Parents: 78c5c22
Author: Matthias Boehm <mb...@gmail.com>
Authored: Wed Jul 18 21:52:03 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Wed Jul 18 22:07:15 2018 -0700

----------------------------------------------------------------------
 .../hops/recompile/LiteralReplacement.java      | 24 ++++--------------
 .../sysml/parser/BuiltinFunctionExpression.java |  4 +--
 .../instructions/cp/ScalarObjectFactory.java    | 26 ++++++++++++++++++--
 .../instructions/cp/VariableCPInstruction.java  | 10 ++++----
 4 files changed, 36 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/9593b7fb/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 d6bac40..b2344c5 100644
--- a/src/main/java/org/apache/sysml/hops/recompile/LiteralReplacement.java
+++ b/src/main/java/org/apache/sysml/hops/recompile/LiteralReplacement.java
@@ -131,7 +131,7 @@ public class LiteralReplacement
 		return ret;
 	}
 	
-	private static LiteralOp replaceLiteralValueTypeCastScalarRead( Hop c, LocalVariableMap vars )
+	private static LiteralOp replaceLiteralValueTypeCastScalarRead(Hop c, LocalVariableMap vars)
 	{
 		LiteralOp ret = null;
 		
@@ -141,30 +141,16 @@ public class LiteralReplacement
 				&& c.getInput().get(0) instanceof DataOp && c.getDataType()==DataType.SCALAR )
 		{
 			Data dat = vars.get(c.getInput().get(0).getName());
-			if( dat != null ) //required for selective constant propagation
-			{
+			if( dat != null ) { //required for selective constant propagation
 				ScalarObject sdat = (ScalarObject)dat;
-				UnaryOp cast = (UnaryOp) c;
-				switch( cast.getOp() ) {
-					case CAST_AS_INT:
-						ret = new LiteralOp(sdat.getLongValue());		
-						break;
-					case CAST_AS_DOUBLE:
-						ret = new LiteralOp(sdat.getDoubleValue());		
-						break;						
-					case CAST_AS_BOOLEAN:
-						ret = new LiteralOp(sdat.getBooleanValue());		
-						break;
-					default:	
-						//otherwise: do nothing
-				}
-			}	
+				ret = ScalarObjectFactory.createLiteralOp(sdat, (UnaryOp) c);
+			}
 		}
 		
 		return ret;
 	}
 	
-	private static LiteralOp replaceLiteralValueTypeCastLiteral( Hop c, LocalVariableMap vars )
+	private static LiteralOp replaceLiteralValueTypeCastLiteral(Hop c, LocalVariableMap vars)
 	{
 		LiteralOp ret = null;
 		

http://git-wip-us.apache.org/repos/asf/systemml/blob/9593b7fb/src/main/java/org/apache/sysml/parser/BuiltinFunctionExpression.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/BuiltinFunctionExpression.java b/src/main/java/org/apache/sysml/parser/BuiltinFunctionExpression.java
index 519bd7d..c2c48cb 100644
--- a/src/main/java/org/apache/sysml/parser/BuiltinFunctionExpression.java
+++ b/src/main/java/org/apache/sysml/parser/BuiltinFunctionExpression.java
@@ -651,8 +651,8 @@ public class BuiltinFunctionExpression extends DataIdentifier
 			output.setDataType(DataType.SCALAR);
 			output.setDimensions(0, 0);
 			output.setBlockDimensions (0, 0);
-			output.setValueType((id.getValueType()!=ValueType.UNKNOWN) ?
-				id.getValueType() : ValueType.DOUBLE);
+			output.setValueType((id.getValueType()!=ValueType.UNKNOWN 
+				|| id.getDataType()==DataType.LIST) ? id.getValueType() : ValueType.DOUBLE);
 			break;
 		case CAST_AS_MATRIX:
 			checkNumParameters(1);

http://git-wip-us.apache.org/repos/asf/systemml/blob/9593b7fb/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarObjectFactory.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarObjectFactory.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarObjectFactory.java
index 7f5342a..b3d5d4a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarObjectFactory.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarObjectFactory.java
@@ -21,6 +21,7 @@ package org.apache.sysml.runtime.instructions.cp;
 
 import org.apache.sysml.hops.HopsException;
 import org.apache.sysml.hops.LiteralOp;
+import org.apache.sysml.hops.UnaryOp;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.util.UtilFunctions;
 
@@ -58,8 +59,8 @@ public abstract class ScalarObjectFactory
 	
 	public static ScalarObject createScalarObject(ValueType vt, ScalarObject so) {
 		switch( vt ) {
-			case DOUBLE:  return new DoubleObject(so.getDoubleValue());
-			case INT:     return new IntObject(so.getLongValue());
+			case DOUBLE:  return castToDouble(so);
+			case INT:     return castToLong(so);
 			case BOOLEAN: return new BooleanObject(so.getBooleanValue());
 			case STRING:  return new StringObject(so.getStringValue());
 			default: throw new RuntimeException("Unsupported scalar value type: "+vt.name());
@@ -90,4 +91,25 @@ public abstract class ScalarObjectFactory
 				throw new HopsException("Unsupported literal value type: "+so.getValueType());
 		}
 	}
+	
+	public static LiteralOp createLiteralOp(ScalarObject so, UnaryOp cast) {
+		switch( cast.getOp() ) {
+			case CAST_AS_DOUBLE:  return new LiteralOp(castToDouble(so).getDoubleValue());
+			case CAST_AS_INT:     return new LiteralOp(castToLong(so).getLongValue());
+			case CAST_AS_BOOLEAN: return new LiteralOp(so.getBooleanValue());
+			default: return null; //otherwise: do nothing
+		}
+	}
+	
+	public static IntObject castToLong(ScalarObject so) {
+		//note: cast with robustness for various combinations of value types
+		return new IntObject(!(so instanceof StringObject) ?
+			so.getLongValue() : UtilFunctions.toLong(Double.parseDouble(so.getStringValue())));
+	}
+	
+	public static DoubleObject castToDouble(ScalarObject so) {
+		//note: cast with robustness for various combinations of value types
+		return new DoubleObject(!(so instanceof StringObject) ?
+			so.getDoubleValue() : Double.parseDouble(so.getStringValue()));
+	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/9593b7fb/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java
index a1c89a3..4da1a17 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java
@@ -643,13 +643,13 @@ public class VariableCPInstruction extends CPInstruction {
 			break;
 		}
 		case CastAsDoubleVariable:{ 
-			ScalarObject scalarInput = ec.getScalarInput(getInput1());
-			ec.setScalarOutput(output.getName(), new DoubleObject(scalarInput.getDoubleValue()));
+			ScalarObject in = ec.getScalarInput(getInput1());
+			ec.setScalarOutput(output.getName(), ScalarObjectFactory.castToDouble(in));
 			break;
 		}
-		case CastAsIntegerVariable:{ 
-			ScalarObject scalarInput = ec.getScalarInput(getInput1());
-			ec.setScalarOutput(output.getName(), new IntObject(scalarInput.getLongValue()));
+		case CastAsIntegerVariable:{
+			ScalarObject in = ec.getScalarInput(getInput1());
+			ec.setScalarOutput(output.getName(), ScalarObjectFactory.castToLong(in));
 			break;
 		}
 		case CastAsBooleanVariable:{