You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by du...@apache.org on 2016/05/13 15:40:09 UTC

incubator-systemml git commit: [SYSTEMML-294] Print matrix capability

Repository: incubator-systemml
Updated Branches:
  refs/heads/master 72390e07c -> ea49e620c


[SYSTEMML-294] Print matrix capability

This adds the ability to print a matrix by introducing a new `toString(...)` function that takes in a matrix and a set of optional arguments, and then outputs a string representation of the matrix for printing.

* Parameters supported : "rows", "cols", "decimal", "sparse",
  "separator", "lineseparator"
* Default parameter values: rows=cols=100, decimal=3, sparse=FALSE,
  separator=" ", lineseparator="\n"

Closes #120.


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

Branch: refs/heads/master
Commit: ea49e620ce846c41c8b9bb9b729986a9aa7fd4d5
Parents: 72390e0
Author: Nakul Jindal <na...@gmail.com>
Authored: Fri May 13 08:38:56 2016 -0700
Committer: Mike Dusenberry <mw...@us.ibm.com>
Committed: Fri May 13 08:38:56 2016 -0700

----------------------------------------------------------------------
 src/main/java/org/apache/sysml/hops/Hop.java    |   4 +-
 .../sysml/hops/ParameterizedBuiltinOp.java      | 118 +++++++-
 .../apache/sysml/lops/ParameterizedBuiltin.java |   7 +
 .../org/apache/sysml/parser/DMLTranslator.java  |  10 +-
 .../org/apache/sysml/parser/Expression.java     |   1 +
 .../ParameterizedBuiltinFunctionExpression.java |  36 ++-
 .../java/org/apache/sysml/parser/dml/Dml.g4     |   4 +-
 .../instructions/CPInstructionParser.java       |   4 +-
 .../cp/ParameterizedBuiltinCPInstruction.java   |  49 ++++
 .../cp/ScalarScalarArithmeticCPInstruction.java |   6 +-
 .../sysml/runtime/util/DataConverter.java       |  74 +++++
 .../functions/misc/ToStringTest.java            | 275 +++++++++++++++++++
 .../org/apache/sysml/test/utils/TestUtils.java  |  19 +-
 src/test/scripts/functions/misc/ToString1.dml   |  28 ++
 src/test/scripts/functions/misc/ToString10.dml  |  24 ++
 src/test/scripts/functions/misc/ToString11.dml  |  24 ++
 src/test/scripts/functions/misc/ToString2.dml   |  24 ++
 src/test/scripts/functions/misc/ToString3.dml   |  24 ++
 src/test/scripts/functions/misc/ToString4.dml   |  24 ++
 src/test/scripts/functions/misc/ToString5.dml   |  24 ++
 src/test/scripts/functions/misc/ToString6.dml   |  24 ++
 src/test/scripts/functions/misc/ToString7.dml   |  24 ++
 src/test/scripts/functions/misc/ToString8.dml   |  24 ++
 src/test/scripts/functions/misc/ToString9.dml   |  24 ++
 .../functions/misc/ZPackageSuite.java           |   1 +
 25 files changed, 855 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/hops/Hop.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/Hop.java b/src/main/java/org/apache/sysml/hops/Hop.java
