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 2017/05/24 06:04:13 UTC

incubator-systemml git commit: [SYSTEMML-1621] Fix value type inference scalar-matrix max/min/log/ppred

Repository: incubator-systemml
Updated Branches:
  refs/heads/master 9845d966a -> d3968fee4


[SYSTEMML-1621] Fix value type inference scalar-matrix max/min/log/ppred

So far the value type inference for binary builtin functions simply
forwarded the value type of the left input which produced wrong meta
data for scalar-matrix operations where the scalar is of type integer
because the output matrix is always of type double. This patch fixes
this value type inference and consolidates the code path via a shared
primitive to set output properties of binary operations.


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

Branch: refs/heads/master
Commit: d3968fee46855106d34ae1fe742def4547739d1f
Parents: 9845d96
Author: Matthias Boehm <mb...@gmail.com>
Authored: Tue May 23 23:03:51 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Tue May 23 23:03:51 2017 -0700

----------------------------------------------------------------------
 .../api/mlcontext/MLContextConversionUtil.java  |   2 -
 .../sysml/parser/BuiltinFunctionExpression.java |  73 +++++------
 .../test/integration/AutomatedTestBase.java     |  16 ++-
 .../misc/ValueTypeMatrixScalarBuiltinTest.java  | 126 +++++++++++++++++++
 .../functions/misc/ValueTypeLogLeftScalar.dml   |  25 ++++
 .../functions/misc/ValueTypeLogRightScalar.dml  |  25 ++++
 .../functions/misc/ValueTypeMaxLeftScalar.dml   |  25 ++++
 .../functions/misc/ValueTypeMaxRightScalar.dml  |  25 ++++
 .../functions/misc/ValueTypePredLeftScalar.dml  |  25 ++++
 .../functions/misc/ValueTypePredRightScalar.dml |  25 ++++
 .../functions/misc/ZPackageSuite.java           |   3 +-
 11 files changed, 326 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/main/java/org/apache/sysml/api/mlcontext/MLContextConversionUtil.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/mlcontext/MLContextConversionUtil.java b/src/main/java/org/apache/sysml/api/mlcontext/MLContextConversionUtil.java
index b37f037..e392a5f 100644
--- a/src/main/java/org/apache/sysml/api/mlcontext/MLContextConversionUtil.java
+++ b/src/main/java/org/apache/sysml/api/mlcontext/MLContextConversionUtil.java
@@ -1309,8 +1309,6 @@ public class MLContextConversionUtil {
 			return frame;
 		} catch (CacheException e) {
 			throw new MLContextException("CacheException while converting frame object to 2D string array", e);
-		} catch (DMLRuntimeException e) {
-			throw new MLContextException("DMLRuntimeException while converting frame object to 2D string array", e);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/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 2cb869e..ca359d1 100644
--- a/src/main/java/org/apache/sysml/parser/BuiltinFunctionExpression.java
+++ b/src/main/java/org/apache/sysml/parser/BuiltinFunctionExpression.java
@@ -235,7 +235,7 @@ public class BuiltinFunctionExpression extends DataIdentifier
 		return newParams;
 	}
 
