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 2016/03/11 20:18:09 UTC

[2/4] incubator-systemml git commit: [SYSTEMML-555] Transform builtin cleanup (spec as string argument)

[SYSTEMML-555] Transform builtin cleanup (spec as string argument)

So far transform took the 'transform specification' as a json file,
where the file path was an input argument. This approach is very
inflexibile with regard to CP operations and jmlc scoring, where we aim
for pure in-memory computation. This patch changes the transform
definition to take the json string as an input argument - now the
specification can be either read from a scalar file or constructed in
memory and passed to transform. Note, however, that we are not yet able
to construct the json specification in DML via string concatenation due
to missing functionality for string escaping wrt quotes. 

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

Branch: refs/heads/master
Commit: ccfe921b84214a527bd8407ff05cedcb720ea558
Parents: 29d5108
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Thu Mar 10 19:28:38 2016 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Thu Mar 10 19:28:38 2016 -0800

----------------------------------------------------------------------
 scripts/algorithms/transform.dml                |   4 +-
 scripts/datagen/genRandData4DecisionTree2.dml   |   3 +-
 .../apache/sysml/lops/ParameterizedBuiltin.java |   4 +-
 .../ParameterizedBuiltinFunctionExpression.java |  20 +--
 .../ParameterizedBuiltinSPInstruction.java      |   6 +-
 .../sysml/runtime/transform/DataTransform.java  | 150 +++++++------------
 .../scripts/functions/transform/Scaling.dml     |   3 +-
 .../scripts/functions/transform/Transform.dml   |   3 +-
 .../functions/transform/Transform_colnames.dml  |   3 +-
 9 files changed, 82 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ccfe921b/scripts/algorithms/transform.dml
----------------------------------------------------------------------
diff --git a/scripts/algorithms/transform.dml b/scripts/algorithms/transform.dml
index 11dcc60..f96e250 100644
--- a/scripts/algorithms/transform.dml
+++ b/scripts/algorithms/transform.dml
@@ -22,5 +22,7 @@
 cmdLine_fmt = ifdef($FMT, "csv")
 
 raw = read($DATA_PATH);
-A = transform(target=raw, transformPath=$TRANSFORM_PATH, transformSpec=$TRANSFORM_SPEC_PATH, outputNames=$OUTPUT_NAMES);
+specJson = read($TRANSFORM_SPEC_PATH, data_type="scalar", value_type"=string");
+
+A = transform(target=raw, transformPath=$TRANSFORM_PATH, spec=specJson, outputNames=$OUTPUT_NAMES);
 write(A, $OUTPUT_DATA_PATH, format=cmdLine_fmt);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ccfe921b/scripts/datagen/genRandData4DecisionTree2.dml
----------------------------------------------------------------------
diff --git a/scripts/datagen/genRandData4DecisionTree2.dml b/scripts/datagen/genRandData4DecisionTree2.dml
index cc8341c..e71bedb 100644
--- a/scripts/datagen/genRandData4DecisionTree2.dml
+++ b/scripts/datagen/genRandData4DecisionTree2.dml
@@ -34,7 +34,8 @@ X_scale = rand (rows = num_records, cols = num_scale_features, min = 0, max = 10
 
 # transform categorical features
 XCF = read (XCatFile);
-X_cat_transformed = transform (target = XCF, transformSpec = transformSpec, transformPath = transformPath);
+specJson = read(transformSpec, data_type="scalar", value_type="string");
+X_cat_transformed = transform (target = XCF, spec = specJson, transformPath = transformPath);
 
 X = append (X_scale, X_cat_transformed);
 write (X, XFile, format = fmt);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ccfe921b/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 f27ef65..17ec274 100644
--- a/src/main/java/org/apache/sysml/lops/ParameterizedBuiltin.java
+++ b/src/main/java/org/apache/sysml/lops/ParameterizedBuiltin.java
@@ -455,7 +455,7 @@ public class ParameterizedBuiltin extends Lop
 			sb.append(iLop.prepInputOperand(getInputIndex("target")));
 			sb.append( OPERAND_DELIMITOR );
 			
-			Lop iLop2 = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_TXMTD);
+			Lop iLop2 = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_MTD);
 			sb.append(iLop2.prepScalarLabel());
 			sb.append( OPERAND_DELIMITOR );
 			