index 782be9a..273c1a2 100644
--- a/src/main/java/org/apache/sysml/hops/Hop.java
+++ b/src/main/java/org/apache/sysml/hops/Hop.java
@@ -1033,7 +1033,7 @@ public abstract class Hop
 
 	public enum OpOp1 {
 		NOT, ABS, SIN, COS, TAN, ASIN, ACOS, ATAN, SIGN, SQRT, LOG, EXP, 
-		CAST_AS_SCALAR, CAST_AS_MATRIX, CAST_AS_FRAME, CAST_AS_DOUBLE, CAST_AS_INT, CAST_AS_BOOLEAN, 
+		CAST_AS_SCALAR, CAST_AS_MATRIX, CAST_AS_FRAME, CAST_AS_DOUBLE, CAST_AS_INT, CAST_AS_BOOLEAN,
 		PRINT, EIGEN, NROW, NCOL, LENGTH, ROUND, IQM, STOP, CEIL, FLOOR, MEDIAN, INVERSE, CHOLESKY,
 		//cumulative sums, products, extreme values
 		CUMSUM, CUMPROD, CUMMIN, CUMMAX,
@@ -1090,6 +1090,7 @@ public abstract class Hop
 	public enum ParamBuiltinOp {
 		INVALID, CDF, INVCDF, GROUPEDAGG, RMEMPTY, REPLACE, REXPAND, 
 		TRANSFORM, TRANSFORMAPPLY, TRANSFORMDECODE, TRANSFORMMETA,
+		TOSTRING
 	};
 
 	/**
@@ -1334,6 +1335,7 @@ public abstract class Hop
 		HopsParameterizedBuiltinLops.put(ParamBuiltinOp.TRANSFORMAPPLY, org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.TRANSFORMAPPLY);		
 		HopsParameterizedBuiltinLops.put(ParamBuiltinOp.TRANSFORMDECODE, org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.TRANSFORMDECODE);
 		HopsParameterizedBuiltinLops.put(ParamBuiltinOp.TRANSFORMMETA, org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.TRANSFORMMETA);
+		HopsParameterizedBuiltinLops.put(ParamBuiltinOp.TOSTRING, org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.TOSTRING);		
 	}
 
 	protected static final HashMap<Hop.OpOp2, String> HopsOpOp2String;

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java b/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
index 9217c77..a793fb1 100644
--- a/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
+++ b/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
@@ -110,6 +110,19 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 	public HashMap<String, Integer> getParamIndexMap(){
 		return _paramIndexMap;
 	}
+	
+	/**
+	 * Returns a parameters by its name. Returns null if not present  
+	 * @param val
+	 * @return
+	 */
+	public Hop getInputParameter(String val){
+		Integer index = getParamIndexMap().get(val);
+		if (index == null)
+			return null;
+		else
+			return getInput().get(index);
+	}
 		
 	@Override
 	public String getOpString() {
@@ -198,7 +211,8 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 			case REPLACE:
 			case TRANSFORMAPPLY: 
 			case TRANSFORMDECODE: 
-			case TRANSFORMMETA: { 
+			case TRANSFORMMETA: 
+			case TOSTRING: {
 				ExecType et = optFindExecType();			
 				ParameterizedBuiltin pbilop = new ParameterizedBuiltin(inputlops,
 						HopsParameterizedBuiltinLops.get(_op), getDataType(), getValueType(), et);
@@ -826,8 +840,100 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 	@Override
 	protected double computeOutputMemEstimate( long dim1, long dim2, long nnz )
 	{	
-		double sparsity = OptimizerUtils.getSparsity(dim1, dim2, nnz);
-		return OptimizerUtils.estimateSizeExactSparsity(dim1, dim2, sparsity);	
+		if (getOp() == ParamBuiltinOp.TOSTRING){
+			// Conservative Assumptions about characteristics of digits
+			final long AVERAGE_CHARS_PER_VALUE = 7;
+			final long AVERAGE_CHARS_PER_INDEX = 4;
+			
+			// Default Values for toString
+			long specifiedRows = 100;
+			long specifiedCols = 100;
+			boolean sparsePrint = false;
+			String sep = " ";
+			String linesep = "\n";
+			
+			Hop rowsHop = getInputParameter("rows");
+			Hop colsHop = getInputParameter("cols");
+			Hop sparsePrintHOP = getInputParameter("sparse");
+			Hop sepHop = getInputParameter("sep");
+			Hop linesepHop = getInputParameter("linesep");
+			
+			long numNonZeroes = getInput().get(0).getNnz();
+			if (numNonZeroes < 0)
+				numNonZeroes = specifiedRows * specifiedCols;
+			long numRows = getInput().get(0).getDim1();
+			if (numRows < 0) 	// If number of rows is not known, set to default
+				numRows = specifiedRows;
+			long numCols = getInput().get(0).getDim2();
+			if (numCols < 0)	// If number of columns is not known, set to default
+				numCols = specifiedCols;
+			
+			
+			// Assume Defaults : 100 * 100, sep = " ", linesep = "\n", sparse = false
+			// String size in bytes is 36 + number_of_chars * 2
+			final long DEFAULT_SIZE = 36 + 2 *
+					(100 * 100 * AVERAGE_CHARS_PER_VALUE 	// Length for digits  
+					+ 1 * 100 * 99 							// Length for separator chars
+					+ 1* 100) ;								// Length for line separator chars
+			
+			try {
+			
+				if (rowsHop != null && rowsHop instanceof LiteralOp) {
+					specifiedRows = ((LiteralOp)rowsHop).getLongValue();
+				}
+				numRows = numRows < specifiedRows ? numRows : specifiedRows;
+				if (colsHop != null && colsHop instanceof LiteralOp){
+					specifiedCols = ((LiteralOp)colsHop).getLongValue();
+				}
+				numCols = numCols < specifiedCols ? numCols : specifiedCols;
+				
+				if (sparsePrintHOP != null && sparsePrintHOP instanceof LiteralOp){
+					sparsePrint = ((LiteralOp)sparsePrintHOP).getBooleanValue();
+				}
+				
+				if (sepHop != null && sepHop instanceof LiteralOp){
+					sep = ((LiteralOp)sepHop).getStringValue();
+				}
+				
+				if (linesepHop != null && linesepHop instanceof LiteralOp){
+					linesep = ((LiteralOp)linesepHop).getStringValue();
+				}
+				
+				long numberOfChars = -1;
+				
+				if (sparsePrint){
+					numberOfChars = AVERAGE_CHARS_PER_VALUE * numNonZeroes			// Length for value digits
+									+ AVERAGE_CHARS_PER_INDEX * 2L * numNonZeroes	// Length for row & column index
+									+ sep.length() * 2L * numNonZeroes				// Length for separator chars
+									+ linesep.length() * numNonZeroes;				// Length for line separator chars
+				} else {
+					numberOfChars = AVERAGE_CHARS_PER_VALUE * numRows * numCols 	// Length for digits
+									+ sep.length() * numRows * (numCols - 1) 		// Length for separator chars
+									+ linesep.length() * numRows;					// Length for line separator chars
+				}
+				
+				/**
+				 * For JVM
+				 * 8 + // object header used by the VM
+				 * 8 + // 64-bit reference to char array (value)
+				 * 8 + string.length() * 2 + // character array itself (object header + 16-bit chars)
+				 * 4 + // offset integer
+				 * 4 + // count integer
+				 * 4 + // cached hash code
+				 */
+				
+				return (36 + numberOfChars * 2);
+				
+			} catch (HopsException e){
+				LOG.warn("Invalid values when trying to compute dims1, dims2 & nnz", e);
+				
+				return DEFAULT_SIZE;
+			}
+			
+		} else {
+			double sparsity = OptimizerUtils.getSparsity(dim1, dim2, nnz);
+			return OptimizerUtils.estimateSizeExactSparsity(dim1, dim2, sparsity);
+		}
 	}
 	
 	@Override
@@ -881,6 +987,8 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 	{
 		//CDF always known because 
 		
+		// TOSTRING outputs a string
+		
 		long[] ret = null;
 	
 		Hop input = getInput().get(_paramIndexMap.get("target"));	
@@ -1018,7 +1126,8 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 		//force CP for in-memory only transform builtins
 		if( _op == ParamBuiltinOp.TRANSFORMAPPLY
 			|| _op == ParamBuiltinOp.TRANSFORMDECODE
-			|| _op == ParamBuiltinOp.TRANSFORMMETA ) {
+			|| _op == ParamBuiltinOp.TRANSFORMMETA 
+			||  _op == ParamBuiltinOp.TOSTRING) {
 			_etype = ExecType.CP;
 		}
 		
@@ -1104,6 +1213,7 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 				Hop target = getInput().get(_paramIndexMap.get("target"));
 				setDim1( target.getDim1() ); //rows remain unchanged
 			}
+				break;
 			default:
 				//do nothing
 				break;

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/lops/ParameterizedBuiltin.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/ParameterizedBuiltin.java b/src/main/java/org/apache/sysml/lops/ParameterizedBuiltin.java
index 0096de8..140adaa 100644
--- a/src/main/java/org/apache/sysml/lops/ParameterizedBuiltin.java
+++ b/src/main/java/org/apache/sysml/lops/ParameterizedBuiltin.java
@@ -42,6 +42,7 @@ public class ParameterizedBuiltin extends Lop
 		INVALID, CDF, INVCDF, RMEMPTY, REPLACE, REXPAND, 
 		PNORM, QNORM, PT, QT, PF, QF, PCHISQ, QCHISQ, PEXP, QEXP,
 		TRANSFORM, TRANSFORMAPPLY, TRANSFORMDECODE, TRANSFORMMETA,
+		TOSTRING
 	};
 	
 	private OperationTypes _operation;
@@ -250,6 +251,12 @@ public class ParameterizedBuiltin extends Lop
 				sb.append(compileGenericParamMap(_inputParams));
 				break;
 			}			
+			case TOSTRING:{
+				sb.append("toString"); //opcode
+				sb.append(OPERAND_DELIMITOR);
+				sb.append(compileGenericParamMap(_inputParams));
+				break;
+			}
 				
 			default:
 				throw new LopsException(this.printErrorLocation() + "In ParameterizedBuiltin Lop, Unknown operation: " + _operation);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/parser/DMLTranslator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/DMLTranslator.java b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
index 9e710e3..4daeef7 100644
--- a/src/main/java/org/apache/sysml/parser/DMLTranslator.java
+++ b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
@@ -2032,8 +2032,7 @@ public class DMLTranslator
 									target.getName(), target.getDataType(), 
 									target.getValueType(), ParamBuiltinOp.TRANSFORMDECODE, 
 									paramHops);
-			break;	
-			
+			break;			
 
 		case TRANSFORMMETA:
 			currBuiltinOp = new ParameterizedBuiltinOp(
@@ -2041,6 +2040,13 @@ public class DMLTranslator
 									target.getValueType(), ParamBuiltinOp.TRANSFORMMETA, 
 									paramHops);
 			break;		
+		
+		case TOSTRING:
+			currBuiltinOp = new ParameterizedBuiltinOp(
+									target.getName(), target.getDataType(), 
+									target.getValueType(), ParamBuiltinOp.TOSTRING, 
+									paramHops);
+			break;
 			
 		default:
 			

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/parser/Expression.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/Expression.java b/src/main/java/org/apache/sysml/parser/Expression.java
index e41654c..d6347a1 100644
--- a/src/main/java/org/apache/sysml/parser/Expression.java
+++ b/src/main/java/org/apache/sysml/parser/Expression.java
@@ -146,6 +146,7 @@ public abstract class Expression
 		// Distribution Functions
 		CDF, INVCDF, PNORM, QNORM, PT, QT, PF, QF, PCHISQ, QCHISQ, PEXP, QEXP,
 		TRANSFORM, TRANSFORMAPPLY, TRANSFORMDECODE, TRANSFORMENCODE, TRANSFORMMETA,
+		TOSTRING,	// The "toString" method for DML; named arguments accepted to format output
 		INVALID
 	};
 	

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/parser/ParameterizedBuiltinFunctionExpression.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/ParameterizedBuiltinFunctionExpression.java b/src/main/java/org/apache/sysml/parser/ParameterizedBuiltinFunctionExpression.java
index ef1745f..eea6f55 100644
--- a/src/main/java/org/apache/sysml/parser/ParameterizedBuiltinFunctionExpression.java
+++ b/src/main/java/org/apache/sysml/parser/ParameterizedBuiltinFunctionExpression.java
@@ -20,7 +20,9 @@
 package org.apache.sysml.parser;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 
 import org.apache.sysml.hops.Hop.ParamBuiltinOp;
 import org.apache.sysml.parser.LanguageException.LanguageErrorCodes;