-	private ArrayList<ParameterExpression>  replaceListParams(ArrayList<ParameterExpression> paramExpression,
+	private ArrayList<ParameterExpression> replaceListParams(ArrayList<ParameterExpression> paramExpression,
 			String inputVarName, String outputVarName, int startIndex) throws LanguageException {
 		ArrayList<ParameterExpression> newParamExpression = new ArrayList<ParameterExpression>();
 		int i = startIndex;
@@ -408,31 +408,19 @@ public class BuiltinFunctionExpression extends DataIdentifier
 			//min(X), min(X,s), min(s,X), min(s,r), min(X,Y)
 			
 			//unary aggregate
-			if (getSecondExpr() == null) 
-			{
+			if (getSecondExpr() == null) {
 				checkNumParameters(1);
 				checkMatrixParam(getFirstExpr());
-				output.setDataType( DataType.SCALAR );
+				output.setDataType(DataType.SCALAR);
+				output.setValueType(id.getValueType());
 				output.setDimensions(0, 0);
 				output.setBlockDimensions (0, 0);
 			}
 			//binary operation
-			else
-			{
+			else {
 				checkNumParameters(2);
-				DataType dt1 = getFirstExpr().getOutput().getDataType();
-				DataType dt2 = getSecondExpr().getOutput().getDataType();
-				DataType dtOut = (dt1==DataType.MATRIX || dt2==DataType.MATRIX)?
-				                   DataType.MATRIX : DataType.SCALAR;				
-				if( dt1==DataType.MATRIX && dt2==DataType.MATRIX )
-					checkMatchingDimensions(getFirstExpr(), getSecondExpr(), true);
-				//determine output dimensions
-				long[] dims = getBinaryMatrixCharacteristics(getFirstExpr(), getSecondExpr());
-				output.setDataType( dtOut );
-				output.setDimensions(dims[0], dims[1]);
-				output.setBlockDimensions (dims[2], dims[3]);
+				setBinaryOutputProperties(output);
 			}
-			output.setValueType(id.getValueType());
 			
 			break;
 		
@@ -578,8 +566,6 @@ public class BuiltinFunctionExpression extends DataIdentifier
 				checkMatrixParam(getFirstExpr());
 			if( dt2 == DataType.MATRIX )
 				checkMatrixParam(getSecondExpr());
-			if( dt1==DataType.MATRIX && dt2==DataType.MATRIX ) //dt1==dt2
-			      checkMatchingDimensions(getFirstExpr(), getSecondExpr(), true);
 			
 			//check operator
 			if (getThirdExpr().getOutput().getDataType() != DataType.SCALAR || 
@@ -588,12 +574,7 @@ public class BuiltinFunctionExpression extends DataIdentifier
 				raiseValidateError("Third argument in ppred() is not an operator ", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
 			}
 			
-			//determine output dimensions
-			long[] dims = getBinaryMatrixCharacteristics(getFirstExpr(), getSecondExpr());
-			output.setDataType(DataType.MATRIX);
-			output.setDimensions(dims[0], dims[1]);
-			output.setBlockDimensions(dims[2], dims[3]);
-			output.setValueType(id.getValueType());
+			setBinaryOutputProperties(output);
 			break;
 
 		case TRANS:
@@ -1193,16 +1174,19 @@ public class BuiltinFunctionExpression extends DataIdentifier
 		}
 		default:
 			if (this.isMathFunction()) {
-				// datatype and dimensions are same as this.getExpr()
-				if (this.getOpCode() == BuiltinFunctionOp.ABS) {
-					output.setValueType(getFirstExpr().getOutput().getValueType());
-				} else {
-					output.setValueType(ValueType.DOUBLE);
-				}
 				checkMathFunctionParam();
-				output.setDataType(id.getDataType());
-				output.setDimensions(id.getDim1(), id.getDim2());
-				output.setBlockDimensions(id.getRowsInBlock(), id.getColumnsInBlock()); 
+				//unary operations
+				if( getSecondExpr() == null ) {
+					output.setDataType(id.getDataType());
+					output.setValueType((output.getDataType()==DataType.SCALAR
+						&& getOpCode()==BuiltinFunctionOp.ABS)?id.getValueType():ValueType.DOUBLE );
+					output.setDimensions(id.getDim1(), id.getDim2());
+					output.setBlockDimensions(id.getRowsInBlock(), id.getColumnsInBlock()); 
+				}
+				//binary operations
+				else {
+					setBinaryOutputProperties(output);
+				}
 			} 
 			else {
 				// always unconditional (because unsupported operation)
@@ -1216,6 +1200,23 @@ public class BuiltinFunctionExpression extends DataIdentifier
 		return;
 	}
 	
+	private void setBinaryOutputProperties(DataIdentifier output) 
+		throws LanguageException 
+	{
+		DataType dt1 = getFirstExpr().getOutput().getDataType();
+		DataType dt2 = getSecondExpr().getOutput().getDataType();
+		DataType dtOut = (dt1==DataType.MATRIX || dt2==DataType.MATRIX) ? 
+			DataType.MATRIX : DataType.SCALAR;				
+		if( dt1==DataType.MATRIX && dt2==DataType.MATRIX )
+			checkMatchingDimensions(getFirstExpr(), getSecondExpr(), true);
+		long[] dims = getBinaryMatrixCharacteristics(getFirstExpr(), getSecondExpr());
+		output.setDataType(dtOut);
+		output.setValueType(dtOut==DataType.MATRIX ? ValueType.DOUBLE : 
+			computeValueType(getFirstExpr(), getSecondExpr(), true));
+		output.setDimensions(dims[0], dims[1]);
+		output.setBlockDimensions (dims[2], dims[3]);
+	}
+	
 	private void expandArguments() {
 	
 		if ( _args == null ) {
@@ -1262,7 +1263,7 @@ public class BuiltinFunctionExpression extends DataIdentifier
 		case ACOS:
 		case ASIN:
 		case ATAN:
-		case SIGN:	
+		case SIGN:
 		case SQRT:
 		case ABS:
 		case LOG:

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java b/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java
index 263a90b..b8aa11e 100644
--- a/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java
+++ b/src/test/java/org/apache/sysml/test/integration/AutomatedTestBase.java
@@ -791,11 +791,6 @@ public abstract class AutomatedTestBase
 		Assert.assertEquals(mc.getCols(), rmc.getCols());
 	}
 	
-	/**
-	 * 
-	 * @param fileName
-	 * @return
-	 */
 	public static MatrixCharacteristics readDMLMetaDataFile(String fileName)
 	{
 		try {
@@ -810,6 +805,17 @@ public abstract class AutomatedTestBase
 		}
 	}
 	
+	public static ValueType readDMLMetaDataValueType(String fileName)
+	{
+		try {
+			String fname = baseDirectory + OUTPUT_DIR + fileName +".mtd";
+			JSONObject meta = new DataExpression().readMetadataFile(fname, false);
+			return ValueType.valueOf(meta.get(DataExpression.VALUETYPEPARAM).toString().toUpperCase());
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
 	
 	/**
 	 * <p>

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/test/java/org/apache/sysml/test/integration/functions/misc/ValueTypeMatrixScalarBuiltinTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/misc/ValueTypeMatrixScalarBuiltinTest.java b/src/test/java/org/apache/sysml/test/integration/functions/misc/ValueTypeMatrixScalarBuiltinTest.java
new file mode 100644
index 0000000..61ffa7d
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/misc/ValueTypeMatrixScalarBuiltinTest.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.test.integration.functions.misc;
+
+import org.apache.sysml.parser.Expression.ValueType;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.integration.TestConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ValueTypeMatrixScalarBuiltinTest extends AutomatedTestBase
+{	
+	private final static String TEST_NAME1 = "ValueTypeMaxLeftScalar";
+	private final static String TEST_NAME2 = "ValueTypeMaxRightScalar";
+	private final static String TEST_NAME3 = "ValueTypeLogLeftScalar";
+	private final static String TEST_NAME4 = "ValueTypeLogRightScalar";
+	private final static String TEST_NAME5 = "ValueTypePredLeftScalar";
+	private final static String TEST_NAME6 = "ValueTypePredRightScalar";
+	
+	private final static String TEST_DIR = "functions/misc/";
+	private static final String TEST_CLASS_DIR = TEST_DIR + ValueTypeMatrixScalarBuiltinTest.class.getSimpleName() + "/";
+	
+	@Override
+	public void setUp() {
+		addTestConfiguration(TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1, new String[] {}));
+		addTestConfiguration(TEST_NAME2, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME2, new String[] {}));
+		addTestConfiguration(TEST_NAME3, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME3, new String[] {}));
+		addTestConfiguration(TEST_NAME4, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME4, new String[] {}));
+		addTestConfiguration(TEST_NAME5, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME5, new String[] {}));
+		addTestConfiguration(TEST_NAME6, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME6, new String[] {}));
+	}
+
+	@Test
+	public void testValueTypeMaxLeftScalarDouble() { 
+		runTest(TEST_NAME1, ValueType.DOUBLE); 
+	}
+	
+	@Test
+	public void testValueTypeMaxLeftScalarInt() { 
+		runTest(TEST_NAME1, ValueType.INT); 
+	}
+	
+	@Test
+	public void testValueTypeMaxRightScalarDouble() { 
+		runTest(TEST_NAME2, ValueType.DOUBLE); 
+	}
+	
+	@Test
+	public void testValueTypeMaxRightScalarInt() { 
+		runTest(TEST_NAME2, ValueType.INT); 
+	}
+	
+	@Test
+	public void testValueTypeLogLeftScalarDouble() { 
+		runTest(TEST_NAME3, ValueType.DOUBLE); 
+	}
+	
+	@Test
+	public void testValueTypeLogLeftScalarInt() { 
+		runTest(TEST_NAME3, ValueType.INT); 
+	}
+	
+	@Test
+	public void testValueTypeLogRightScalarDouble() { 
+		runTest(TEST_NAME4, ValueType.DOUBLE); 
+	}
+	
+	@Test
+	public void testValueTypeLogRightScalarInt() { 
+		runTest(TEST_NAME4, ValueType.INT); 
+	}
+	
+	@Test
+	public void testValueTypePredLeftScalarDouble() { 
+		runTest(TEST_NAME5, ValueType.DOUBLE); 
+	}
+	
+	@Test
+	public void testValueTypePredLeftScalarInt() { 
+		runTest(TEST_NAME5, ValueType.INT); 
+	}
+	
+	@Test
+	public void testValueTypePredRightScalarDouble() { 
+		runTest(TEST_NAME6, ValueType.DOUBLE); 
+	}
+	
+	@Test
+	public void testValueTypePredRightScalarInt() { 
+		runTest(TEST_NAME6, ValueType.INT); 
+	}
+	
+	private void runTest(String testName, ValueType vtIn) 
+	{
+		loadTestConfiguration(getTestConfiguration(testName));
+		
+		//setup arguments and run test
+        String RI_HOME = SCRIPT_DIR + TEST_DIR;
+		fullDMLScriptName = RI_HOME + testName + ".dml";
+		programArgs = new String[]{"-args", 
+			vtIn==ValueType.DOUBLE ? "7.7" : "7", output("R")};
+		runTest(true, false, null, -1);
+		
+		//check output value type
+		ValueType vtOut = readDMLMetaDataValueType("R");
+		Assert.assertTrue("Wrong output value type: " + 
+			vtOut.name(), vtOut.equals(ValueType.DOUBLE));
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/test/scripts/functions/misc/ValueTypeLogLeftScalar.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ValueTypeLogLeftScalar.dml b/src/test/scripts/functions/misc/ValueTypeLogLeftScalar.dml
new file mode 100644
index 0000000..42d2d2b
--- /dev/null
+++ b/src/test/scripts/functions/misc/ValueTypeLogLeftScalar.dml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# 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 = rand(rows=10, cols=10);
+Y = log($1, X);
+write(Y, $2);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/test/scripts/functions/misc/ValueTypeLogRightScalar.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ValueTypeLogRightScalar.dml b/src/test/scripts/functions/misc/ValueTypeLogRightScalar.dml
new file mode 100644
index 0000000..f142bcf
--- /dev/null
+++ b/src/test/scripts/functions/misc/ValueTypeLogRightScalar.dml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# 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 = rand(rows=10, cols=10);
+Y = log(X, $1);
+write(Y, $2);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/test/scripts/functions/misc/ValueTypeMaxLeftScalar.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ValueTypeMaxLeftScalar.dml b/src/test/scripts/functions/misc/ValueTypeMaxLeftScalar.dml
new file mode 100644
index 0000000..db819b1
--- /dev/null
+++ b/src/test/scripts/functions/misc/ValueTypeMaxLeftScalar.dml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# 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 = rand(rows=10, cols=10);
+Y = max($1, X);
+write(Y, $2);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/test/scripts/functions/misc/ValueTypeMaxRightScalar.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ValueTypeMaxRightScalar.dml b/src/test/scripts/functions/misc/ValueTypeMaxRightScalar.dml
new file mode 100644
index 0000000..a67deda
--- /dev/null
+++ b/src/test/scripts/functions/misc/ValueTypeMaxRightScalar.dml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# 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 = rand(rows=10, cols=10);
+Y = max(X, $1);
+write(Y, $2);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/test/scripts/functions/misc/ValueTypePredLeftScalar.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ValueTypePredLeftScalar.dml b/src/test/scripts/functions/misc/ValueTypePredLeftScalar.dml
new file mode 100644
index 0000000..cc09f42
--- /dev/null
+++ b/src/test/scripts/functions/misc/ValueTypePredLeftScalar.dml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# 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 = rand(rows=10, cols=10);
+Y = ppred($1, X, ">");
+write(Y, $2);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/test/scripts/functions/misc/ValueTypePredRightScalar.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ValueTypePredRightScalar.dml b/src/test/scripts/functions/misc/ValueTypePredRightScalar.dml
new file mode 100644
index 0000000..19deb5e
--- /dev/null
+++ b/src/test/scripts/functions/misc/ValueTypePredRightScalar.dml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# 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 = rand(rows=10, cols=10);
+Y = ppred(X, $1, "<");
+write(Y, $2);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d3968fee/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java
index 81eb089..cba9120 100644
--- a/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java
+++ b/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java
@@ -64,7 +64,8 @@ import org.junit.runners.Suite;
 	SetWorkingDirTest.class,
 	ToStringTest.class,
 	ValueTypeAutoCastingTest.class,
-	ValueTypeCastingTest.class
+	ValueTypeCastingTest.class,
+	ValueTypeMatrixScalarBuiltinTest.class,
 })