@@ -468,7 +468,7 @@ public class ParameterizedBuiltin extends Lop
 				iLop3 = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_APPLYMTD);
 			}
 			else {
-				iLop3 = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_TXSPEC);
+				iLop3 = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_SPEC);
 			}
 			sb.append(iLop3.prepScalarLabel());
 			sb.append( OPERAND_DELIMITOR );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ccfe921b/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 b0fe09d..8d1f454 100644
--- a/src/main/java/org/apache/sysml/parser/ParameterizedBuiltinFunctionExpression.java
+++ b/src/main/java/org/apache/sysml/parser/ParameterizedBuiltinFunctionExpression.java
@@ -33,8 +33,8 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 	private HashMap<String,Expression> _varParams;
 	
 	public static final String TF_FN_PARAM_DATA = "target";
-	public static final String TF_FN_PARAM_TXMTD = "transformPath";
-	public static final String TF_FN_PARAM_TXSPEC = "transformSpec";
+	public static final String TF_FN_PARAM_MTD = "transformPath";
+	public static final String TF_FN_PARAM_SPEC = "spec";
 	public static final String TF_FN_PARAM_APPLYMTD = "applyTransformPath";
 	public static final String TF_FN_PARAM_OUTNAMES = "outputNames";
 	
@@ -247,22 +247,22 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 			raiseValidateError("Input to tansform() must be of type 'frame'. It is of type '"+data.getOutput().getDataType()+"'.", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
 		}	
 		