@@ -69,6 +71,9 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 		opcodeMap.put("transformdecode", Expression.ParameterizedBuiltinFunctionOp.TRANSFORMDECODE);
 		opcodeMap.put("transformencode", Expression.ParameterizedBuiltinFunctionOp.TRANSFORMENCODE);
 		opcodeMap.put("transformmeta", Expression.ParameterizedBuiltinFunctionOp.TRANSFORMMETA);
+
+		// toString
+		opcodeMap.put("toString", Expression.ParameterizedBuiltinFunctionOp.TOSTRING);
 	}
 	
 	public static HashMap<Expression.ParameterizedBuiltinFunctionOp, ParamBuiltinOp> pbHopMap;
@@ -96,6 +101,9 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 		pbHopMap.put(Expression.ParameterizedBuiltinFunctionOp.QF, ParamBuiltinOp.INVCDF);
 		pbHopMap.put(Expression.ParameterizedBuiltinFunctionOp.QCHISQ, ParamBuiltinOp.INVCDF);
 		pbHopMap.put(Expression.ParameterizedBuiltinFunctionOp.QEXP, ParamBuiltinOp.INVCDF);
+		
+		// toString
+		pbHopMap.put(Expression.ParameterizedBuiltinFunctionOp.TOSTRING, ParamBuiltinOp.TOSTRING);
 	}
 	
 	public static ParameterizedBuiltinFunctionExpression getParamBuiltinFunctionExpression(String functionName, ArrayList<ParameterExpression> paramExprsPassed,
@@ -243,7 +251,11 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 		
 		case TRANSFORMMETA:
 			validateTransformMeta(output, conditional);
-			break;	
+			break;
+			
+		case TOSTRING:
+			validateCastAsString(output, conditional);
+			break;
 			
 		default: //always unconditional (because unsupported operation)
 			raiseValidateError("Unsupported parameterized function "+ getOpCode(), false, LanguageErrorCodes.INVALID_PARAMETERS);
@@ -725,6 +737,28 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 	}
 	
 
+	private void validateCastAsString(DataIdentifier output, boolean conditional) throws LanguageException {
+		
+		HashMap<String, Expression> varParams = getVarParams();
+		// null is for the matrix argument
+		String[] validArgsArr = {null, "rows", "cols", "decimal", "sparse", "sep", "linesep"};
+		HashSet<String> validArgs = new HashSet<String>(Arrays.asList(validArgsArr));
+		for (String k : varParams.keySet()){
+			if (!validArgs.contains(k)){
+				String errMsg = "Invalid parameter " + k + " for as.string, valid parameters are " + validArgsArr[0];
+				for (int i=1; i<validArgsArr.length; ++i) 
+					errMsg += "," + validArgsArr[i];
+				raiseValidateError(errMsg, conditional, LanguageErrorCodes.INVALID_PARAMETERS);
+			}
+		}
+		
+		// Output is a string
+		output.setDataType(DataType.SCALAR);
+		output.setValueType(ValueType.STRING);
+		output.setDimensions(0, 0);
+	}
+
+
 	/**
 	 * 
 	 * @param fname

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/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 31fcec0..64598ff 100644
--- a/src/main/java/org/apache/sysml/parser/dml/Dml.g4
+++ b/src/main/java/org/apache/sysml/parser/dml/Dml.g4
@@ -29,9 +29,9 @@ grammar Dml;
  * 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

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java b/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
index 5c3f591..48ef9a8 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
@@ -1,4 +1,5 @@
 /*
+
  * 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
@@ -185,7 +186,8 @@ public class CPInstructionParser extends InstructionParser
 		String2CPInstructionType.put( "transformdecode",CPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2CPInstructionType.put( "transformencode",CPINSTRUCTION_TYPE.MultiReturnParameterizedBuiltin);
 		String2CPInstructionType.put( "transformmeta",CPINSTRUCTION_TYPE.ParameterizedBuiltin);
-
+		String2CPInstructionType.put( "toString"    , CPINSTRUCTION_TYPE.ParameterizedBuiltin);
+		
 		// Variable Instruction Opcodes 
 		String2CPInstructionType.put( "assignvar"   , CPINSTRUCTION_TYPE.Variable);
 		String2CPInstructionType.put( "cpvar"    	, CPINSTRUCTION_TYPE.Variable);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
index 750ecfd..12bfbc6 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
@@ -43,6 +43,7 @@ import org.apache.sysml.runtime.transform.TfUtils;
 import org.apache.sysml.runtime.transform.decode.Decoder;
 import org.apache.sysml.runtime.transform.decode.DecoderFactory;
 import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
+import org.apache.sysml.runtime.util.DataConverter;
 
 
 public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction 
@@ -132,6 +133,10 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 		{
 			return new ParameterizedBuiltinCPInstruction(null, paramsMap, out, opcode, str);
 		}
+		else if (	opcode.equals("toString"))
+		{
+			return new ParameterizedBuiltinCPInstruction(null, paramsMap, out, opcode, str);
+		}
 		else {
 			throw new DMLRuntimeException("Unknown opcode (" + opcode + ") for ParameterizedBuiltin Instruction.");
 		}
@@ -288,6 +293,50 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 			//release locks
 			ec.setFrameOutput(output.getName(), meta);
 		}
+		else if ( opcode.equalsIgnoreCase("toString")) {
+			// Default Arguments
+			final int MAXROWS = 100;
+			final int MAXCOLS = 100;
+			final int DECIMAL = 3;
+			final boolean SPARSE = false;
+			final String SEPARATOR = " ";
+			final String LINESEPARATOR = "\n";
+			
+			int rows=MAXROWS, cols=MAXCOLS, decimal=DECIMAL;
+			boolean sparse = SPARSE;
+			String separator=SEPARATOR, lineseparator=LINESEPARATOR; 
+			
+			String rowsStr = getParameterMap().get("rows");
+			if (rowsStr != null){ rows = Integer.parseInt(rowsStr); }
+			
+			String colsStr = getParameterMap().get("cols");
+			if (colsStr != null) { cols = Integer.parseInt(rowsStr); }
+			
+			String decimalStr = getParameterMap().get("decimal");
+			if (decimalStr != null) { decimal = Integer.parseInt(decimalStr); }
+			
+			String sparseStr = getParameterMap().get("sparse");
+			if (sparseStr != null) { sparse = Boolean.parseBoolean(sparseStr); }
+			
+			String separatorStr = getParameterMap().get("sep");
+			if (separatorStr != null) { separator = separatorStr; }
+			
+			String lineseparatorStr = getParameterMap().get("linesep");
+			if (lineseparatorStr != null) { lineseparator = lineseparatorStr; }
+			
+			// The matrix argument is "null"
+			String matrixStr = getParameterMap().get("null");
+			Data data = ec.getVariable(matrixStr);
+			if (!(data instanceof MatrixObject))
+				throw new DMLRuntimeException("toString only converts matrix objects to string");
+			MatrixBlock matrix = ec.getMatrixInput(matrixStr);
+
+			String outputStr = DataConverter.convertToString(matrix, sparse, separator, lineseparator, rows, cols, decimal);
+			
+			ec.releaseMatrixInput(matrixStr);
+			ec.setScalarOutput(output.getName(), new StringObject(outputStr));
+			
+		}
 		else {
 			throw new DMLRuntimeException("Unknown opcode : " + opcode);
 		}		

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarArithmeticCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarArithmeticCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarArithmeticCPInstruction.java
index 030bfbc..11bd779 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarArithmeticCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarArithmeticCPInstruction.java
@@ -57,7 +57,11 @@ public class ScalarScalarArithmeticCPInstruction extends ArithmeticBinaryCPInstr
 			//pre-check (for robustness regarding too long strings)
 			String val1 = so1.getStringValue();
 			String val2 = so2.getStringValue();
-			StringObject.checkMaxStringLength(val1.length() + val2.length());
+			// This line was commented out because of the addition of 
+			// the built-in function toString.
+			// The toString function adds its own memory estimation
+			// and does not need a hard check on the limit of the size of the string.
+			// StringObject.checkMaxStringLength(val1.length() + val2.length());
 			
 			String rval = dop.fn.execute(val1, val2);
 			sores = new StringObject(rval);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/main/java/org/apache/sysml/runtime/util/DataConverter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/util/DataConverter.java b/src/main/java/org/apache/sysml/runtime/util/DataConverter.java
index 211c054..6974d9b 100644
--- a/src/main/java/org/apache/sysml/runtime/util/DataConverter.java
+++ b/src/main/java/org/apache/sysml/runtime/util/DataConverter.java
@@ -20,6 +20,7 @@
 package org.apache.sysml.runtime.util;
 
 import java.io.IOException;
+import java.text.DecimalFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -846,4 +847,77 @@ public class DataConverter
 		mo.release();		
 		return new Array2DRowRealMatrix(data, false);
 	}
+	
+	/**
+	 * Returns a string representation of a matrix
+	 * @param mb
+	 * @param sparse if true, string will contain a table with row index, col index, value (where value != 0.0)
+	 * 				 otherwise it will be a rectangular string with all values of the matrix block
+	 * @param separator Separator string between each element in a row, or between the columns in sparse format
+	 * @param lineseparator Separator string between each row
+	 * @param rowsToPrint maximum number of rows to print, -1 for all
+	 * @param colsToPrint maximum number of columns to print, -1 for all
+	 * @param decimal number of decimal places to print, -1 for default
+	 * @return
+	 */
+	public static String convertToString(MatrixBlock mb, boolean sparse, String separator, String lineseparator, int rowsToPrint, int colsToPrint, int decimal){
+		StringBuffer sb = new StringBuffer();
+		
+		// Setup number of rows and columns to print
+		int rlen = mb.getNumRows();
+		int clen = mb.getNumColumns();
+		int rowLength = rlen;
+		int colLength = clen;
+		if (rowsToPrint >= 0)
+			rowLength = rowsToPrint < rlen ? rowsToPrint : rlen;
+		if (colsToPrint >= 0)
+			colLength = colsToPrint < clen ? colsToPrint : clen;
+		
+		DecimalFormat df = new DecimalFormat();
+		df.setGroupingUsed(false);
+		if (decimal >= 0){
+			df.setMinimumFractionDigits(decimal);
+		}
+		
+		if (sparse){ // Sparse Print Format
+			if (mb.isInSparseFormat()){	// Block is in sparse format
+				Iterator<IJV> sbi = mb.getSparseBlockIterator();
+				while (sbi.hasNext()){
+					IJV ijv = sbi.next();
+					int row = ijv.getI();
+					int col = ijv.getJ();
+					double value = ijv.getV();
+					if (row < rowLength && col < colLength) {
+						// Print (row+1) and (col+1) since for a DML user, everything is 1-indexed
+						sb.append(row+1).append(separator).append(col+1).append(separator);
+						sb.append(df.format(value)).append(lineseparator);
+					}
+				}
+			} else {	// Block is in dense format
+				for (int i=0; i<rowLength; ++i){
+					for (int j=0; j<colLength; ++j){
+						double value = mb.getValue(i, j);
+						if (value != 0.0){
+							sb.append(i+1).append(separator).append(j+1).append(separator);
+							sb.append(df.format(value)).append(lineseparator);
+						}
+					}
+				}
+			}
+		}
+		else {	// Dense Print Format
+			for (int i=0; i<rowLength; i++){
+				for (int j=0; j<colLength-1; j++){
+					double value = mb.quickGetValue(i, j);
+					sb.append(df.format(value));
+					sb.append(separator);
+				}
+				double value = mb.quickGetValue(i, colLength-1);
+				sb.append(df.format(value));	// Do not put separator after last element
+				sb.append(lineseparator);
+			}
+		}
+		
+		return sb.toString();
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/java/org/apache/sysml/test/integration/functions/misc/ToStringTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/misc/ToStringTest.java b/src/test/java/org/apache/sysml/test/integration/functions/misc/ToStringTest.java
new file mode 100644
index 0000000..c823849
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/misc/ToStringTest.java
@@ -0,0 +1,275 @@
+/*
+ * 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.api.DMLScript;
+import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.integration.TestConfiguration;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+public class ToStringTest extends AutomatedTestBase {
+
+	 private static final String TEST_DIR = "functions/misc/";
+	 private static final String TEST_CLASS_DIR = TEST_DIR + ToStringTest.class.getSimpleName() + "/";
+	 private static final String OUTPUT_NAME = "tostring";
+
+	
+	@Override
+	public void setUp() {
+		TestUtils.clearAssertionInformation();
+	}
+	
+	/**
+	 * Default parameters
+	 */
+	@Test
+	public void testdefaultPrint(){
+		String testName = "ToString1";
+		String expectedOutput = 
+				"1.000 2.000 3.000 4.000 5.000\n" +  
+				"6.000 7.000 8.000 9.000 10.000\n"+
+				"11.000 12.000 13.000 14.000 15.000\n" +
+				"16.000 17.000 18.000 19.000 20.000\n";
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, expectedOutput);
+	}
+	
+	/**
+	 * Specify number of rows and columns on small matrix
+	 */
+	@Test
+	public void testRowsColsPrint(){
+		String testName = "ToString2";
+		String expectedOutput = 
+				"1.000 2.000 3.000\n" +
+				"5.000 6.000 7.000\n" +
+				"9.000 10.000 11.000\n";		
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, expectedOutput);
+	}
+
+	/**
+	 * Change number of digits after decimal
+	 */
+	@Test
+	public void testDecimal(){
+		String testName = "ToString3";
+		String expectedOutput = 
+				"1.00 2.00 3.00\n" +
+				"4.00 5.00 6.00\n" +
+				"7.00 8.00 9.00\n";		
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, expectedOutput);
+	}
+	
+	/**
+	 * Change separator character
+	 */
+	@Test
+	public void testSeparator(){
+		String testName = "ToString4";
+		String expectedOutput = 
+				"1.000 | 2.000 | 3.000\n" +
+				"4.000 | 5.000 | 6.000\n" +
+				"7.000 | 8.000 | 9.000\n";		
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, expectedOutput);
+	}
+	
+	/**
+	 * Change line separator char
+	 */
+	@Test
+	public void testLineSeparator(){
+		String testName = "ToString5";
+		String expectedOutput = 
+				"1.000 2.000 3.000\t" +
+				"4.000 5.000 6.000\t" +
+				"7.000 8.000 9.000\t";		
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, expectedOutput);
+	}
+	
+
+	/**
+	 * Initialize a big matrix ( > 100x100), print to see if defaults kick in
+	 */
+	@Test
+	public void testBiggerArrayDefaultRowsCols(){
+		final int INPUT_ROWS = 200;
+		final int INPUT_COLS = 200;
+		final int MAX_ROWS = 100;
+		final int MAX_COLS = 100;
+		final String SEP = " ";
+		final String LINESEP = "\n";
+				
+		String testName = "ToString6";
+		StringBuilder sb = new StringBuilder();
+		long k=1;
+		long i=1, j=1;
+		for (i=1; i<=MAX_ROWS; i++){
+			for (j=1; j<=MAX_COLS-1; j++){
+				sb.append(k).append(".000").append(SEP);
+				k++;
+			}
+			sb.append(k).append(".000").append(LINESEP);
+			k++; j++;
+			k += (INPUT_COLS - j + 1);
+		}
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, sb.toString());
+	}
+	
+	/**
+	 * Initialize a big matrix ( > 100x100), specify rows and cols, bigger than default
+	 */
+	@Test
+	public void testBiggerArraySpecifyRowsCols(){
+		final int INPUT_ROWS = 200;
+		final int INPUT_COLS = 200;
+		final int MAX_ROWS = 190;
+		final int MAX_COLS = 190;
+		final String SEP = " ";
+		final String LINESEP = "\n";
+				
+		String testName = "ToString7";
+		StringBuilder sb = new StringBuilder();
+		long k=1;
+		long i=1, j=1;
+		for (i=1; i<=MAX_ROWS; i++){
+			for (j=1; j<=MAX_COLS-1; j++){
+				sb.append(k).append(".000").append(SEP);
+				k++;
+			}
+			sb.append(k).append(".000").append(LINESEP);
+			k++; j++;
+			k += (INPUT_COLS - j + 1);
+		}
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, sb.toString());
+	}
+	
+	/**
+	 * Basic sparse print test
+	 */
+	@Test
+	public void testSparsePrint(){
+		String testName = "ToString8";
+		String expectedOutput = "1 1 1.000\n" +
+								"1 2 2.000\n" +
+								"2 1 3.000\n" +
+								"2 2 4.000\n" +
+								"3 1 5.000\n" +
+								"3 2 6.000\n" +
+								"4 1 7.000\n" +
+								"4 2 8.000\n";
+								
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, expectedOutput);
+	}
+	
+	/**
+	 * Basic sparse print test with zeroes
+	 */
+	@Test
+	public void testSparsePrintWithZeroes(){
+		String testName = "ToString9";
+		String expectedOutput = "1 1 1.000\n" +
+								"1 2 2.000\n" +
+								"1 3 3.000\n" +
+								"1 4 4.000\n" +
+								"3 1 5.000\n" +
+								"3 2 6.000\n" +
+								"3 3 7.000\n";
+								
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, expectedOutput);
+	}
+	
+	/**
+	 * Sparse print with specified separator and lineseparator
+	 */
+	@Test
+	public void testSparsePrintWithZeroesAndFormatting(){
+		String testName = "ToString10";
+		String expectedOutput = "1  1  1.000|" +
+								"1  2  2.000|" +
+								"1  3  3.000|" +
+								"1  4  4.000|" +
+								"3  1  5.000|" +
+								"3  2  6.000|" +
+								"3  3  7.000|";
+								
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, expectedOutput);
+	}
+	
+	/**
+	 * Sparse print with custom number of rows and columns
+	 */
+	@Test
+	public void testSparsePrintWithZeroesRowsCols(){
+		String testName = "ToString11";
+		String expectedOutput = "1 1 1.000\n" +
+								"1 2 2.000\n" +
+								"1 3 3.000\n" +
+								"2 1 2.000\n" +
+								"3 1 5.000\n" +
+								"3 2 6.000\n" +
+								"3 3 7.000\n";
+								
+		addTestConfiguration(testName, new TestConfiguration(TEST_CLASS_DIR, testName));
+		toStringTestHelper(RUNTIME_PLATFORM.SINGLE_NODE, testName, expectedOutput);
+	}
+
+	protected void toStringTestHelper(RUNTIME_PLATFORM platform, String testName, String expectedOutput) {
+		RUNTIME_PLATFORM platformOld = rtplatform;
+		
+		rtplatform = platform;
+		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
+        if (rtplatform == RUNTIME_PLATFORM.SPARK)
+            DMLScript.USE_LOCAL_SPARK_CONFIG = true;
+        try {
+            // Create and load test configuration
+        	getAndLoadTestConfiguration(testName);
+            String HOME = SCRIPT_DIR + TEST_DIR;
+            fullDMLScriptName = HOME + testName + ".dml";
+            programArgs = new String[]{"-args", output(OUTPUT_NAME)};
+
+
+            // Run DML and R scripts
+            runTest(true, false, null, -1);
+
+            // Compare output strings
+            String output = TestUtils.readDMLString(output(OUTPUT_NAME));
+            TestUtils.compareScalars(expectedOutput, output);
+           
+        }
+        finally {
+            // Reset settings
+            rtplatform = platformOld;
+            DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
+        }
+	}
+	
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/java/org/apache/sysml/test/utils/TestUtils.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/utils/TestUtils.java b/src/test/java/org/apache/sysml/test/utils/TestUtils.java
index da17587..9e3c55d 100644
--- a/src/test/java/org/apache/sysml/test/utils/TestUtils.java
+++ b/src/test/java/org/apache/sysml/test/utils/TestUtils.java
@@ -46,6 +46,7 @@ import java.util.Random;
 import java.util.StringTokenizer;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -60,6 +61,8 @@ import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
 import org.apache.sysml.test.integration.BinaryMatrixCharacteristics;
 