-		Expression txmtd = getVarParam(TF_FN_PARAM_TXMTD);
+		Expression txmtd = getVarParam(TF_FN_PARAM_MTD);
 		if( txmtd==null ) {
-			raiseValidateError("Named parameter '" + TF_FN_PARAM_TXMTD + "' missing. Please specify the transformation metadata file path.", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
+			raiseValidateError("Named parameter '" + TF_FN_PARAM_MTD + "' missing. Please specify the transformation metadata file path.", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
 		}
 		else if( txmtd.getOutput().getDataType() != DataType.SCALAR || txmtd.getOutput().getValueType() != ValueType.STRING ){				
-			raiseValidateError("Transformation metadata file '" + TF_FN_PARAM_TXMTD + "' must be a string value (a scalar).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
+			raiseValidateError("Transformation metadata file '" + TF_FN_PARAM_MTD + "' must be a string value (a scalar).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
 		}
 		
-		Expression txspec = getVarParam(TF_FN_PARAM_TXSPEC);
+		Expression txspec = getVarParam(TF_FN_PARAM_SPEC);
 		Expression applyMTD = getVarParam(TF_FN_PARAM_APPLYMTD);
 		if( txspec==null ) {
 			if ( applyMTD == null )
-				raiseValidateError("Named parameter '" + TF_FN_PARAM_TXSPEC + "' missing. Please specify the transformation specification file (in JSON format).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
+				raiseValidateError("Named parameter '" + TF_FN_PARAM_SPEC + "' missing. Please specify the transformation specification (JSON string).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
 		}
 		else if( txspec.getOutput().getDataType() != DataType.SCALAR  || txspec.getOutput().getValueType() != ValueType.STRING ){	
-			raiseValidateError("Transformation specification file '" + TF_FN_PARAM_TXSPEC + "' must be a string value (a scalar).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
+			raiseValidateError("Transformation specification '" + TF_FN_PARAM_SPEC + "' must be a string value (a scalar).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
 		}	
 		
 		if ( applyMTD != null ) {
@@ -271,14 +271,14 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 			}
 			
 			if(txspec != null ) {
-				raiseValidateError("Only one of '" + TF_FN_PARAM_APPLYMTD + "' or '" + TF_FN_PARAM_TXSPEC + "' can be specified in transform().", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
+				raiseValidateError("Only one of '" + TF_FN_PARAM_APPLYMTD + "' or '" + TF_FN_PARAM_SPEC + "' can be specified in transform().", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
 			}
 		}
 		
 		Expression outNames = getVarParam(TF_FN_PARAM_OUTNAMES);
 		if ( outNames != null ) {
 			if( outNames.getOutput().getDataType() != DataType.SCALAR || outNames.getOutput().getValueType() != ValueType.STRING )				
-				raiseValidateError("The parameter specifying column names in the output file '" + TF_FN_PARAM_TXMTD + "' must be a string value (a scalar).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
+				raiseValidateError("The parameter specifying column names in the output file '" + TF_FN_PARAM_MTD + "' must be a string value (a scalar).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
 			if ( applyMTD != null)
 				raiseValidateError("Only one of '" + TF_FN_PARAM_APPLYMTD + "' or '" + TF_FN_PARAM_OUTNAMES + "' can be specified in transform().", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
 		}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ccfe921b/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java
index 56b1c8b..c5c5d46 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java
@@ -168,12 +168,12 @@ public class ParameterizedBuiltinSPInstruction  extends ComputationSPInstruction
 			else if ( opcode.equalsIgnoreCase("transform") ) 
 			{
 				func = ParameterizedBuiltin.getParameterizedBuiltinFnObject(opcode);
-				String specFile = paramsMap.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_TXSPEC);
+				String specJson = paramsMap.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_SPEC);
 				String applyTxPath = paramsMap.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_APPLYMTD);
-				if ( specFile != null && applyTxPath != null)
+				if ( specJson != null && applyTxPath != null)
 					throw new DMLRuntimeException(
 							"Invalid parameters to transform(). Only one of '"
-									+ ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_TXSPEC
+									+ ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_SPEC
 									+ "' or '"
 									+ ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_APPLYMTD
 									+ "' can be specified.");

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ccfe921b/src/main/java/org/apache/sysml/runtime/transform/DataTransform.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/DataTransform.java b/src/main/java/org/apache/sysml/runtime/transform/DataTransform.java
index 957c89a..fb6df5e 100644
--- a/src/main/java/org/apache/sysml/runtime/transform/DataTransform.java
+++ b/src/main/java/org/apache/sysml/runtime/transform/DataTransform.java
@@ -81,7 +81,10 @@ import org.apache.sysml.runtime.util.MapReduceTool;
 import org.apache.sysml.runtime.util.UtilFunctions;
 import org.apache.sysml.utils.JSONHelper;
 
-public class DataTransform {
+public class DataTransform 
+{
+	private static final String ERROR_MSG_ZERO_ROWS = "Number of rows in the transformed output (potentially, after ommitting the ones with missing values) is zero. Cannot proceed.";
+
 	
 	/**
 	 * Method to read the header line from the input data file.
@@ -200,11 +203,8 @@ public class DataTransform {
 	 * @throws IOException
 	 * @throws JSONException 
 	 */
-	private static String processSpecFile(FileSystem fs, String inputPath, String smallestFile, HashMap<String,Integer> colNames, CSVFileFormatProperties prop, String specFileWithNames) throws IllegalArgumentException, IOException, JSONException {
-		// load input spec file with Names
-		BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(new Path(specFileWithNames))));
-		JSONObject inputSpec = JSONHelper.parse(br);
-		br.close();
+	private static String processSpecFile(FileSystem fs, String inputPath, String smallestFile, HashMap<String,Integer> colNames, CSVFileFormatProperties prop, String specWithNames) throws IllegalArgumentException, IOException, JSONException {
+		JSONObject inputSpec = new JSONObject(specWithNames);
 		
 		final String NAME = "name";
 		final String ID = "id";
@@ -277,7 +277,7 @@ public class DataTransform {
 				else if ( stmp.equals(MV_METHOD_CONSTANT))
 					btmp = (byte)3;
 				else
-					throw new IOException("Unknown missing value imputation method (" + stmp + ") in transformation specification file: " + specFileWithNames);
+					throw new IOException("Unknown missing value imputation method (" + stmp + ") in transformation specification: " + specWithNames);
 				mvMethods[i] = btmp;
 				
 				//txMethods.add( btmp );
@@ -342,9 +342,9 @@ public class DataTransform {
 				if(stmp.equals(BIN_METHOD_WIDTH))
 					btmp = (byte)1;
 				else if ( stmp.equals(BIN_METHOD_HEIGHT))
-					throw new IOException("Equi-height binning method is not yet supported, in transformation specification file: " + specFileWithNames);
+					throw new IOException("Equi-height binning method is not yet supported, in transformation specification: " + specWithNames);
 				else
-					throw new IOException("Unknown missing value imputation method (" + stmp + ") in transformation specification file: " + specFileWithNames);
+					throw new IOException("Unknown missing value imputation method (" + stmp + ") in transformation specification: " + specWithNames);
 				binMethods[i] = btmp;
 				
 				numBins[i] = entry.get(TransformationAgent.JSON_NBINS);
@@ -408,7 +408,7 @@ public class DataTransform {
 				else if ( stmp.equals(SCALE_METHOD_Z))
 					btmp = (byte)2;
 				else
-					throw new IOException("Unknown missing value imputation method (" + stmp + ") in transformation specification file: " + specFileWithNames);
+					throw new IOException("Unknown missing value imputation method (" + stmp + ") in transformation specification: " + specWithNames);
 				scaleMethods[i] = btmp;
 			}
 			
@@ -584,7 +584,8 @@ public class DataTransform {
 			outputSpec.put(TX_METHOD.MVRCD.toString(), mvrcd);
 		}
 		
-		 // write out the spec with IDs
+		// write out the spec with IDs
+		// TODO return JSON object in order to prevent write in CP
 		String specFileWithIDs = MRJobConfiguration.constructTempOutputFilename();
 		BufferedWriter out = new BufferedWriter(new OutputStreamWriter(fs.create(new Path(specFileWithIDs),true)));
 		out.write(outputSpec.toString());
@@ -610,80 +611,9 @@ public class DataTransform {
 	}
 
 	private static JSONArray toJSONArray(Object[] list) 
+		throws JSONException 
 	{
-		JSONArray ret = new JSONArray(list.length);
-		for(int i=0; i < list.length; i++)
-			ret.add(list[i]);
-		return ret;
-	}
-
-	private static final String ERROR_MSG_ZERO_ROWS = "Number of rows in the transformed output (potentially, after ommitting the ones with missing values) is zero. Cannot proceed.";
-	
-	/**
-	 * Private class to hold the relevant input parameters to transform operation.
-	 */
-	private static class TransformOperands {
-		String inputPath=null, txMtdPath=null, applyTxPath=null, specFile=null, outNamesFile=null;
-		boolean isApply=false;
-		CSVFileFormatProperties inputCSVProperties = null;
-		
-		TransformOperands(String inst, MatrixObject inputMatrix) {
-			String[] instParts = inst.split(Instruction.OPERAND_DELIM);
-			
-			inputPath = inputMatrix.getFileName();
-			txMtdPath = instParts[3];
-			
-			isApply = Boolean.parseBoolean(instParts[5]);
-			if ( isApply ) {
-				applyTxPath = instParts[4];
-			}
-			else {
-				specFile = instParts[4];
-			}
-			
-			if (instParts.length == 8)
-				outNamesFile = instParts[6];
-			
-			inputCSVProperties = (CSVFileFormatProperties)inputMatrix.getFileFormatProperties();
-		}
-		
-		TransformOperands(ParameterizedBuiltinCPInstruction inst, MatrixObject inputMatrix) {
-			HashMap<String, String> params = inst.getParameterMap();
-			
-			inputPath = inputMatrix.getFileName();
-			txMtdPath = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_TXMTD);
-			
-			if ( params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_TXSPEC) != null ) {
-				isApply = false;
-				specFile = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_TXSPEC);
-				applyTxPath = null;
-			}
-			else if ( params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_APPLYMTD) != null ) {
-				isApply = true;
-				specFile = null;
-				applyTxPath = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_APPLYMTD);
-			}
-			
-			if ( params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_OUTNAMES) != null)
-				outNamesFile = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_OUTNAMES);
-			
-			inputCSVProperties = (CSVFileFormatProperties)inputMatrix.getFileFormatProperties();
-		}
-		
-		TransformOperands(ParameterizedBuiltinSPInstruction inst, MatrixObject inputMatrix) {
-			HashMap<String,String> params = inst.getParams();
-			
-			inputPath = inputMatrix.getFileName();
-			txMtdPath = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_TXMTD);
-			
-			specFile = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_TXSPEC);
-			applyTxPath = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_APPLYMTD);
-			isApply = (applyTxPath != null);
-			outNamesFile =  params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_OUTNAMES); // can be null
-			
-			inputCSVProperties = (CSVFileFormatProperties)inputMatrix.getFileFormatProperties();
-		}
-
+		return new JSONArray(list);
 	}
 	
 	/**
@@ -833,9 +763,7 @@ public class DataTransform {
 		if (!oprnds.isApply) {
 			// build specification file with column IDs insteadof column names
 			String specFileWithIDs = processSpecFile(fs, oprnds.inputPath, 
-														smallestFile, colNamesToIds, 
-														oprnds.inputCSVProperties, 
-														oprnds.specFile);
+							smallestFile, colNamesToIds, oprnds.inputCSVProperties, oprnds.spec);
 			colNamesToIds = null; // enable GC on colNamesToIds
 
 			// Build transformation metadata, including recode maps, bin definitions, etc.
@@ -1062,7 +990,7 @@ public class DataTransform {
 	 * @throws IllegalArgumentException 
 	 */
 	public static JobReturn cpDataTransform(ParameterizedBuiltinCPInstruction inst, MatrixObject[] inputMatrices, MatrixObject[] outputMatrices) throws IOException, DMLRuntimeException, IllegalArgumentException, JSONException {
-		TransformOperands oprnds = new TransformOperands(inst, inputMatrices[0]);
+		TransformOperands oprnds = new TransformOperands(inst.getParameterMap(), inputMatrices[0]);
 		return cpDataTransform(oprnds, inputMatrices, outputMatrices);
 	}
 
@@ -1105,7 +1033,7 @@ public class DataTransform {
 		
 		if (!oprnds.isApply) {
 			// build specification file with column IDs insteadof column names
-			String specFileWithIDs = processSpecFile(fs, oprnds.inputPath, smallestFile, colNamesToIds, oprnds.inputCSVProperties, oprnds.specFile);
+			String specFileWithIDs = processSpecFile(fs, oprnds.inputPath, smallestFile, colNamesToIds, oprnds.inputCSVProperties, oprnds.spec);
 			MapReduceTool.copyFileOnHDFS(specFileWithIDs, oprnds.txMtdPath + "/" + "spec.json");
 	
 			ret = performTransform(job, fs, oprnds.inputPath, colNamesToIds.size(), oprnds.inputCSVProperties, specFileWithIDs, oprnds.txMtdPath, oprnds.isApply, outputMatrices[0], outHeader, isBB, isCSV );
@@ -1439,7 +1367,7 @@ public class DataTransform {
 		SparkExecutionContext sec = (SparkExecutionContext)ec;
 		
 		// Parse transform instruction (the first instruction) to obtain relevant fields
-		TransformOperands oprnds = new TransformOperands(inst, inputMatrices[0]);
+		TransformOperands oprnds = new TransformOperands(inst.getParams(), inputMatrices[0]);
 		
 		JobConf job = new JobConf();
 		FileSystem fs = FileSystem.get(job);
@@ -1467,10 +1395,8 @@ public class DataTransform {
 		
 		if (!oprnds.isApply) {
 			// build specification file with column IDs insteadof column names
-			String specFileWithIDs = processSpecFile(fs, oprnds.inputPath, 
-														smallestFile, colNamesToIds, 
-														oprnds.inputCSVProperties, 
-														oprnds.specFile);
+			String specFileWithIDs = processSpecFile(fs, oprnds.inputPath, smallestFile,
+						colNamesToIds, oprnds.inputCSVProperties, oprnds.spec);
 			colNamesToIds = null; // enable GC on colNamesToIds
 
 			// Build transformation metadata, including recode maps, bin definitions, etc.
@@ -1533,5 +1459,41 @@ public class DataTransform {
 			mcOut.setNonZeros(-1);
 		}
 	}
+	
+
+	/**
+	 * Private class to hold the relevant input parameters to transform operation.
+	 */
+	private static class TransformOperands 
+	{
+		private String inputPath=null;
+		private String txMtdPath=null;
+		private String applyTxPath=null;
+		private String spec=null;
+		private String outNamesFile=null;
+		private boolean isApply=false;
+		private CSVFileFormatProperties inputCSVProperties = null;
+		
+		private TransformOperands(String inst, MatrixObject inputMatrix) {
+			String[] instParts = inst.split(Instruction.OPERAND_DELIM);
+			inputPath = inputMatrix.getFileName();
+			txMtdPath = instParts[3];
+			isApply = Boolean.parseBoolean(instParts[5]);
+			applyTxPath = isApply ? instParts[4] : null;
+			spec = isApply ? null : instParts[4];
+			outNamesFile = (instParts.length==8) ? instParts[6] : null;
+			inputCSVProperties = (CSVFileFormatProperties)inputMatrix.getFileFormatProperties();
+		}
+		
+		private TransformOperands(HashMap<String, String> params, MatrixObject inputMatrix) {
+			inputPath = inputMatrix.getFileName();
+			txMtdPath = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_MTD);
+			spec = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_SPEC);
+			applyTxPath = params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_APPLYMTD);
+			isApply = (applyTxPath != null);
+			outNamesFile =  params.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_OUTNAMES); // can be null
+			inputCSVProperties = (CSVFileFormatProperties)inputMatrix.getFileFormatProperties();
+		}
+	}
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ccfe921b/src/test/scripts/functions/transform/Scaling.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/Scaling.dml b/src/test/scripts/functions/transform/Scaling.dml
index a5bd9b2..1542477 100644
--- a/src/test/scripts/functions/transform/Scaling.dml
+++ b/src/test/scripts/functions/transform/Scaling.dml
@@ -21,10 +21,11 @@
 
 
 raw = read($DATA);
+specJson = read($TFSPEC, data_type="scalar", value_type="string")
 
 A = transform(target = raw, 
               transformPath = $TFMTD, 
-              transformSpec = $TFSPEC);
+              spec = specJson);
 
 write(A, $TFDATA, format=$OFMT);
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ccfe921b/src/test/scripts/functions/transform/Transform.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/Transform.dml b/src/test/scripts/functions/transform/Transform.dml
index a5bd9b2..d36957a 100644
--- a/src/test/scripts/functions/transform/Transform.dml
+++ b/src/test/scripts/functions/transform/Transform.dml
@@ -21,10 +21,11 @@
 
 
 raw = read($DATA);
+specJson = read($TFSPEC, data_type="scalar", value_type="string");
 
 A = transform(target = raw, 
               transformPath = $TFMTD, 
-              transformSpec = $TFSPEC);
+              spec = specJson);
 
 write(A, $TFDATA, format=$OFMT);
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ccfe921b/src/test/scripts/functions/transform/Transform_colnames.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/Transform_colnames.dml b/src/test/scripts/functions/transform/Transform_colnames.dml
index 64a5712..8f40dcc 100644
--- a/src/test/scripts/functions/transform/Transform_colnames.dml
+++ b/src/test/scripts/functions/transform/Transform_colnames.dml
@@ -21,10 +21,11 @@
 
 
 raw = read($DATA);
+specJson = read($TFSPEC, data_type="scalar", value_type="string");
 
 A = transform(target = raw, 
               transformPath = $TFMTD, 
-              transformSpec = $TFSPEC,
+              spec = specJson,
 	      outputNames = $COLNAMES);
 
 write(A, $TFDATA, format=$OFMT);