+import junit.framework.Assert;
+
 
 /**
  * <p>
@@ -585,21 +588,15 @@ public class TestUtils
 	public static String readDMLString(String filePath) {
 		FileSystem fs;
 		try {
-			String s =  null;
+			StringBuilder sb =  new StringBuilder();
 			fs = FileSystem.get(conf);
 			Path outDirectory = new Path(filePath);
-			String line;
 			FileStatus[] outFiles = fs.listStatus(outDirectory);
 			for (FileStatus file : outFiles) {
 				FSDataInputStream fsout = fs.open(file.getPath());
-				BufferedReader outIn = new BufferedReader(new InputStreamReader(fsout));
-				
-				while ((line = outIn.readLine()) != null) { // only 1 scalar value in file
-					s = line; 
-				}
-				outIn.close();
+				sb.append(IOUtils.toString(new InputStreamReader(fsout)));
 			}
-			return s;
+			return sb.toString();
 		} catch (IOException e) {
 			assertTrue("could not read from file " + filePath, false);
 		}
@@ -747,6 +744,10 @@ public class TestUtils
 			assertTrue("Given scalars do not match: " + d1 + " != " + d2 , false);
 		}
 	}
+	
+	public static void compareScalars(String expected, String actual) {
+			assertEquals(expected, actual);
+	}
 
 	/**
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString1.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString1.dml b/src/test/scripts/functions/misc/ToString1.dml
new file mode 100644
index 0000000..a4615cd
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString1.dml
@@ -0,0 +1,28 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix(seq(1, 20), rows=4, cols=5)
+# 1 2 3 4 5
+# 6 7 8 9 10
+# 11 12 13 14 15
+# 16 17 18 19 20
+str = toString(X)
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString10.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString10.dml b/src/test/scripts/functions/misc/ToString10.dml
new file mode 100644
index 0000000..4e856e8
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString10.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix("1 2 3 4 0 0 0 0 5 6 7 0", rows=3, cols=4)
+str = toString(X, sparse=TRUE, sep="  ", linesep="|")
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString11.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString11.dml b/src/test/scripts/functions/misc/ToString11.dml
new file mode 100644
index 0000000..b568ca2
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString11.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix("1 2 3 4 2 0 0 0 5 6 7 0 8 9 10 11", rows=4, cols=4)
+str = toString(X, sparse=TRUE, rows=3, cols=3)
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString2.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString2.dml b/src/test/scripts/functions/misc/ToString2.dml
new file mode 100644
index 0000000..7e87178
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString2.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix(seq(1, 16), rows=4, cols=4)
+str = toString(X, rows=3, cols=3)
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString3.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString3.dml b/src/test/scripts/functions/misc/ToString3.dml
new file mode 100644
index 0000000..4d0c4b8
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString3.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix(seq(1, 9), rows=3, cols=3)
+str = toString(X, decimal=2)
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString4.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString4.dml b/src/test/scripts/functions/misc/ToString4.dml
new file mode 100644
index 0000000..bbd677d
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString4.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix(seq(1, 9), rows=3, cols=3)
+str = toString(X, sep=" | ")
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString5.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString5.dml b/src/test/scripts/functions/misc/ToString5.dml
new file mode 100644
index 0000000..1f66322
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString5.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix(seq(1, 9), rows=3, cols=3)
+str = toString(X, linesep="\t")
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString6.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString6.dml b/src/test/scripts/functions/misc/ToString6.dml
new file mode 100644
index 0000000..f4f245c
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString6.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix(seq(1, 40000), rows=200, cols=200)
+str = toString(X)
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString7.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString7.dml b/src/test/scripts/functions/misc/ToString7.dml
new file mode 100644
index 0000000..4eeb687
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString7.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix(seq(1, 40000), rows=200, cols=200)
+str = toString(X, rows=190, cols=190)
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString8.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString8.dml b/src/test/scripts/functions/misc/ToString8.dml
new file mode 100644
index 0000000..fae287c
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString8.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix(seq(1, 8), rows=4, cols=2)
+str = toString(X, sparse=TRUE)
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/src/test/scripts/functions/misc/ToString9.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/ToString9.dml b/src/test/scripts/functions/misc/ToString9.dml
new file mode 100644
index 0000000..0ff547d
--- /dev/null
+++ b/src/test/scripts/functions/misc/ToString9.dml
@@ -0,0 +1,24 @@
+#-------------------------------------------------------------
+#
+# 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 = matrix("1 2 3 4 0 0 0 0 5 6 7 0", rows=3, cols=4)
+str = toString(X, sparse=TRUE)
+write(str, $1)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ea49e620/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 23d8947..f535de8 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
@@ -51,6 +51,7 @@ import org.junit.runners.Suite;
 	ScalarMatrixUnaryBinaryTermTest.class,
 	ScalarToMatrixInLoopTest.class,
 	SetWorkingDirTest.class,
+	ToStringTest.class,
 	ValueTypeAutoCastingTest.class,
 	ValueTypeCastingTest.class
 })