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

[1/6] systemml git commit: [SYSTEMML-1300] Remove file-based transform from compiler/runtime

Repository: systemml
Updated Branches:
  refs/heads/master 8a2757341 -> 0cd3905f5


http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/ScalingTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/ScalingTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/ScalingTest.java
deleted file mode 100644
index a80e8b3..0000000
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/ScalingTest.java
+++ /dev/null
@@ -1,244 +0,0 @@
-/*
- * 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.transform;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.BufferedWriter;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.wink.json4j.JSONArray;
-import org.apache.wink.json4j.JSONObject;
-import org.junit.Test;
-import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.conf.ConfigurationManager;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.io.ReaderBinaryBlock;
-import org.apache.sysml.runtime.io.ReaderTextCSV;
-import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.transform.TfUtils;
-import org.apache.sysml.test.integration.AutomatedTestBase;
-import org.apache.sysml.test.integration.TestConfiguration;
-import org.apache.sysml.test.utils.TestUtils;
-
-/**
- * 
- * 
- */
-public class ScalingTest extends AutomatedTestBase 
-{
-	
-	private final static String TEST_NAME = "Scaling";
-
-	private final static String TEST_DIR = "functions/transform/";
-	private final static String TEST_CLASS_DIR = TEST_DIR + ScalingTest.class.getSimpleName() + "/";
-	
-	private final static int rows1 = 1500;
-	private final static int cols1 = 16;
-	
-	@Override
-	public void setUp() 
-	{
-		TestUtils.clearAssertionInformation();
-		addTestConfiguration(TEST_NAME,
-			new TestConfiguration(TEST_CLASS_DIR, TEST_NAME, new String[]{"R"}));
-	}
-
-	// ---- Scaling CSV ---- 
-	
-	@Test
-	public void testTransformScalingHybridCSV() throws IOException, DMLRuntimeException, Exception
-	{
-		runScalingTest(rows1, cols1, RUNTIME_PLATFORM.HYBRID, "csv");
-	}
-	
-	@Test
-	public void testTransformScalingSPHybridCSV() throws IOException, DMLRuntimeException, Exception
-	{
-		runScalingTest(rows1, cols1, RUNTIME_PLATFORM.HYBRID_SPARK, "csv");
-	}
-	
-	@Test
-	public void testTransformScalingHadoopCSV() throws IOException, DMLRuntimeException, Exception 
-	{
-		runScalingTest(rows1, cols1, RUNTIME_PLATFORM.HADOOP, "csv");
-	}
-	
-	@Test
-	public void testTransformScalingSparkCSV() throws IOException, DMLRuntimeException, Exception 
-	{
-		runScalingTest(rows1, cols1, RUNTIME_PLATFORM.SPARK, "csv");
-	}
-	
-	// ---- Scaling BinaryBlock ---- 
-	
-	@Test
-	public void testTransformScalingHybridBinary() throws IOException, DMLRuntimeException, Exception 
-	{
-		runScalingTest(rows1, cols1, RUNTIME_PLATFORM.HYBRID, "binary");
-	}
-	
-	@Test
-	public void testTransformScalingSPHybridBinary() throws IOException, DMLRuntimeException, Exception 
-	{
-		runScalingTest(rows1, cols1, RUNTIME_PLATFORM.HYBRID_SPARK, "binary");
-	}
-	
-	@Test
-	public void testTransformScalingHadoopBinary() throws IOException, DMLRuntimeException, Exception 
-	{
-		runScalingTest(rows1, cols1, RUNTIME_PLATFORM.HADOOP, "binary");
-	}
-	
-	@Test
-	public void testTransformScalingSparkBinary() throws IOException, DMLRuntimeException, Exception 
-	{
-		runScalingTest(rows1, cols1, RUNTIME_PLATFORM.SPARK, "binary");
-	}
-	
-	// ----------------------------
-	
-	private void generateSpecFile(int cols, String specFile) throws IOException , Exception
-	{
-		final String NAME = "name";
-		final String METHOD = "method";
-		final String SCALE_METHOD_Z = "z-score";
-		final String SCALE_METHOD_M = "mean-subtraction";
-		
-		JSONObject outputSpec = new JSONObject();
-		JSONArray scaleSpec = new JSONArray();
-
-		for(int colID=1; colID <= cols; colID++)
-		{
-			JSONObject obj = new JSONObject();
-			obj.put(NAME, "V"+colID);
-			if(colID <= cols/2)
-				obj.put(METHOD, SCALE_METHOD_M);
-			else
-				obj.put(METHOD, SCALE_METHOD_Z);
-			scaleSpec.add(obj);
-		}
-		outputSpec.put(TfUtils.TXMETHOD_SCALE, scaleSpec);
-		
-		FileSystem fs = IOUtilFunctions.getFileSystem(specFile);
-		try( BufferedWriter out = new BufferedWriter(new OutputStreamWriter(fs.create(new Path(specFile),true))) ) {
-			out.write(outputSpec.toString());
-		}
-
-	}
-	
-	private void generateFrameMTD(String datafile) throws IllegalArgumentException, IOException , Exception
-	{
-		JSONObject mtd = new JSONObject();
-		
-		mtd.put("data_type", "frame");
-		mtd.put("format", "csv");
-		mtd.put("header", false);
-		
-		FileSystem fs = IOUtilFunctions.getFileSystem(datafile);
-		try( BufferedWriter out = new BufferedWriter(new OutputStreamWriter(fs.create(new Path(datafile+".mtd"),true))) ) {
-			out.write(mtd.toString());
-		}
-	}
-	
-	/**
-	 * 
-	 * @param sparseM1
-	 * @param sparseM2
-	 * @param instType
-	 * @throws IOException 
-	 * @throws DMLRuntimeException 
-	 */
-	private void runScalingTest( int rows, int cols, RUNTIME_PLATFORM rt, String ofmt) throws IOException, DMLRuntimeException, Exception
-	{
-		RUNTIME_PLATFORM platformOld = rtplatform;
-		rtplatform = rt;
-	
-		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
-		if( rtplatform == RUNTIME_PLATFORM.SPARK || rtplatform == RUNTIME_PLATFORM.HYBRID_SPARK)
-			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
-
-		try
-		{
-			TestConfiguration config = getTestConfiguration(TEST_NAME);
-			loadTestConfiguration(config);
-			
-			String HOME = SCRIPT_DIR + TEST_DIR;
-			String specFile = input("spec.json");
-			String inputFile = input("X");
-			String outputFile = output(config.getOutputFiles()[0]);
-			String outputFileR = expected(config.getOutputFiles()[0]);
-			
-			generateSpecFile(cols, specFile);
-			
-			// This is for running the junit test the new way, i.e., construct the arguments directly
-			fullDMLScriptName = HOME + TEST_NAME + ".dml";
-			programArgs = new String[]{"-nvargs", 
-					"DATA=" + inputFile,
-					"TFSPEC=" + specFile,
-					"TFMTD=" + output("tfmtd"),
-					"TFDATA=" + outputFile,
-					"OFMT=" + ofmt };
-			
-			fullRScriptName = HOME + TEST_NAME + ".R";
-			rCmd = "Rscript" + " " + fullRScriptName + " " + inputFile + " " + outputFileR;
-	
-			//generate actual dataset 
-			double[][] X = getRandomMatrix(rows, cols, -50, 50, 1.0, 7); 
-			TestUtils.writeCSVTestMatrix(inputFile, X);
-			generateFrameMTD(inputFile);
-			
-			runTest(true, false, null, -1); 
-			runRScript(true); 
-		
-			ReaderTextCSV expReader=  new ReaderTextCSV(new CSVFileFormatProperties(false, ",", true, 0, null));
-			MatrixBlock exp = expReader.readMatrixFromHDFS(outputFileR, -1, -1, -1, -1, -1);
-			MatrixBlock out = null;
-			
-			if ( ofmt.equals("csv") ) 
-			{
-				ReaderTextCSV outReader=  new ReaderTextCSV(new CSVFileFormatProperties(false, ",", true, 0, null));
-				out = outReader.readMatrixFromHDFS(outputFile, -1, -1, -1, -1, -1);
-			}
-			else
-			{
-				ReaderBinaryBlock bbReader = new ReaderBinaryBlock(false);
-				out = bbReader.readMatrixFromHDFS(
-						outputFile, exp.getNumRows(), exp.getNumColumns(), 
-						ConfigurationManager.getBlocksize(), 
-						ConfigurationManager.getBlocksize(),
-						-1);
-			}
-			
-			assertTrue("Incorrect output from data transform.", TransformTest.equals(out,exp,  1e-10));
-		}
-		finally
-		{
-			rtplatform = platformOld;
-			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-		}
-	}	
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformAndApplyTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformAndApplyTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformAndApplyTest.java
deleted file mode 100644
index 90c1e2f..0000000
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformAndApplyTest.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * 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.transform;
-
-import java.io.IOException;
-import java.util.HashMap;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.io.MatrixWriter;
-import org.apache.sysml.runtime.io.MatrixWriterFactory;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
-import org.apache.sysml.runtime.matrix.data.OutputInfo;
-import org.apache.sysml.runtime.util.DataConverter;
-import org.apache.sysml.runtime.util.MapReduceTool;
-import org.apache.sysml.test.integration.AutomatedTestBase;
-import org.apache.sysml.test.integration.TestConfiguration;
-import org.apache.sysml.test.utils.TestUtils;
-
-/**
- * 
- * 
- */
-public class TransformAndApplyTest extends AutomatedTestBase 
-{
-	private static final String TEST_NAME1 = "TransformAndApply";
-	private static final String TEST_DIR = "functions/transform/";
-	private static final String TEST_CLASS_DIR = TEST_DIR + TransformAndApplyTest.class.getSimpleName() + "/";
-	
-	private static final String SPEC_X = "TransformAndApplySpecX.json";
-	private static final String SPEC_Y = "TransformAndApplySpecY.json";
-	
-	private static final int rows = 1234;
-	
-	@Override
-	public void setUp() {
-		TestUtils.clearAssertionInformation();
-		addTestConfiguration(TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1,new String[]{"R1","R2"}));
-	}
-	
-	@Test
-	public void runTestCP() throws DMLRuntimeException, IOException {
-		runTransformAndApplyTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv");
-	}
-	
-	@Test
-	public void runTestHadoop() throws DMLRuntimeException, IOException {
-		runTransformAndApplyTest(RUNTIME_PLATFORM.HADOOP, "csv");
-	}
-
-	@Test
-	public void runTestSpark() throws DMLRuntimeException, IOException {
-		runTransformAndApplyTest(RUNTIME_PLATFORM.SPARK, "csv");
-	}
-	
-	/**
-	 * 
-	 * @param sparseM1
-	 * @param sparseM2
-	 * @param instType
-	 * @throws IOException 
-	 * @throws DMLRuntimeException 
-	 */
-	private void runTransformAndApplyTest( RUNTIME_PLATFORM rt, String ofmt) throws IOException, DMLRuntimeException
-	{
-		RUNTIME_PLATFORM platformOld = rtplatform;
-		rtplatform = rt;
-	
-		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
-		if( rtplatform == RUNTIME_PLATFORM.SPARK  || rtplatform == RUNTIME_PLATFORM.HYBRID_SPARK)
-			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
-
-		try
-		{
-			getAndLoadTestConfiguration(TEST_NAME1);
-			
-			//generate input data
-			double[][] X = DataConverter.convertToDoubleMatrix(
-					MatrixBlock.seqOperations(0.5, rows/2, 0.5).appendOperations(
-					MatrixBlock.seqOperations(0.5, rows/2, 0.5), new MatrixBlock()));
-			double[][] Y = DataConverter.convertToDoubleMatrix(
-					MatrixBlock.seqOperations(rows/2, 0.5, -0.5));
-			
-			//write inputs
-			MatrixBlock mbX = DataConverter.convertToMatrixBlock(X);
-			MatrixBlock mbY = DataConverter.convertToMatrixBlock(Y);
-			MatrixWriter writer = MatrixWriterFactory.createMatrixWriter(OutputInfo.CSVOutputInfo);
-			writer.writeMatrixToHDFS(mbX, input("X"), rows, 2, -1, -1, -1);
-			writer.writeMatrixToHDFS(mbY, input("Y"), rows, 1, -1, -1, -1);
-			
-			//read specs transform X and Y
-			String specX = MapReduceTool.readStringFromHDFSFile(SCRIPT_DIR+TEST_DIR+SPEC_X);
-			String specY = MapReduceTool.readStringFromHDFSFile(SCRIPT_DIR+TEST_DIR+SPEC_Y);
-			
-			
-			fullDMLScriptName = SCRIPT_DIR+TEST_DIR + TEST_NAME1 + ".dml";
-			programArgs = new String[]{"-args", input("X"), input("Y"), specX, specY, 
-					output("M1"), output("M2"), output("R1"), output("R2") };
-			
-			//run test
-			runTest(true, false, null, -1); 
-			
-			//compare matrices (values recoded to identical codes)
-			HashMap<CellIndex, Double> dml1 = readDMLMatrixFromHDFS("R1");
-			HashMap<CellIndex, Double> dml2  = readDMLMatrixFromHDFS("R2");			
-			double[][] R1 = TestUtils.convertHashMapToDoubleArray(dml1);
-			double[][] R2 = TestUtils.convertHashMapToDoubleArray(dml2);
-			for( int i=0; i<rows; i++ ) {
-				Assert.assertEquals("Output values don't match: "+R1[i][0]+" vs "+R2[i][0], 
-						new Double(R1[i][0]), new Double(R2[rows-i-1][0]));
-			}
-		}
-		catch(Exception ex) {
-			throw new IOException(ex);
-		}
-		finally
-		{
-			rtplatform = platformOld;
-			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-		}
-	}	
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeDecodeTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeDecodeTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeDecodeTest.java
index 9ee3d5f..059bdb8 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeDecodeTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeDecodeTest.java
@@ -23,7 +23,6 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.io.FrameReader;
 import org.apache.sysml.runtime.io.FrameReaderFactory;
 import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
@@ -75,7 +74,6 @@ public class TransformCSVFrameEncodeDecodeTest extends AutomatedTestBase
 	{
 		//set runtime platform
 		RUNTIME_PLATFORM rtold = rtplatform;
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
 		rtplatform = rt;
 
 		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
@@ -93,8 +91,7 @@ public class TransformCSVFrameEncodeDecodeTest extends AutomatedTestBase
 			fullDMLScriptName = HOME + TEST_NAME1 + ".dml";
 			programArgs = new String[]{"-explain","-args", 
 				HOME + "input/" + DATASET, output("R") };
-	
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
+			
 			runTest(true, false, null, -1); 
 			
 			//read input/output and compare
@@ -118,7 +115,6 @@ public class TransformCSVFrameEncodeDecodeTest extends AutomatedTestBase
 		finally {
 			rtplatform = rtold;
 			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
 		}
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeReadTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeReadTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeReadTest.java
index b35f2ac..535d88d 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeReadTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformCSVFrameEncodeReadTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.functions.transform;
 import org.junit.Test;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.io.FrameReader;
 import org.apache.sysml.runtime.io.FrameReaderTextCSV;
 import org.apache.sysml.runtime.io.FrameReaderTextCSVParallel;
@@ -120,7 +119,6 @@ public class TransformCSVFrameEncodeReadTest extends AutomatedTestBase
 	{
 		//set runtime platform
 		RUNTIME_PLATFORM rtold = rtplatform;
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
 		rtplatform = rt;
 
 		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
@@ -139,8 +137,7 @@ public class TransformCSVFrameEncodeReadTest extends AutomatedTestBase
 			fullDMLScriptName = HOME + TEST_NAME1 + ".dml";
 			programArgs = new String[]{"-explain", "-stats","-args", 
 				HOME + "input/" + DATASET, String.valueOf(nrows), output("R") };
-	
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
+			
 			runTest(true, false, null, -1); 
 			
 			//read input/output and compare
@@ -156,7 +153,6 @@ public class TransformCSVFrameEncodeReadTest extends AutomatedTestBase
 		finally {
 			rtplatform = rtold;
 			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
 		}
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameApplyTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameApplyTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameApplyTest.java
deleted file mode 100644
index 6f1c74c..0000000
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameApplyTest.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- * 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.transform;
-
-import org.junit.Test;
-import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.hops.OptimizerUtils;
-import org.apache.sysml.runtime.io.MatrixReader;
-import org.apache.sysml.runtime.io.MatrixReaderFactory;
-import org.apache.sysml.runtime.matrix.data.InputInfo;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.util.DataConverter;
-import org.apache.sysml.test.integration.AutomatedTestBase;
-import org.apache.sysml.test.integration.TestConfiguration;
-import org.apache.sysml.test.utils.TestUtils;
-
-public class TransformFrameApplyTest extends AutomatedTestBase 
-{
-	private final static String TEST_NAME1 = "Transform";
-	private final static String TEST_NAME2 = "ApplyFrame";
-	private final static String TEST_DIR = "functions/transform/";
-	private final static String TEST_CLASS_DIR = TEST_DIR + TransformFrameApplyTest.class.getSimpleName() + "/";
-	
-	//dataset and transform tasks without missing values
-	private final static String DATASET1 	= "homes3/homes.csv";
-	private final static String SPEC1 		= "homes3/homes.tfspec_recode.json"; 
-	private final static String SPEC2 		= "homes3/homes.tfspec_dummy.json";
-	private final static String SPEC3 		= "homes3/homes.tfspec_bin.json"; //incl recode
-	
-	//dataset and transform tasks with missing values
-	private final static String DATASET2 	= "homes/homes.csv";
-	private final static String SPEC4 		= "homes3/homes.tfspec_impute.json";
-	private final static String SPEC5 		= "homes3/homes.tfspec_omit.json";
-	
-	public enum TransformType {
-		RECODE,
-		DUMMY,
-		BIN,
-		IMPUTE,
-		OMIT,
-	}
-	
-	@Override
-	public void setUp()  {
-		TestUtils.clearAssertionInformation();
-		addTestConfiguration(TEST_NAME1, 
-			new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1, new String[] { "y" }) );
-	}
-	
-	@Test
-	public void testHomesRecodeSingleNodeCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", TransformType.RECODE);
-	}
-	
-	@Test
-	public void testHomesRecodeSparkCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", TransformType.RECODE);
-	}
-	
-	@Test
-	public void testHomesDummycodeSingleNodeCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", TransformType.DUMMY);
-	}
-	
-	@Test
-	public void testHomesDummycodeSparkCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", TransformType.DUMMY);
-	}
-	
-	@Test
-	public void testHomesBinningSingleNodeCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", TransformType.BIN);
-	}
-	
-	@Test
-	public void testHomesBinningSparkCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", TransformType.BIN);
-	}
-	
-	@Test
-	public void testHomesOmitSingleNodeCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", TransformType.OMIT);
-	}
-	
-	@Test
-	public void testHomesOmitSparkCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", TransformType.OMIT);
-	}
-	
-	@Test
-	public void testHomesImputeSingleNodeCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", TransformType.IMPUTE);
-	}
-	
-	@Test
-	public void testHomesImputeSparkCSV() {
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", TransformType.IMPUTE);
-	}
-
-	/**
-	 * 
-	 * @param rt
-	 * @param ofmt
-	 * @param dataset
-	 */
-	private void runTransformTest( RUNTIME_PLATFORM rt, String ofmt, TransformType type )
-	{
-		//set runtime platform
-		RUNTIME_PLATFORM rtold = rtplatform;
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
-		rtplatform = rt;
-
-		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
-		if( rtplatform == RUNTIME_PLATFORM.SPARK || rtplatform == RUNTIME_PLATFORM.HYBRID_SPARK)
-			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
-
-		//set transform specification
-		String SPEC = null; String DATASET = null;
-		switch( type ) {
-			case RECODE: SPEC = SPEC1; DATASET = DATASET1; break;
-			case DUMMY:  SPEC = SPEC2; DATASET = DATASET1; break;
-			case BIN:    SPEC = SPEC3; DATASET = DATASET1; break;
-			case IMPUTE: SPEC = SPEC4; DATASET = DATASET2; break;
-			case OMIT:   SPEC = SPEC5; DATASET = DATASET2; break;
-		}
-
-		if( !ofmt.equals("csv") )
-			throw new RuntimeException("Unsupported test output format");
-		
-		try
-		{
-			getAndLoadTestConfiguration(TEST_NAME1);
-	
-			/* This is for running the junit test the new way, i.e., construct the arguments directly */
-			String HOME = SCRIPT_DIR + TEST_DIR;
-			fullDMLScriptName = HOME + TEST_NAME1 + ".dml";
-			programArgs = new String[]{"-nvargs", 
-				"DATA=" + HOME + "input/" + DATASET,
-				"TFSPEC=" + HOME + "input/" + SPEC,
-				"TFMTD=" + output("tfmtd"),
-				"TFDATA=" + output("tfout"),
-				"OFMT=" + ofmt };
-	
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
-			runTest(true, false, null, -1); 
-			
-			fullDMLScriptName = HOME + TEST_NAME2 + ".dml";
-			programArgs = new String[]{"-explain","-nvargs", 
-				"DATA=" + HOME + "input/" + DATASET,
-				"TFSPEC=" + HOME + "input/" + SPEC,
-				"APPLYMTD=" + output("tfmtd"),  // generated above
-				"TFDATA=" + output("test_tfout"),
-				"OFMT=" + ofmt };
-	
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
-			runTest(true, false, null, -1); 
-			
-			//read both outputs and compare
-			MatrixReader reader1 = MatrixReaderFactory.createMatrixReader(InputInfo.CSVInputInfo);
-			MatrixBlock mb1 = reader1.readMatrixFromHDFS(output("tfout"), -1, -1, -1, -1, -1);
-			MatrixReader reader2 = MatrixReaderFactory.createMatrixReader(InputInfo.CSVInputInfo);
-			MatrixBlock mb2 = reader2.readMatrixFromHDFS(output("test_tfout"), -1, -1, -1, -1, -1);
-			double[][] R1 = DataConverter.convertToDoubleMatrix(mb1);
-			double[][] R2 = DataConverter.convertToDoubleMatrix(mb2);
-			TestUtils.compareMatrices(R1, R2, R1.length, R1[0].length, 0);			
-		}
-		catch(Exception ex) {
-			throw new RuntimeException(ex);
-		}
-		finally {
-			rtplatform = rtold;
-			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeApplyTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeApplyTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeApplyTest.java
index 2d17c17..405661b 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeApplyTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeApplyTest.java
@@ -23,7 +23,6 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.io.MatrixReaderFactory;
 import org.apache.sysml.runtime.matrix.data.InputInfo;
 import org.apache.sysml.runtime.util.DataConverter;
@@ -228,7 +227,6 @@ public class TransformFrameEncodeApplyTest extends AutomatedTestBase
 	{
 		//set runtime platform
 		RUNTIME_PLATFORM rtold = rtplatform;
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
 		rtplatform = rt;
 
 		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
@@ -261,7 +259,6 @@ public class TransformFrameEncodeApplyTest extends AutomatedTestBase
 				"TFDATA2=" + output("tfout2"),
 				"OFMT=" + ofmt };
 	
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
 			runTest(true, false, null, -1); 
 			
 			//read input/output and compare
@@ -284,7 +281,6 @@ public class TransformFrameEncodeApplyTest extends AutomatedTestBase
 		finally {
 			rtplatform = rtold;
 			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
 		}
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java
index a879356..3af3149 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTest.java
@@ -21,7 +21,6 @@ package org.apache.sysml.test.integration.functions.transform;
 
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.io.FrameReader;
 import org.apache.sysml.runtime.io.FrameReaderFactory;
 import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
@@ -133,7 +132,6 @@ public class TransformFrameEncodeDecodeTest extends AutomatedTestBase
 	{
 		//set runtime platform
 		RUNTIME_PLATFORM rtold = rtplatform;
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
 		rtplatform = rt;
 
 		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
@@ -170,7 +168,6 @@ public class TransformFrameEncodeDecodeTest extends AutomatedTestBase
 			// This is just a feature/bug and is reported in CLI-262,
 			// though even a fix is unlikely to be backported to 1.2
 
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
 			runTest(true, false, null, -1); 
 			
 			//read input/output and compare
@@ -194,7 +191,6 @@ public class TransformFrameEncodeDecodeTest extends AutomatedTestBase
 		finally {
 			rtplatform = rtold;
 			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
 		}
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTokenTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTokenTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTokenTest.java
index fa89e28..f9dbe06 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTokenTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformFrameEncodeDecodeTokenTest.java
@@ -23,7 +23,6 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.io.FrameReader;
 import org.apache.sysml.runtime.io.FrameReaderFactory;
 import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
@@ -81,7 +80,6 @@ public class TransformFrameEncodeDecodeTokenTest extends AutomatedTestBase
 	{
 		//set runtime platform
 		RUNTIME_PLATFORM rtold = rtplatform;
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
 		rtplatform = rt;
 
 		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
@@ -103,7 +101,6 @@ public class TransformFrameEncodeDecodeTokenTest extends AutomatedTestBase
 				"TFDATA=" + output("tfout"), "SEP= ",
 				"OFMT=" + ofmt, "OSEP= " };
 	
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
 			runTest(true, false, null, -1); 
 			
 			//read input/output and compare
@@ -128,7 +125,6 @@ public class TransformFrameEncodeDecodeTokenTest extends AutomatedTestBase
 		finally {
 			rtplatform = rtold;
 			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
 		}
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformReadMetaTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformReadMetaTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformReadMetaTest.java
deleted file mode 100644
index 4e0240b..0000000
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformReadMetaTest.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/*
- * 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.transform;
-
-import java.io.IOException;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.parser.Expression.ValueType;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.io.FrameReader;
-import org.apache.sysml.runtime.io.FrameReaderFactory;
-import org.apache.sysml.runtime.io.MatrixWriter;
-import org.apache.sysml.runtime.io.MatrixWriterFactory;
-import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
-import org.apache.sysml.runtime.matrix.data.FrameBlock;
-import org.apache.sysml.runtime.matrix.data.InputInfo;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.matrix.data.OutputInfo;
-import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
-import org.apache.sysml.runtime.util.DataConverter;
-import org.apache.sysml.runtime.util.MapReduceTool;
-import org.apache.sysml.runtime.util.UtilFunctions;
-import org.apache.sysml.test.integration.AutomatedTestBase;
-import org.apache.sysml.test.integration.TestConfiguration;
-import org.apache.sysml.test.utils.TestUtils;
-
-/**
- * 
- * 
- */
-public class TransformReadMetaTest extends AutomatedTestBase 
-{
-	private static final String TEST_NAME1 = "TransformReadMeta";
-	private static final String TEST_NAME2 = "TransformReadMeta2";
-	private static final String TEST_DIR = "functions/transform/";
-	private static final String TEST_CLASS_DIR = TEST_DIR + TransformReadMetaTest.class.getSimpleName() + "/";
-	private static final String SPEC_X = "TransformReadMetaSpecX.json";
-	
-	private static final int rows = 1432;
-	
-	@Override
-	public void setUp() {
-		TestUtils.clearAssertionInformation();
-		addTestConfiguration(TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1,new String[]{"M1, M"}));
-		addTestConfiguration(TEST_NAME2, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME2,new String[]{"M1, M"}));
-	}
-	
-	@Test
-	public void runTestCsvCP() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", ",");
-	}
-	
-	@Test
-	public void runTestCsvHadoop() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.HADOOP, "csv", ",");
-	}
-
-	@Test
-	public void runTestCsvSpark() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SPARK, "csv", ",");
-	}
-	
-	@Test
-	public void runTestCsvTabCP() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", "\t");
-	}
-	
-	@Test
-	public void runTestCsvTabHadoop() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.HADOOP, "csv", "\t");
-	}
-
-	@Test
-	public void runTestCsvTabSpark() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SPARK, "csv", "\t");
-	}
-	
-	@Test
-	public void runTestCsvColonCP() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", ":");
-	}
-	
-	@Test
-	public void runTestCsvColonHadoop() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.HADOOP, "csv", ":");
-	}
-
-	@Test
-	public void runTestCsvColonSpark() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SPARK, "csv", ":");
-	}
-	
-	
-	@Test
-	public void runTestTextCP() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SINGLE_NODE, "text", ",");
-	}
-	
-	@Test
-	public void runTestTextHadoop() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.HADOOP, "text", ",");
-	}
-
-	@Test
-	public void runTestTextSpark() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SPARK, "text", ",");
-	}
-
-	@Test
-	public void runTestBinaryCP() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SINGLE_NODE, "binary", ",");
-	}
-	
-	@Test
-	public void runTestBinaryHadoop() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.HADOOP, "binary", ",");
-	}
-
-	@Test
-	public void runTestBinarySpark() throws DMLRuntimeException, IOException {
-		runTransformReadMetaTest(RUNTIME_PLATFORM.SPARK, "binary", ",");
-	}
-
-	
-	/**
-	 * 
-	 * @param sparseM1
-	 * @param sparseM2
-	 * @param instType
-	 * @throws IOException 
-	 * @throws DMLRuntimeException 
-	 */
-	private void runTransformReadMetaTest( RUNTIME_PLATFORM rt, String ofmt, String delim) throws IOException, DMLRuntimeException
-	{
-		RUNTIME_PLATFORM platformOld = rtplatform;
-		rtplatform = rt;
-	
-		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
-		if( rtplatform == RUNTIME_PLATFORM.SPARK  || rtplatform == RUNTIME_PLATFORM.HYBRID_SPARK)
-			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
-
-		try
-		{
-			String testname = delim.equals(",") ? TEST_NAME1 : TEST_NAME2;
-			
-			getAndLoadTestConfiguration(testname);
-			
-			//generate input data
-			double[][] X = DataConverter.convertToDoubleMatrix(
-					MatrixBlock.seqOperations(0.5, rows/2, 0.5).appendOperations(
-					MatrixBlock.seqOperations(0.5, rows/2, 0.5), new MatrixBlock()));
-			MatrixBlock mbX = DataConverter.convertToMatrixBlock(X);
-			CSVFileFormatProperties fprops = new CSVFileFormatProperties(false, delim, false);
-			MatrixWriter writer = MatrixWriterFactory.createMatrixWriter(OutputInfo.CSVOutputInfo, 1, fprops);
-			writer.writeMatrixToHDFS(mbX, input("X"), rows, 2, -1, -1, -1);
-			
-			//read specs transform X and Y
-			String specX = MapReduceTool.readStringFromHDFSFile(SCRIPT_DIR+TEST_DIR+SPEC_X);
-			
-			fullDMLScriptName = SCRIPT_DIR+TEST_DIR + testname + ".dml";
-			programArgs = new String[]{"-args", input("X"), specX, output("M1"), output("M"), ofmt, delim};
-			
-			//run test
-			runTest(true, false, null, -1); 
-			
-			//compare meta data frames
-			InputInfo iinfo = InputInfo.stringExternalToInputInfo(ofmt);
-			FrameReader reader = FrameReaderFactory.createFrameReader(iinfo); 
-			FrameBlock mExpected = TfMetaUtils.readTransformMetaDataFromFile(specX, output("M1"), delim);
-			FrameBlock mRet = reader.readFrameFromHDFS(output("M"), rows, 2);
-			for( int i=0; i<rows; i++ )
-				for( int j=0; j<2; j++ ) {
-					Assert.assertTrue("Wrong result: "+mRet.get(i, j)+".", 
-						UtilFunctions.compareTo(ValueType.STRING, mExpected.get(i, j), mRet.get(i, j))==0);
-				}
-		}
-		catch(Exception ex) {
-			throw new IOException(ex);
-		}
-		finally {
-			rtplatform = platformOld;
-			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-		}
-	}	
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformTest.java
deleted file mode 100644
index 3d799f0..0000000
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/TransformTest.java
+++ /dev/null
@@ -1,709 +0,0 @@
-/*
- * 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.transform;
-
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Test;
-
-import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.conf.ConfigurationManager;
-import org.apache.sysml.runtime.io.ReaderBinaryBlock;
-import org.apache.sysml.runtime.io.ReaderTextCSV;
-import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.test.integration.AutomatedTestBase;
-import org.apache.sysml.test.integration.TestConfiguration;
-import org.apache.sysml.test.utils.TestUtils;
-
-public class TransformTest extends AutomatedTestBase 
-{
-	
-	private final static String TEST_NAME1 = "Transform";
-	private final static String TEST_NAME2 = "Apply";
-	private final static String TEST_DIR = "functions/transform/";
-	private final static String TEST_CLASS_DIR = TEST_DIR + TransformTest.class.getSimpleName() + "/";
-	
-	private final static String HOMES_DATASET 	= "homes/homes.csv";
-	private final static String HOMES_SPEC 		= "homes/homes.tfspec.json";
-	private final static String HOMES_IDSPEC 	= "homes/homes.tfidspec.json";
-	private final static String HOMES_TFDATA 	= "homes/homes.transformed.csv";
-	
-	private final static String HOMES_OMIT_DATASET 	= "homes/homes.csv";
-	private final static String HOMES_OMIT_SPEC 	= "homes/homesOmit.tfspec.json";
-	private final static String HOMES_OMIT_IDSPEC 	= "homes/homesOmit.tfidspec.json";
-	private final static String HOMES_OMIT_TFDATA 	= "homes/homesOmit.transformed.csv";
-	
-	// Homes data set in two parts
-	private final static String HOMES2_DATASET 	= "homes2/homes.csv";
-	private final static String HOMES2_SPEC 	= "homes2/homes.tfspec.json";
-	private final static String HOMES2_IDSPEC 	= "homes2/homes.tfidspec.json";
-	private final static String HOMES2_TFDATA 	= "homes/homes.transformed.csv"; // same as HOMES_TFDATA
-	
-	private final static String IRIS_DATASET 	= "iris/iris.csv";
-	private final static String IRIS_SPEC 		= "iris/iris.tfspec.json";
-	private final static String IRIS_IDSPEC 	= "iris/iris.tfidspec.json";
-	private final static String IRIS_TFDATA 	= "iris/iris.transformed.csv";
-	
-	@Override
-	public void setUp() 
-	{
-		addTestConfiguration(TEST_NAME1, 
-			new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1, new String[] { "y" }) );
-	}
-	
-	// ---- Iris CSV ----
-	
-	@Test
-	public void testIrisHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "csv", "iris", false);
-	}
-	
-	@Test
-	public void testIrisSingleNodeCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", "iris", false);
-	}
-	
-	@Test
-	public void testIrisSPHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "csv", "iris", false);
-	}
-	
-	@Test
-	public void testIrisHadoopCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "csv", "iris", false);
-	}
-
-	@Test
-	public void testIrisSparkCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", "iris", false);
-	}
-
-	// ---- Iris BinaryBlock ----
-	
-	@Test
-	public void testIrisHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "binary", "iris", false);
-	}
-	
-	@Test
-	public void testIrisSingleNodeBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "binary", "iris", false);
-	}
-	
-	@Test
-	public void testIrisSPHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "binary", "iris", false);
-	}
-	
-	@Test
-	public void testIrisHadoopBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "binary", "iris", false);
-	}
-	
-	@Test
-	public void testIrisSparkBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "binary", "iris", false);
-	}
-	
-	// ---- Homes CSV ----
-	
-	@Test
-	public void testHomesHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "csv", "homes", false);
-	}
-	
-	@Test
-	public void testHomesSingleNodeCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", "homes", false);
-	}
-	
-	@Test
-	public void testHomesHadoopCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "csv", "homes", false);
-	}
-
-	@Test
-	public void testHomesSPHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "csv", "homes", false);
-	}
-
-	@Test
-	public void testHomesSparkCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", "homes", false);
-	}
-
-	// ---- Homes BinaryBlock ----
-	
-	@Test
-	public void testHomesHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "binary", "homes", false);
-	}
-	
-	@Test
-	public void testHomesSingleNodeBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "binary", "homes", false);
-	}
-	
-	@Test
-	public void testHomesHadoopBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "binary", "homes", false);
-	}
-	
-	@Test
-	public void testHomesSPHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "binary", "homes", false);
-	}
-	
-	@Test
-	public void testHomesSparkBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "binary", "homes", false);
-	}
-	
-	// ---- OmitHomes CSV ----
-	
-	@Test
-	public void testOmitHomesHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "csv", "homesomit", false);
-	}
-	
-	@Test
-	public void testOmitHomesSingleNodeCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", "homesomit", false);
-	}
-	
-	@Test
-	public void testOmitHomesHadoopCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "csv", "homesomit", false);
-	}
-
-	@Test
-	public void testOmitHomesSPHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "csv", "homesomit", false);
-	}
-
-	@Test
-	public void testOmitHomesSparkCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", "homesomit", false);
-	}
-
-	// ---- OmitHomes BinaryBlock ----
-	
-	@Test
-	public void testOmitHomesHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "binary", "homesomit", false);
-	}
-	
-	@Test
-	public void testOmitHomesSingleNodeBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "binary", "homesomit", false);
-	}
-	
-	@Test
-	public void testOmitHomesHadoopBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "binary", "homesomit", false);
-	}
-	
-	@Test
-	public void testOmitHomesSPHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "binary", "homesomit", false);
-	}
-	
-	@Test
-	public void testOmitHomesSparkBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "binary", "homesomit", false);
-	}
-	
-	// ---- Homes2 CSV ----
-	
-	@Test
-	public void testHomes2HybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "csv", "homes2", false);
-	}
-	
-	@Test
-	public void testHomes2SingleNodeCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", "homes2", false);
-	}
-	
-	@Test
-	public void testHomes2HadoopCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "csv", "homes2", false);
-	}
-
-	@Test
-	public void testHomes2SPHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "csv", "homes2", false);
-	}
-
-	@Test
-	public void testHomes2SparkCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", "homes2", false);
-	}
-
-	// ---- Homes2 BinaryBlock ----
-	
-	@Test
-	public void testHomes2HybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "binary", "homes2", false);
-	}
-	
-	@Test
-	public void testHomes2SingleNodeBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "binary", "homes2", false);
-	}
-	
-	@Test
-	public void testHomes2HadoopBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "binary", "homes2", false);
-	}
-	
-	@Test
-	public void testHomes2SPHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "binary", "homes2", false);
-	}
-	
-	@Test
-	public void testHomes2SparkBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "binary", "homes2", false);
-	}
-	
-	// ---- Iris ID CSV ----
-	
-	@Test
-	public void testIrisHybridIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "csv", "iris", true);
-	}
-	
-	@Test
-	public void testIrisSingleNodeIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", "iris", true);
-	}
-	
-	@Test
-	public void testIrisSPHybridIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "csv", "iris", true);
-	}
-	
-	@Test
-	public void testIrisHadoopIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "csv", "iris", true);
-	}
-
-	@Test
-	public void testIrisSparkIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", "iris", true);
-	}
-
-	// ---- Iris ID BinaryBlock ----
-	
-	@Test
-	public void testIrisHybridIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "binary", "iris", true);
-	}
-	
-	@Test
-	public void testIrisSingleNodeIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "binary", "iris", true);
-	}
-	
-	@Test
-	public void testIrisSPHybridIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "binary", "iris", true);
-	}
-	
-	@Test
-	public void testIrisHadoopIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "binary", "iris", true);
-	}
-	
-	@Test
-	public void testIrisSparkIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "binary", "iris", true);
-	}
-	
-	// ---- Homes ID CSV ----
-	
-	@Test
-	public void testHomesHybridIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "csv", "homes", true);
-	}
-	
-	@Test
-	public void testHomesSingleNodeIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", "homes", true);
-	}
-	
-	@Test
-	public void testHomesSPHybridIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "csv", "homes", true);
-	}
-	
-	@Test
-	public void testHomesHadoopIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "csv", "homes", true);
-	}
-
-	@Test
-	public void testHomesSparkIDCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", "homes", true);
-	}
-
-	// ---- Homes ID BinaryBlock ----
-	
-	@Test
-	public void testHomesHybridIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "binary", "homes", true);
-	}
-	
-	@Test
-	public void testHomesSingleNodeIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "binary", "homes", true);
-	}
-	
-	@Test
-	public void testHomesSPHybridIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "binary", "homes", true);
-	}
-	
-	@Test
-	public void testHomesHadoopIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "binary", "homes", true);
-	}
-	
-	@Test
-	public void testHomesSparkIDBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "binary", "homes", true);
-	}
-	
-	// ---- OmitHomes CSV ----
-	
-	@Test
-	public void testOmitHomesIDHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "csv", "homesomit", true);
-	}
-	
-	@Test
-	public void testOmitHomesIDSingleNodeCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", "homesomit", true);
-	}
-	
-	@Test
-	public void testOmitHomesIDHadoopCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "csv", "homesomit", true);
-	}
-
-	@Test
-	public void testOmitHomesIDSPHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "csv", "homesomit", true);
-	}
-
-	@Test
-	public void testOmitHomesIDSparkCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", "homesomit", true);
-	}
-
-	// ---- OmitHomes BinaryBlock ----
-	
-	@Test
-	public void testOmitHomesIDHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "binary", "homesomit", true);
-	}
-	
-	@Test
-	public void testOmitHomesIDSingleNodeBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "binary", "homesomit", true);
-	}
-	
-	@Test
-	public void testOmitHomesIDHadoopBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "binary", "homesomit", true);
-	}
-	
-	@Test
-	public void testOmitHomesIDSPHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "binary", "homesomit", true);
-	}
-	
-	@Test
-	public void testOmitHomesIDSparkBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "binary", "homes2", true);
-	}
-	
-	// ---- Homes2 CSV ----
-	
-	@Test
-	public void testHomes2IDHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "csv", "homes2", true);
-	}
-	
-	@Test
-	public void testHomes2IDSingleNodeCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "csv", "homes2", true);
-	}
-	
-	@Test
-	public void testHomes2IDHadoopCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "csv", "homes2", true);
-	}
-
-	@Test
-	public void testHomes2IDSPHybridCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "csv", "homes2", true);
-	}
-
-	@Test
-	public void testHomes2IDSparkCSV() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "csv", "homes2", true);
-	}
-
-	// ---- Homes2 BinaryBlock ----
-	
-	@Test
-	public void testHomes2IDHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID, "binary", "homes2", true);
-	}
-	
-	@Test
-	public void testHomes2IDSingleNodeBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SINGLE_NODE, "binary", "homes2", true);
-	}
-	
-	@Test
-	public void testHomes2IDHadoopBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HADOOP, "binary", "homes2", true);
-	}
-	
-	@Test
-	public void testHomes2IDSPHybridBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.HYBRID_SPARK, "binary", "homes2", true);
-	}
-	
-	@Test
-	public void testHomes2IDSparkBB() 
-	{
-		runTransformTest(RUNTIME_PLATFORM.SPARK, "binary", "homes2", true);
-	}
-	
-	// ------------------------------
-	
-	private void runTransformTest( RUNTIME_PLATFORM rt, String ofmt, String dataset, boolean byid )
-	{
-		String DATASET = null, SPEC=null, TFDATA=null;
-		
-		if(dataset.equals("homes"))
-		{
-			DATASET = HOMES_DATASET;
-			SPEC = (byid ? HOMES_IDSPEC : HOMES_SPEC);
-			TFDATA = HOMES_TFDATA;
-		}
-		else if(dataset.equals("homesomit"))
-		{
-			DATASET = HOMES_OMIT_DATASET;
-			SPEC = (byid ? HOMES_OMIT_IDSPEC : HOMES_OMIT_SPEC);
-			TFDATA = HOMES_OMIT_TFDATA;
-		}
-
-		else if(dataset.equals("homes2"))
-		{
-			DATASET = HOMES2_DATASET;
-			SPEC = (byid ? HOMES2_IDSPEC : HOMES2_SPEC);
-			TFDATA = HOMES2_TFDATA;
-		}
-		else if (dataset.equals("iris"))
-		{
-			DATASET = IRIS_DATASET;
-			SPEC = (byid ? IRIS_IDSPEC : IRIS_SPEC);
-			TFDATA = IRIS_TFDATA;
-		}
-
-		RUNTIME_PLATFORM rtold = rtplatform;
-		rtplatform = rt;
-
-		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
-		if( rtplatform == RUNTIME_PLATFORM.SPARK || rtplatform == RUNTIME_PLATFORM.HYBRID_SPARK)
-			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
-
-		try
-		{
-			getAndLoadTestConfiguration(TEST_NAME1);
-			
-			/* This is for running the junit test the new way, i.e., construct the arguments directly */
-			String HOME = SCRIPT_DIR + TEST_DIR;
-			fullDMLScriptName = HOME + TEST_NAME1 + ".dml";
-			programArgs = new String[]{"-nvargs", 
-				"DATA=" + HOME + "input/" + DATASET,
-				"TFSPEC=" + HOME + "input/" + SPEC,
-				"TFMTD=" + output("tfmtd"),
-				"TFDATA=" + output("tfout"),
-				"OFMT=" + ofmt };
-	
-			boolean exceptionExpected = false;
-			runTest(true, exceptionExpected, null, -1); 
-			
-			fullDMLScriptName = HOME + TEST_NAME2 + ".dml";
-			programArgs = new String[]{"-nvargs", 
-				"DATA=" + HOME + "input/" + DATASET,
-				"APPLYMTD=" + output("tfmtd"),  // generated above
-				"TFMTD=" + output("test_tfmtd"),
-				"TFDATA=" + output("test_tfout"),
-				"OFMT=" + ofmt };
-	
-			exceptionExpected = false;
-			runTest(true, exceptionExpected, null, -1); 
-			
-			try {
-				ReaderTextCSV csvReader=  new ReaderTextCSV(new CSVFileFormatProperties(true, ",", true, 0, null));
-				MatrixBlock exp = csvReader.readMatrixFromHDFS(HOME+"input/"+ TFDATA, -1, -1, -1, -1, -1);
-				
-				MatrixBlock out = null, out2=null;
-				if(ofmt.equals("csv"))
-				{
-					ReaderTextCSV outReader=  new ReaderTextCSV(new CSVFileFormatProperties(false, ",", true, 0, null));
-					out = outReader.readMatrixFromHDFS(output("tfout"), -1, -1, -1, -1, -1);
-					out2 = outReader.readMatrixFromHDFS(output("test_tfout"), -1, -1, -1, -1, -1);
-				}
-				else
-				{
-					ReaderBinaryBlock bbReader = new ReaderBinaryBlock(false);
-					out = bbReader.readMatrixFromHDFS(
-							output("tfout"), exp.getNumRows(), exp.getNumColumns(), 
-							ConfigurationManager.getBlocksize(), 
-							ConfigurationManager.getBlocksize(),
-							-1);
-					out2 = bbReader.readMatrixFromHDFS(
-							output("test_tfout"), exp.getNumRows(), exp.getNumColumns(), 
-							ConfigurationManager.getBlocksize(), 
-							ConfigurationManager.getBlocksize(),
-							-1);
-				}
-				
-				assertTrue("Incorrect output from data transform.", equals(out,exp,  1e-10));
-				assertTrue("Incorrect output from apply transform.", equals(out2,exp,  1e-10));
-					
-			} catch (Exception e) {
-				throw new RuntimeException(e);
-			}
-		}
-		finally
-		{
-			rtplatform = rtold;
-			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-		}
-	}
-	
-	public static boolean equals(MatrixBlock mb1, MatrixBlock mb2, double epsilon)
-	{
-		if(mb1.getNumRows() != mb2.getNumRows() || mb1.getNumColumns() != mb2.getNumColumns() || mb1.getNonZeros() != mb2.getNonZeros() )
-			return false;
-		
-		// TODO: this implementation is to be optimized for different block representations
-		for(int i=0; i < mb1.getNumRows(); i++) 
-			for(int j=0; j < mb1.getNumColumns(); j++ )
-				if(!TestUtils.compareCellValue(mb1.getValue(i, j), mb2.getValue(i,j), epsilon, false))
-				{
-					System.err.println("(i="+ (i+1) + ",j=" + (j+1) + ")  " + mb1.getValue(i, j) + " != " + mb2.getValue(i, j));
-					return false;
-				}
-		
-		return true;
-	}
-	
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/scripts/functions/transform/Apply.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/Apply.dml b/src/test/scripts/functions/transform/Apply.dml
deleted file mode 100644
index 8cbec31..0000000
--- a/src/test/scripts/functions/transform/Apply.dml
+++ /dev/null
@@ -1,30 +0,0 @@
-#-------------------------------------------------------------
-#
-# 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.
-#
-#-------------------------------------------------------------
-
-
-raw = read($DATA);
-
-A = transform(target = raw, 
-              transformPath = $TFMTD, 
-              applyTransformPath = $APPLYMTD);
-
-write(A, $TFDATA, format=$OFMT);
-

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/scripts/functions/transform/ApplyFrame.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/ApplyFrame.dml b/src/test/scripts/functions/transform/ApplyFrame.dml
deleted file mode 100644
index 4fbcc36..0000000
--- a/src/test/scripts/functions/transform/ApplyFrame.dml
+++ /dev/null
@@ -1,29 +0,0 @@
-#-------------------------------------------------------------
-#
-# 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.
-#
-#-------------------------------------------------------------
-
-raw = read($DATA);
-jspec = read($TFSPEC, data_type="scalar", value_type="string")
-
-M = transformmeta(spec=jspec, transformPath=$APPLYMTD);
-A = transformapply(target=raw, spec=jspec, meta=M);
-
-write(A, $TFDATA, format=$OFMT);
-

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/scripts/functions/transform/Scaling.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/Scaling.R b/src/test/scripts/functions/transform/Scaling.R
deleted file mode 100644
index a3bfe59..0000000
--- a/src/test/scripts/functions/transform/Scaling.R
+++ /dev/null
@@ -1,36 +0,0 @@
-#-------------------------------------------------------------
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-# 
-#   http://www.apache.org/licenses/LICENSE-2.0
-# 
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-#
-#-------------------------------------------------------------
-
-
-args <- commandArgs(TRUE)
-options(digits=22)
-library("Matrix")
-
-A = read.table(args[1], sep=",");
-B = matrix(0, nrow=nrow(A), ncol=ncol(A));
-
-cols = ncol(A);
-A1 = A[, 1:cols/2];
-A2 = A[,(cols/2+1):cols]
-B[, 1:cols/2] = scale(A1, center=T, scale=F)
-B[, (cols/2+1):cols] = scale(A2)
-
-write.table(B, args[2], sep=",", row.names = F, col.names=F)

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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
deleted file mode 100644
index 1542477..0000000
--- a/src/test/scripts/functions/transform/Scaling.dml
+++ /dev/null
@@ -1,31 +0,0 @@
-#-------------------------------------------------------------
-#
-# 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.
-#
-#-------------------------------------------------------------
-
-
-raw = read($DATA);
-specJson = read($TFSPEC, data_type="scalar", value_type="string")
-
-A = transform(target = raw, 
-              transformPath = $TFMTD, 
-              spec = specJson);
-
-write(A, $TFDATA, format=$OFMT);
-

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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
deleted file mode 100644
index d36957a..0000000
--- a/src/test/scripts/functions/transform/Transform.dml
+++ /dev/null
@@ -1,31 +0,0 @@
-#-------------------------------------------------------------
-#
-# 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.
-#
-#-------------------------------------------------------------
-
-
-raw = read($DATA);
-specJson = read($TFSPEC, data_type="scalar", value_type="string");
-
-A = transform(target = raw, 
-              transformPath = $TFMTD, 
-              spec = specJson);
-
-write(A, $TFDATA, format=$OFMT);
-

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/scripts/functions/transform/TransformAndApply.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/TransformAndApply.dml b/src/test/scripts/functions/transform/TransformAndApply.dml
deleted file mode 100644
index fc97402..0000000
--- a/src/test/scripts/functions/transform/TransformAndApply.dml
+++ /dev/null
@@ -1,37 +0,0 @@
-#-------------------------------------------------------------
-#
-# 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.
-#
-#-------------------------------------------------------------
-
-#transform
-X = read($1, data_type="frame", format="csv");
-specX = $3;
-R1 = transform(target = X, spec = specX, transformPath = $5);
-
-if( 1==1 ){}
-
-#transform apply
-Y = read($2, data_type="frame", format="csv");
-specY = $4;
-R2 = transform(target = Y, spec = specY, applyTransformPath = $5, transformPath = $6);
-
-if( 1==1 ){}
-
-write(R1, $7)
-write(R2, $8);

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/scripts/functions/transform/TransformAndApplySpecX.json
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/TransformAndApplySpecX.json b/src/test/scripts/functions/transform/TransformAndApplySpecX.json
deleted file mode 100644
index b5c9a84..0000000
--- a/src/test/scripts/functions/transform/TransformAndApplySpecX.json
+++ /dev/null
@@ -1,5 +0,0 @@
-{
-    "ids": true
-    ,"recode": [ 1, 2 ]
- 
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/scripts/functions/transform/TransformAndApplySpecY.json
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/TransformAndApplySpecY.json b/src/test/scripts/functions/transform/TransformAndApplySpecY.json
deleted file mode 100644
index e10a5fa..0000000
--- a/src/test/scripts/functions/transform/TransformAndApplySpecY.json
+++ /dev/null
@@ -1,5 +0,0 @@
-{
-    "ids": true
-    ,"recode": [ 1 ]
- 
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/scripts/functions/transform/TransformReadMeta.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/TransformReadMeta.dml b/src/test/scripts/functions/transform/TransformReadMeta.dml
deleted file mode 100644
index 7328aa9..0000000
--- a/src/test/scripts/functions/transform/TransformReadMeta.dml
+++ /dev/null
@@ -1,33 +0,0 @@
-#-------------------------------------------------------------
-#
-# 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.
-#
-#-------------------------------------------------------------
-
-#transform
-X = read($1, data_type="frame", format="csv");
-specX = $2;
-R1 = transform(target = X, spec = specX, transformPath = $3);
-
-if( 1==1 ){}
-
-print(sum(R1));
-
-#transform read meta data and write as frame
-M = transformmeta(spec = specX, transformPath = $3);
-write(M, $4, format=$5);

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/scripts/functions/transform/TransformReadMeta2.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/TransformReadMeta2.dml b/src/test/scripts/functions/transform/TransformReadMeta2.dml
deleted file mode 100644
index f86911e..0000000
--- a/src/test/scripts/functions/transform/TransformReadMeta2.dml
+++ /dev/null
@@ -1,33 +0,0 @@
-#-------------------------------------------------------------
-#
-# 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.
-#
-#-------------------------------------------------------------
-
-#transform
-X = read($1, data_type="frame", format="csv", sep=$6);
-specX = $2;
-R1 = transform(target = X, spec = specX, transformPath = $3);
-
-if( 1==1 ){}
-
-print(sum(R1));
-
-#transform read meta data and write as frame
-M = transformmeta(spec = specX, transformPath = $3, sep=$6);
-write(M, $4, format=$5);

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/scripts/functions/transform/TransformReadMetaSpecX.json
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/transform/TransformReadMetaSpecX.json b/src/test/scripts/functions/transform/TransformReadMetaSpecX.json
deleted file mode 100644
index b5c9a84..0000000
--- a/src/test/scripts/functions/transform/TransformReadMetaSpecX.json
+++ /dev/null
@@ -1,5 +0,0 @@
-{
-    "ids": true
-    ,"recode": [ 1, 2 ]
- 
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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
deleted file mode 100644
index 8f40dcc..0000000
--- a/src/test/scripts/functions/transform/Transform_colnames.dml
+++ /dev/null
@@ -1,32 +0,0 @@
-#-------------------------------------------------------------
-#
-# 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.
-#
-#-------------------------------------------------------------
-
-
-raw = read($DATA);
-specJson = read($TFSPEC, data_type="scalar", value_type="string");
-
-A = transform(target = raw, 
-              transformPath = $TFMTD, 
-              spec = specJson,
-	      outputNames = $COLNAMES);
-
-write(A, $TFDATA, format=$OFMT);
-

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test_suites/java/org/apache/sysml/test/integration/functions/transform/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/transform/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/transform/ZPackageSuite.java
index e36d4a0..645a468 100644
--- a/src/test_suites/java/org/apache/sysml/test/integration/functions/transform/ZPackageSuite.java
+++ b/src/test_suites/java/org/apache/sysml/test/integration/functions/transform/ZPackageSuite.java
@@ -27,18 +27,12 @@ import org.junit.runners.Suite;
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
 	FrameCSVReadWriteTest.class,
-	RunTest.class,
-	ScalingTest.class,
-	TransformAndApplyTest.class,
 	TransformCSVFrameEncodeDecodeTest.class,
 	TransformCSVFrameEncodeReadTest.class,
 	TransformEncodeDecodeTest.class,
-	TransformFrameApplyTest.class,
 	TransformFrameEncodeApplyTest.class,
 	TransformFrameEncodeDecodeTest.class,
 	TransformFrameEncodeDecodeTokenTest.class,
-	TransformReadMetaTest.class,
-	TransformTest.class,
 })
 
 


[6/6] systemml git commit: [SYSTEMML-1300] Remove file-based transform from compiler/runtime

Posted by mb...@apache.org.
[SYSTEMML-1300] Remove file-based transform from compiler/runtime

This patch removes the old (deprecated) file-based transform in favor of
the new frame-based transform. In detail, this includes the removal of
existing cp/sp/mr file-based implementations, the entire transform
language/compiler/runtime integration, and transform-specific tests as
well as a refactoring of existing encoders and various cleanups.


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

Branch: refs/heads/master
Commit: 0cd3905f592b5ee0c867ebc952d9ba367fb8a0c9
Parents: 8a27573
Author: Matthias Boehm <mb...@gmail.com>
Authored: Wed Jun 7 22:12:01 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Jun 8 12:24:12 2017 -0700

----------------------------------------------------------------------
 src/main/java/org/apache/sysml/hops/Hop.java    |   11 +-
 .../org/apache/sysml/hops/OptimizerUtils.java   |   10 -
 .../sysml/hops/ParameterizedBuiltinOp.java      |   20 -
 .../apache/sysml/hops/recompile/Recompiler.java |   21 -
 .../sysml/hops/rewrite/HopRewriteUtils.java     |   17 -
 .../rewrite/RewriteBlockSizeAndReblock.java     |   26 +-
 .../RewriteInjectSparkPReadCheckpointing.java   |    5 +-
 .../rewrite/RewriteSplitDagUnknownCSVRead.java  |    3 +-
 .../java/org/apache/sysml/lops/CSVReBlock.java  |   41 +-
 src/main/java/org/apache/sysml/lops/Data.java   |   25 +-
 .../apache/sysml/lops/ParameterizedBuiltin.java |   64 +-
 src/main/java/org/apache/sysml/lops/Unary.java  |   10 +-
 .../java/org/apache/sysml/lops/compile/Dag.java |   58 +-
 .../org/apache/sysml/lops/compile/JobType.java  |   18 +-
 .../apache/sysml/lops/runtime/RunMRJobs.java    |   34 +-
 .../org/apache/sysml/parser/DMLTranslator.java  |    7 -
 .../org/apache/sysml/parser/Expression.java     |    2 +-
 .../ParameterizedBuiltinFunctionExpression.java |   68 +-
 .../functionobjects/ParameterizedBuiltin.java   |    6 +-
 .../instructions/CPInstructionParser.java       |    1 -
 .../instructions/MRInstructionParser.java       |    1 -
 .../instructions/SPInstructionParser.java       |    1 -
 .../cp/ParameterizedBuiltinCPInstruction.java   |   15 +-
 .../runtime/instructions/mr/MRInstruction.java  |    2 +-
 ...ReturnParameterizedBuiltinSPInstruction.java |   34 +-
 .../ParameterizedBuiltinSPInstruction.java      |   14 -
 .../instructions/spark/WriteSPInstruction.java  |   70 +-
 .../sysml/runtime/matrix/CSVReblockMR.java      |   21 +-
 .../matrix/mapred/CSVAssignRowIDMapper.java     |   19 +-
 .../matrix/mapred/MRJobConfiguration.java       |   27 -
 .../sysml/runtime/transform/ApplyTfBBMR.java    |  155 --
 .../runtime/transform/ApplyTfBBMapper.java      |  157 --
 .../sysml/runtime/transform/ApplyTfCSVMR.java   |  129 --
 .../runtime/transform/ApplyTfCSVMapper.java     |  113 --
 .../runtime/transform/ApplyTfCSVSPARK.java      |  164 --
 .../sysml/runtime/transform/BinAgent.java       |  382 -----
 .../sysml/runtime/transform/DataTransform.java  | 1496 ------------------
 .../sysml/runtime/transform/DistinctValue.java  |  105 --
 .../sysml/runtime/transform/DummycodeAgent.java |  461 ------
 .../sysml/runtime/transform/GTFMTDMapper.java   |  111 --
 .../sysml/runtime/transform/GTFMTDReducer.java  |  127 --
 .../sysml/runtime/transform/GenTfMtdMR.java     |  105 --
 .../sysml/runtime/transform/GenTfMtdSPARK.java  |  240 ---
 .../sysml/runtime/transform/MVImputeAgent.java  | 1046 ------------
 .../sysml/runtime/transform/OmitAgent.java      |  148 --
 .../sysml/runtime/transform/RecodeAgent.java    |  534 -------
 .../apache/sysml/runtime/transform/TfUtils.java |  446 +-----
 .../sysml/runtime/transform/encode/Encoder.java |   17 -
 .../runtime/transform/encode/EncoderBin.java    |  188 +++
 .../transform/encode/EncoderComposite.java      |   25 -
 .../transform/encode/EncoderDummycode.java      |  139 ++
 .../transform/encode/EncoderFactory.java        |   16 +-
 .../transform/encode/EncoderMVImpute.java       |  422 +++++
 .../runtime/transform/encode/EncoderOmit.java   |  123 ++
 .../transform/encode/EncoderPassThrough.java    |   25 -
 .../runtime/transform/encode/EncoderRecode.java |  253 +++
 .../runtime/transform/meta/TfMetaUtils.java     |    1 -
 .../functions/frame/FrameFunctionTest.java      |    3 -
 .../functions/frame/FrameMatrixReblockTest.java |    6 -
 .../functions/frame/FrameMetaReadWriteTest.java |    6 -
 .../transform/FrameCSVReadWriteTest.java        |    4 -
 .../functions/transform/RunTest.java            |  268 ----
 .../functions/transform/ScalingTest.java        |  244 ---
 .../transform/TransformAndApplyTest.java        |  143 --
 .../TransformCSVFrameEncodeDecodeTest.java      |    6 +-
 .../TransformCSVFrameEncodeReadTest.java        |    6 +-
 .../transform/TransformFrameApplyTest.java      |  194 ---
 .../TransformFrameEncodeApplyTest.java          |    4 -
 .../TransformFrameEncodeDecodeTest.java         |    4 -
 .../TransformFrameEncodeDecodeTokenTest.java    |    4 -
 .../transform/TransformReadMetaTest.java        |  205 ---
 .../functions/transform/TransformTest.java      |  709 ---------
 src/test/scripts/functions/transform/Apply.dml  |   30 -
 .../scripts/functions/transform/ApplyFrame.dml  |   29 -
 src/test/scripts/functions/transform/Scaling.R  |   36 -
 .../scripts/functions/transform/Scaling.dml     |   31 -
 .../scripts/functions/transform/Transform.dml   |   31 -
 .../functions/transform/TransformAndApply.dml   |   37 -
 .../transform/TransformAndApplySpecX.json       |    5 -
 .../transform/TransformAndApplySpecY.json       |    5 -
 .../functions/transform/TransformReadMeta.dml   |   33 -
 .../functions/transform/TransformReadMeta2.dml  |   33 -
 .../transform/TransformReadMetaSpecX.json       |    5 -
 .../functions/transform/Transform_colnames.dml  |   32 -
 .../functions/transform/ZPackageSuite.java      |    6 -
 85 files changed, 1209 insertions(+), 8689 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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 6c7089b..31e3aa6 100644
--- a/src/main/java/org/apache/sysml/hops/Hop.java
+++ b/src/main/java/org/apache/sysml/hops/Hop.java
@@ -288,11 +288,9 @@ public abstract class Hop
 			
 			try
 			{
-				if(    (this instanceof DataOp  // CSV
-							&& ((DataOp)this).getDataOpType() == DataOpTypes.PERSISTENTREAD
-							&& ((DataOp)this).getInputFormatType() == FileFormatTypes.CSV ) 
-					|| (this instanceof ParameterizedBuiltinOp 
-							&& ((ParameterizedBuiltinOp)this).getOp() == ParamBuiltinOp.TRANSFORM) )
+				if( this instanceof DataOp  // CSV
+					&& ((DataOp)this).getDataOpType() == DataOpTypes.PERSISTENTREAD
+					&& ((DataOp)this).getInputFormatType() == FileFormatTypes.CSV  )
 				{
 					reblock = new CSVReBlock( input, getRowsInBlock(), getColsInBlock(), 
 							getDataType(), getValueType(), et);
@@ -1038,7 +1036,7 @@ public abstract class Hop
 
 	public enum ParamBuiltinOp {
 		INVALID, CDF, INVCDF, GROUPEDAGG, RMEMPTY, REPLACE, REXPAND, 
-		TRANSFORM, TRANSFORMAPPLY, TRANSFORMDECODE, TRANSFORMMETA,
+		TRANSFORMAPPLY, TRANSFORMDECODE, TRANSFORMMETA,
 		TOSTRING
 	};
 
@@ -1298,7 +1296,6 @@ public abstract class Hop
 		HopsParameterizedBuiltinLops.put(ParamBuiltinOp.RMEMPTY, org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.RMEMPTY);
 		HopsParameterizedBuiltinLops.put(ParamBuiltinOp.REPLACE, org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.REPLACE);
 		HopsParameterizedBuiltinLops.put(ParamBuiltinOp.REXPAND, org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.REXPAND);
-		HopsParameterizedBuiltinLops.put(ParamBuiltinOp.TRANSFORM, org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.TRANSFORM);
 		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);

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/hops/OptimizerUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/OptimizerUtils.java b/src/main/java/org/apache/sysml/hops/OptimizerUtils.java
index 0c1c838..a40e36c 100644
--- a/src/main/java/org/apache/sysml/hops/OptimizerUtils.java
+++ b/src/main/java/org/apache/sysml/hops/OptimizerUtils.java
@@ -137,11 +137,6 @@ public class OptimizerUtils
 	public static boolean ALLOW_WORSTCASE_SIZE_EXPRESSION_EVALUATION = true;
 
 	public static boolean ALLOW_RAND_JOB_RECOMPILE = true;
-	
-	/**
-	 * Enables CP-side data transformation for small files.
-	 */
-	public static boolean ALLOW_TRANSFORM_RECOMPILE = true;
 
 	/**
 	 * Enables parfor runtime piggybacking of MR jobs into the packed jobs for
@@ -205,11 +200,6 @@ public class OptimizerUtils
 	 */
 	public static final boolean ALLOW_COMBINE_FILE_INPUT_FORMAT = true;
 	
-	/**
-	 * Enables automatic csv-binary block reblock.
-	 */
-	public static boolean ALLOW_FRAME_CSV_REBLOCK = true;
-	
 	
 	public static long GPU_MEMORY_BUDGET = -1;
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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 74542f4..7bff4bd 100644
--- a/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
+++ b/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
@@ -35,7 +35,6 @@ import org.apache.sysml.lops.GroupedAggregateM;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.lops.LopProperties.ExecType;
 import org.apache.sysml.lops.LopsException;
-import org.apache.sysml.lops.OutputParameters.Format;
 import org.apache.sysml.lops.PMMJ;
 import org.apache.sysml.lops.PartialAggregate.CorrectionLocationType;
 import org.apache.sysml.lops.ParameterizedBuiltin;
@@ -201,20 +200,6 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 				constructLopsRExpand(inputlops, et);
 				break;
 			} 
-			case TRANSFORM: {
-				ExecType et = optFindExecType();
-				
-				ParameterizedBuiltin pbilop = new ParameterizedBuiltin(inputlops,
-						HopsParameterizedBuiltinLops.get(_op), getDataType(), getValueType(), et);
-				setOutputDimensions(pbilop);
-				setLineNumbers(pbilop);
-				// output of transform is always in CSV format
-				// to produce a blocked output, this lop must be 
-				// fed into CSV Reblock lop.
-				pbilop.getOutputParameters().setFormat(Format.CSV);
-				setLops(pbilop);
-				break;
-			}
 			case CDF:
 			case INVCDF: 
 			case REPLACE:
@@ -1084,11 +1069,6 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 		}
 		else 
 		{
-			if( _op == ParamBuiltinOp.TRANSFORM ) {
-				// force remote, at runtime cp transform triggered for small files.
-				return (_etype = REMOTE);
-			}
-			
 			if ( OptimizerUtils.isMemoryBasedOptLevel() ) {
 				_etype = findExecTypeByMemEstimate();
 			}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java b/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
index 9942035..c92b735 100644
--- a/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
+++ b/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
@@ -125,7 +125,6 @@ public class Recompiler
 	//note that we scale this threshold up by the degree of available parallelism
 	private static final long CP_REBLOCK_THRESHOLD_SIZE = (long)1024*1024*1024; 
 	private static final long CP_CSV_REBLOCK_UNKNOWN_THRESHOLD_SIZE = (long)256*1024*1024;
-	private static final long CP_TRANSFORM_UNKNOWN_THRESHOLD_SIZE = (long)1024*1024*1024;
 	
 	/** Local reused rewriter for dynamic rewrites during recompile */
 
@@ -1838,26 +1837,6 @@ public class Recompiler
 		return (estFilesize < cpThreshold);
 	}
 	
-	public static boolean checkCPTransform(MRJobInstruction inst, MatrixObject[] inputs) 
-		throws DMLRuntimeException, IOException 
-	{
-		boolean ret = true;
-		
-		MatrixObject input = inputs[0]; // there can only be one input in TRANSFORM job
-		
-		Path path = new Path(input.getFileName());
-		long sizeOnHDFS = MapReduceTool.getFilesizeOnHDFS(path);
-		
-		// dimensions are not checked here, since the worst case dimensions 
-		// after transformations (with potential dummycoding) are typically unknown.
-		
-		if( sizeOnHDFS > CP_TRANSFORM_UNKNOWN_THRESHOLD_SIZE 
-				|| sizeOnHDFS*4 > OptimizerUtils.getLocalMemBudget() )
-			ret = false;
-		LOG.info("checkCPTransform(): size = " + sizeOnHDFS + ", recompile to CP = " + ret);
-		return ret;
-	}
-	
 	public static boolean checkCPDataGen( MRJobInstruction inst, String updatedRandInst ) 
 		throws DMLRuntimeException 
 	{

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java b/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
index b406bb7..bec7b38 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
@@ -982,23 +982,6 @@ public class HopRewriteUtils
 		return ret;
 	}
 	
-	public static boolean hasTransformParents( Hop hop )
-	{
-		boolean ret = false;
-		
-		ArrayList<Hop> parents = hop.getParent();
-		for( Hop p : parents )
-		{
-			if(    p instanceof ParameterizedBuiltinOp 
-				&& ((ParameterizedBuiltinOp)p).getOp()==ParamBuiltinOp.TRANSFORM) {
-				ret = true;
-			}
-		}
-			
-				
-		return ret;
-	}
-	
 	public static boolean alwaysRequiresReblock(Hop hop)
 	{
 		return (    hop instanceof DataOp 

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/hops/rewrite/RewriteBlockSizeAndReblock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteBlockSizeAndReblock.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteBlockSizeAndReblock.java
index 2e2f91f..245a333 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteBlockSizeAndReblock.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteBlockSizeAndReblock.java
@@ -28,11 +28,8 @@ import org.apache.sysml.hops.DataOp;
 import org.apache.sysml.hops.FunctionOp;
 import org.apache.sysml.hops.Hop;
 import org.apache.sysml.hops.OptimizerUtils;
-import org.apache.sysml.hops.Hop.DataOpTypes;
 import org.apache.sysml.hops.Hop.FileFormatTypes;
-import org.apache.sysml.hops.Hop.ParamBuiltinOp;
 import org.apache.sysml.hops.HopsException;
-import org.apache.sysml.hops.ParameterizedBuiltinOp;
 import org.apache.sysml.parser.Expression.DataType;
 
 /**
@@ -98,7 +95,7 @@ public class RewriteBlockSizeAndReblock extends HopRewriteRule
 			if( canReblock && 
 				( (dop.getDataType() == DataType.MATRIX && (dop.getRowsInBlock() != blocksize || dop.getColsInBlock() != blocksize))
 				||(dop.getDataType() == DataType.FRAME && OptimizerUtils.isSparkExecutionMode() && (dop.getInputFormatType()==FileFormatTypes.TEXT
-						  || dop.getInputFormatType()==FileFormatTypes.CSV && OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK))) ) 
+						  || dop.getInputFormatType()==FileFormatTypes.CSV))) ) 
 			{
 				if( dop.getDataOpType() == DataOp.DataOpTypes.PERSISTENTREAD) 
 				{
@@ -146,27 +143,6 @@ public class RewriteBlockSizeAndReblock extends HopRewriteRule
 				}
 			}
 		} 
-		//TODO remove once transform rebased to frames
-		else if ( (hop instanceof ParameterizedBuiltinOp && ((ParameterizedBuiltinOp)hop).getOp() == ParamBuiltinOp.TRANSFORM) ) {
-			
-			// check if there exists a non-csv-write output. If yes, add reblock
-			boolean rblk = false;
-			for(Hop out : hop.getParent()) 
-			{
-				if ( !(out instanceof DataOp 
-						&& ((DataOp)out).getDataOpType() == DataOpTypes.PERSISTENTWRITE 
-						&& ((DataOp)out).getInputFormatType() == FileFormatTypes.CSV) )
-				{
-					rblk = true;
-					break;
-				}
-			}
-			if ( rblk )
-			{
-				hop.setRequiresReblock(true);
-				hop.setOutputBlocksizes(blocksize, blocksize);
-			}
-		}
 		else //NO DATAOP 
 		{
 			// TODO: following two lines are commented, and the subsequent hack is used instead!

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkPReadCheckpointing.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkPReadCheckpointing.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkPReadCheckpointing.java
index 9aceab4..755c2e8 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkPReadCheckpointing.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkPReadCheckpointing.java
@@ -68,9 +68,8 @@ public class RewriteInjectSparkPReadCheckpointing extends HopRewriteRule
 			return;
 		
 		// The reblocking is performed after transform, and hence checkpoint only non-transformed reads.   
-		if(    (hop instanceof DataOp && ((DataOp)hop).getDataOpType()==DataOpTypes.PERSISTENTREAD && !HopRewriteUtils.hasTransformParents(hop))
-			|| (hop.requiresReblock())
-			)
+		if(    (hop instanceof DataOp && ((DataOp)hop).getDataOpType()==DataOpTypes.PERSISTENTREAD)
+			|| hop.requiresReblock() )
 		{
 			//make given hop for checkpointing (w/ default storage level)
 			//note: we do not recursively process childs here in order to prevent unnecessary checkpoints

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java
index ee7b9c6..8396813 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java
@@ -146,8 +146,7 @@ public class RewriteSplitDagUnknownCSVRead extends StatementBlockRewriteRule
 			if(    dop.getDataOpType() == DataOpTypes.PERSISTENTREAD
 				&& dop.getInputFormatType() == FileFormatTypes.CSV
 				&& !dop.dimsKnown()
-				&& !HopRewriteUtils.hasOnlyWriteParents(dop, true, false)
-				&& !HopRewriteUtils.hasTransformParents(hop) )
+				&& !HopRewriteUtils.hasOnlyWriteParents(dop, true, false) )
 			{
 				cand.add(dop);
 			}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/lops/CSVReBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/CSVReBlock.java b/src/main/java/org/apache/sysml/lops/CSVReBlock.java
index a15e036..55571f7 100644
--- a/src/main/java/org/apache/sysml/lops/CSVReBlock.java
+++ b/src/main/java/org/apache/sysml/lops/CSVReBlock.java
@@ -21,12 +21,10 @@ package org.apache.sysml.lops;
 
 import org.apache.sysml.lops.LopProperties.ExecLocation;
 import org.apache.sysml.lops.LopProperties.ExecType;
-import org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes;
 import org.apache.sysml.lops.compile.JobType;
 import org.apache.sysml.parser.DataExpression;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
-import org.apache.sysml.parser.ParameterizedBuiltinFunctionExpression;
 
 
 /**
@@ -55,18 +53,7 @@ public class CSVReBlock extends Lop
 		boolean breaksAlignment = false;
 		boolean aligner = false;
 		boolean definesMRJob = true;
-		
-		// If the input to reblock is a tranform, then piggyback it along with transform
-		if ( input instanceof ParameterizedBuiltin 
-				&& ((ParameterizedBuiltin)input).getOp() == OperationTypes.TRANSFORM ) 
-		{
-			definesMRJob = false;
-			lps.addCompatibility(JobType.TRANSFORM);
-		}
-		else 
-		{
-			lps.addCompatibility(JobType.CSV_REBLOCK);
-		}
+		lps.addCompatibility(JobType.CSV_REBLOCK);
 		
 		if(et == ExecType.MR) {
 			this.lps.setProperties( inputs, ExecType.MR, ExecLocation.MapAndReduce, breaksAlignment, aligner, definesMRJob );
@@ -88,19 +75,16 @@ public class CSVReBlock extends Lop
 	private String prepCSVProperties() throws LopsException {
 		StringBuilder sb = new StringBuilder();
 
-		boolean isSparkTransformInput = false;
-		Data dataInput = null;
-		if(getInputs().get(0).getType() == Type.Data)
-			dataInput = (Data)getInputs().get(0);
-		else if ( getInputs().get(0).getType() == Type.ParameterizedBuiltin && ((ParameterizedBuiltin)getInputs().get(0)).getOp() == OperationTypes.TRANSFORM) {
-			isSparkTransformInput = (getExecType() == ExecType.SPARK);
-			dataInput = (Data) ((ParameterizedBuiltin)getInputs().get(0)).getNamedInput(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_DATA);
-		}
+		Data dataInput = (Data)getInputs().get(0);
 		
-		Lop headerLop = dataInput.getNamedInputLop(DataExpression.DELIM_HAS_HEADER_ROW, String.valueOf(DataExpression.DEFAULT_DELIM_HAS_HEADER_ROW));
-		Lop delimLop = dataInput.getNamedInputLop(DataExpression.DELIM_DELIMITER, DataExpression.DEFAULT_DELIM_DELIMITER);
-		Lop fillLop = dataInput.getNamedInputLop(DataExpression.DELIM_FILL, String.valueOf(DataExpression.DEFAULT_DELIM_FILL)); 
-		Lop fillValueLop = dataInput.getNamedInputLop(DataExpression.DELIM_FILL_VALUE, String.valueOf(DataExpression.DEFAULT_DELIM_FILL_VALUE));
+		Lop headerLop = dataInput.getNamedInputLop(DataExpression.DELIM_HAS_HEADER_ROW, 
+			String.valueOf(DataExpression.DEFAULT_DELIM_HAS_HEADER_ROW));
+		Lop delimLop = dataInput.getNamedInputLop(DataExpression.DELIM_DELIMITER, 
+			DataExpression.DEFAULT_DELIM_DELIMITER);
+		Lop fillLop = dataInput.getNamedInputLop(DataExpression.DELIM_FILL, 
+			String.valueOf(DataExpression.DEFAULT_DELIM_FILL)); 
+		Lop fillValueLop = dataInput.getNamedInputLop(DataExpression.DELIM_FILL_VALUE, 
+			String.valueOf(DataExpression.DEFAULT_DELIM_FILL_VALUE));
 		
 		if (headerLop.isVariable())
 			throw new LopsException(this.printErrorLocation()
@@ -119,10 +103,7 @@ public class CSVReBlock extends Lop
 					+ "Parameter " + DataExpression.DELIM_FILL_VALUE
 					+ " must be a literal.");
 
-		// Output from transform() does not have a header
-		// On MR, reblock is piggybacked along with transform, and hence 
-		// specific information about header needn't be passed through instruction
-		sb.append( ((Data)headerLop).getBooleanValue() && !isSparkTransformInput );
+		sb.append( ((Data)headerLop).getBooleanValue() );
 		sb.append( OPERAND_DELIMITOR );
 		sb.append( ((Data)delimLop).getStringValue() );
 		sb.append( OPERAND_DELIMITOR );

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/lops/Data.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/Data.java b/src/main/java/org/apache/sysml/lops/Data.java
index b8e40af..3d5db8a 100644
--- a/src/main/java/org/apache/sysml/lops/Data.java
+++ b/src/main/java/org/apache/sysml/lops/Data.java
@@ -139,18 +139,9 @@ public class Data extends Lop
 		
 
 		if ( getFileFormatType() == FileFormatTypes.CSV ) {
-			Lop input = getInputs().get(0);
-			// If the input is data transform, then csv write can be piggybacked onto TRANSFORM job.
-			// Otherwise, the input must be converted to csv format via WriteCSV MR job.
-			if ( input instanceof ParameterizedBuiltin 
-					&& ((ParameterizedBuiltin)input).getOp() == org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.TRANSFORM ) {
-				lps.addCompatibility(JobType.TRANSFORM);
-				definesMRJob = false;
-			}
-			else {
-				lps.addCompatibility(JobType.CSV_WRITE);
-				definesMRJob = true;
-			}
+			// The input must be converted to csv format via WriteCSV MR job.
+			lps.addCompatibility(JobType.CSV_WRITE);
+			definesMRJob = true;
 		}
 		else {
 			/*
@@ -477,16 +468,8 @@ public class Data extends Lop
 				
 				if ( this.getExecType() == ExecType.SPARK ) 
 				{
-					boolean isInputMatrixBlock = true;
-					Lop input = getInputs().get(0);
-					if ( input instanceof ParameterizedBuiltin 
-							&& ((ParameterizedBuiltin)input).getOp() == ParameterizedBuiltin.OperationTypes.TRANSFORM ) {
-						// in the case of transform input, the input will be Text strings insteadof MatrixBlocks 
-						// This information is used to have correct class information while accessing RDDs from the symbol table 
-						isInputMatrixBlock = false;
-					}
 					sb.append(OPERAND_DELIMITOR);
-					sb.append(isInputMatrixBlock);
+					sb.append(true); //isInputMatrixBlock
 				}
 			}
 			

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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 fdaf3c5..fb48d2f 100644
--- a/src/main/java/org/apache/sysml/lops/ParameterizedBuiltin.java
+++ b/src/main/java/org/apache/sysml/lops/ParameterizedBuiltin.java
@@ -28,7 +28,6 @@ import org.apache.sysml.lops.LopProperties.ExecType;
 import org.apache.sysml.lops.compile.JobType;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
-import org.apache.sysml.parser.ParameterizedBuiltinFunctionExpression;
 
 
 /**
@@ -40,7 +39,7 @@ public class ParameterizedBuiltin extends Lop
 	
 	public enum OperationTypes { 
 		CDF, INVCDF, RMEMPTY, REPLACE, REXPAND,
-		TRANSFORM, TRANSFORMAPPLY, TRANSFORMDECODE, TRANSFORMMETA,
+		TRANSFORMAPPLY, TRANSFORMDECODE, TRANSFORMMETA,
 		TOSTRING
 	};
 	
@@ -99,11 +98,6 @@ public class ParameterizedBuiltin extends Lop
 			lps.addCompatibility(JobType.REBLOCK);
 			breaksAlignment=true;
 		}
-		else if ( _operation == OperationTypes.TRANSFORM && et == ExecType.MR ) {
-			definesMRJob = true;
-			eloc = ExecLocation.MapAndReduce;
-			lps.addCompatibility(JobType.TRANSFORM);
-		}
 		else //executed in CP / CP_FILE / SPARK
 		{
 			eloc = ExecLocation.ControlProgram;
@@ -212,8 +206,7 @@ public class ParameterizedBuiltin extends Lop
 				}
 				
 				break;
-				
-			case TRANSFORM: 
+			
 			case TRANSFORMAPPLY:
 			case TRANSFORMDECODE:
 			case TRANSFORMMETA:	{
@@ -400,59 +393,6 @@ public class ParameterizedBuiltin extends Lop
 		
 		return sb.toString();
 	}
-	
-	@Override 
-	public String getInstructions(int output_index) 
-		throws LopsException
-	{
-		StringBuilder sb = new StringBuilder();
-		sb.append( getExecType() );
-		sb.append( Lop.OPERAND_DELIMITOR );
-
-		if(_operation== OperationTypes.TRANSFORM) 
-		{
-			sb.append( "transform" );
-			sb.append( OPERAND_DELIMITOR );
-			
-			Lop iLop = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_DATA);
-			sb.append(iLop.prepInputOperand(getInputIndex("target")));
-			sb.append( OPERAND_DELIMITOR );
-			
-			Lop iLop2 = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_MTD);
-			sb.append(iLop2.prepScalarLabel());
-			sb.append( OPERAND_DELIMITOR );
-			
-			// apply transform
-			Lop iLop3a = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_APPLYMTD);
-			if ( iLop3a != null ) {
-				sb.append("applymtd=");
-				sb.append(iLop3a.prepScalarLabel());
-				sb.append( OPERAND_DELIMITOR );
-			}
-			
-			// transform specification (transform: mandatory, transformapply: optional)
-			Lop iLop3b = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_SPEC);
-			if ( iLop3b != null ) {
-				sb.append("spec=");
-				sb.append(iLop3b.prepScalarLabel());
-				sb.append( OPERAND_DELIMITOR );
-			}
-			
-			Lop iLop4 = _inputParams.get(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_OUTNAMES);
-			if( iLop4 != null ) {
-				sb.append("outnames=");
-				sb.append(iLop4.prepScalarLabel());
-				sb.append( OPERAND_DELIMITOR );
-			}
-			
-			sb.append( prepOutputOperand(output_index));
-		}
-		else
-			throw new LopsException(this.printErrorLocation() + "In ParameterizedBuiltin Lop, Unknown operation: " + _operation);
-	
-		return sb.toString();
-	}
-	
 
 	@Override
 	public String toString() {

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/lops/Unary.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/Unary.java b/src/main/java/org/apache/sysml/lops/Unary.java
index 087ffc4..cc53666 100644
--- a/src/main/java/org/apache/sysml/lops/Unary.java
+++ b/src/main/java/org/apache/sysml/lops/Unary.java
@@ -95,12 +95,11 @@ public class Unary extends Lop
 			lps.addCompatibility(JobType.ANY);
 			lps.removeNonPiggybackableJobs();
 			lps.removeCompatibility(JobType.CM_COV); // CM_COV allows only reducer instructions but this is MapOrReduce. TODO: piggybacking should be updated to take this extra constraint.
-			lps.removeCompatibility(JobType.TRANSFORM);
-			this.lps.setProperties(inputs, et, ExecLocation.MapOrReduce, breaksAlignment, aligner, definesMRJob);
+			lps.setProperties(inputs, et, ExecLocation.MapOrReduce, breaksAlignment, aligner, definesMRJob);
 		}
 		else {
 			lps.addCompatibility(JobType.INVALID);
-			this.lps.setProperties(inputs, et, ExecLocation.ControlProgram, breaksAlignment, aligner, definesMRJob);
+			lps.setProperties(inputs, et, ExecLocation.ControlProgram, breaksAlignment, aligner, definesMRJob);
 		}
 	}
 
@@ -150,12 +149,11 @@ public class Unary extends Lop
 			lps.addCompatibility(JobType.ANY);
 			lps.removeNonPiggybackableJobs();
 			lps.removeCompatibility(JobType.CM_COV); // CM_COV allows only reducer instructions but this is MapOrReduce. TODO: piggybacking should be updated to take this extra constraint.
-			lps.removeCompatibility(JobType.TRANSFORM);
-			this.lps.setProperties(inputs, et, ExecLocation.MapOrReduce, breaksAlignment, aligner, definesMRJob);
+			lps.setProperties(inputs, et, ExecLocation.MapOrReduce, breaksAlignment, aligner, definesMRJob);
 		}
 		else {
 			lps.addCompatibility(JobType.INVALID);
-			this.lps.setProperties(inputs, et, ExecLocation.ControlProgram, breaksAlignment, aligner, definesMRJob);
+			lps.setProperties(inputs, et, ExecLocation.ControlProgram, breaksAlignment, aligner, definesMRJob);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/lops/compile/Dag.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/compile/Dag.java b/src/main/java/org/apache/sysml/lops/compile/Dag.java
index 5816f3f..aab4303 100644
--- a/src/main/java/org/apache/sysml/lops/compile/Dag.java
+++ b/src/main/java/org/apache/sysml/lops/compile/Dag.java
@@ -54,14 +54,12 @@ import org.apache.sysml.lops.MapMult;
 import org.apache.sysml.lops.OutputParameters;
 import org.apache.sysml.lops.OutputParameters.Format;
 import org.apache.sysml.lops.PMMJ;
-import org.apache.sysml.lops.ParameterizedBuiltin;
 import org.apache.sysml.lops.PickByCount;
 import org.apache.sysml.lops.SortKeys;
 import org.apache.sysml.lops.Unary;
 import org.apache.sysml.parser.DataExpression;
 import org.apache.sysml.parser.Expression;
 import org.apache.sysml.parser.Expression.DataType;
-import org.apache.sysml.parser.ParameterizedBuiltinFunctionExpression;
 import org.apache.sysml.parser.StatementBlock;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.parfor.ProgramConverter;
@@ -2374,50 +2372,6 @@ public class Dag<N extends Lop>
 				
 				out.addLastInstruction(currInstr);
 			}
-			else if(node instanceof ParameterizedBuiltin 
-					&& ((ParameterizedBuiltin)node).getOp() == org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.TRANSFORM) {
-				
-				ParameterizedBuiltin pbi = (ParameterizedBuiltin)node;
-				Lop input = pbi.getNamedInput(ParameterizedBuiltinFunctionExpression.TF_FN_PARAM_DATA);
-				if(input.getDataType()== DataType.FRAME) {
-					
-					// Output of transform is in CSV format, which gets subsequently reblocked 
-					// TODO: change it to output binaryblock
-					
-					Data dataInput = (Data) input;
-					oparams.setFile_name(getNextUniqueFilename());
-					oparams.setLabel(getNextUniqueVarname(DataType.MATRIX));
-
-					// generate an instruction that creates a symbol table entry for the new variable in CSV format
-					Data delimLop = (Data) dataInput.getNamedInputLop(
-							DataExpression.DELIM_DELIMITER, DataExpression.DEFAULT_DELIM_DELIMITER);
-					
-					Instruction createvarInst = VariableCPInstruction.prepareCreateVariableInstruction(
-					        oparams.getLabel(), oparams.getFile_name(), true, 
-					        DataType.MATRIX, OutputInfo.outputInfoToString(OutputInfo.CSVOutputInfo),
-							new MatrixCharacteristics(oparams.getNumRows(), oparams.getNumCols(), -1, -1, oparams.getNnz()), oparams.getUpdateType(), 
-							false, delimLop.getStringValue(), true
-						);
-					
-					createvarInst.setLocation(node);
-					
-					out.addPreInstruction(createvarInst);
-
-					// temp file as well as the variable has to be deleted at the end
-					Instruction currInstr = VariableCPInstruction.prepareRemoveInstruction(oparams.getLabel());
-					
-					currInstr.setLocation(node);
-					
-					out.addLastInstruction(currInstr);
-
-					// finally, add the generated filename and variable name to the list of outputs
-					out.setFileName(oparams.getFile_name());
-					out.setVarName(oparams.getLabel());
-				}
-				else {
-					throw new LopsException("Input to transform() has an invalid type: " + input.getDataType() + ", it must be FRAME.");
-				}
-			}
 			else if(!(node instanceof FunctionCallCP)) //general case
 			{
 				// generate temporary filename and a variable name to hold the
@@ -2913,7 +2867,7 @@ public class Dag<N extends Lop>
 		 */
 		
 		// 
-		if ( jt != JobType.REBLOCK && jt != JobType.CSV_REBLOCK && jt != JobType.DATAGEN && jt != JobType.TRANSFORM) {
+		if ( jt != JobType.REBLOCK && jt != JobType.CSV_REBLOCK && jt != JobType.DATAGEN ) {
 			for (int i=0; i < inputInfos.size(); i++)
 				if ( inputInfos.get(i) == InputInfo.BinaryCellInputInfo || inputInfos.get(i) == InputInfo.TextCellInputInfo )
 					cellModeOverride = true;
@@ -3129,9 +3083,7 @@ public class Dag<N extends Lop>
 		}
 
 		if (node.getExecLocation() == ExecLocation.Data ) {
-			if ( ((Data)node).getFileFormatType() == FileFormatTypes.CSV 
-					&& !(node.getInputs().get(0) instanceof ParameterizedBuiltin 
-							&& ((ParameterizedBuiltin)node.getInputs().get(0)).getOp() == org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.TRANSFORM)) {
+			if ( ((Data)node).getFileFormatType() == FileFormatTypes.CSV ) {
 				// Generate write instruction, which goes into CSV_WRITE Job
 				int output_index = start_index[0];
 				shuffleInstructions.add(node.getInstructions(inputIndices.get(0), output_index));
@@ -3171,12 +3123,6 @@ public class Dag<N extends Lop>
 				break;
 				
 			case ParameterizedBuiltin:
-				if( ((ParameterizedBuiltin)node).getOp() == org.apache.sysml.lops.ParameterizedBuiltin.OperationTypes.TRANSFORM ) {
-					shuffleInstructions.add(node.getInstructions(output_index));
-					if(DMLScript.ENABLE_DEBUG_MODE) {
-						MRJobLineNumbers.add(node._beginLine);
-					}
-				}
 				break;
 				
 			/* Lop types that take two inputs */

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/lops/compile/JobType.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/compile/JobType.java b/src/main/java/org/apache/sysml/lops/compile/JobType.java
index 0ca5b49..492be7e 100644
--- a/src/main/java/org/apache/sysml/lops/compile/JobType.java
+++ b/src/main/java/org/apache/sysml/lops/compile/JobType.java
@@ -22,7 +22,6 @@ package org.apache.sysml.lops.compile;
 import org.apache.sysml.hops.Hop.FileFormatTypes;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.lops.Data;
-import org.apache.sysml.lops.ParameterizedBuiltin;
 import org.apache.sysml.runtime.DMLRuntimeException;
 
 
@@ -70,8 +69,7 @@ public enum JobType
 	DATA_PARTITION	(11, "DATAPARTITION", 	false, 			false, 								true),
 	CSV_REBLOCK		(12, "CSV_REBLOCK", 	false, 			false, 								false),
 	CSV_WRITE		(13, "CSV_WRITE", 		false, 			false, 								true),
-	TRANSFORM		(14, "TRANSFORM", 		false, 			true, 								false),
-	GMRCELL			(15, "GMRCELL", 		false, 			false, 								false);
+	GMRCELL			(14, "GMRCELL", 		false, 			false, 								false);
 
 
 	
@@ -140,8 +138,6 @@ public enum JobType
 				return Lop.Type.MMRJ;
 			else if ( getName().equals("SORT") )
 				return Lop.Type.SortKeys;
-			else if ( getName().equals("TRANSFORM"))
-				return Lop.Type.ParameterizedBuiltin;
 			else 
 				throw new DMLRuntimeException("Shuffle Lop Type is not defined for a job (" + getName() + ") that allows a single shuffle instruction.");
 		}
@@ -178,19 +174,13 @@ public enum JobType
 		
 		case CSVReBlock:	return JobType.CSV_REBLOCK;
 		
-		case ParameterizedBuiltin:		
-				if( ((ParameterizedBuiltin)node).getOp() == ParameterizedBuiltin.OperationTypes.TRANSFORM )
-					return JobType.TRANSFORM;
-		
 		case Data:
 			/*
 			 * Only Write LOPs with external data formats (except MatrixMarket) produce MR Jobs
 			 */
 			FileFormatTypes fmt = ((Data) node).getFileFormatType();
-			if ( fmt == FileFormatTypes.CSV )
-				return JobType.CSV_WRITE;
-			else
-				return null;
+			return ( fmt == FileFormatTypes.CSV ) ?
+				JobType.CSV_WRITE : null;
 			
 		default:
 			return null; 
@@ -205,7 +195,7 @@ public enum JobType
 		else {
 			if ( getName().equals("MMCJ") )
 				return false;
-			else if ( getName().equals("MMRJ") || getName().equals("SORT")  || getName().equals("TRANSFORM"))
+			else if ( getName().equals("MMRJ") || getName().equals("SORT"))
 				return true;
 			else 
 				throw new DMLRuntimeException("Implementation for isCompatibleWithParentNodes() is missing for a job (" + getName() + ") that allows a single shuffle instruction.");

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/lops/runtime/RunMRJobs.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/runtime/RunMRJobs.java b/src/main/java/org/apache/sysml/lops/runtime/RunMRJobs.java
index 71de7c8..5cd0f60 100644
--- a/src/main/java/org/apache/sysml/lops/runtime/RunMRJobs.java
+++ b/src/main/java/org/apache/sysml/lops/runtime/RunMRJobs.java
@@ -22,7 +22,6 @@ package org.apache.sysml.lops.runtime;
 import java.io.IOException;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.wink.json4j.JSONException;
 
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
@@ -69,7 +68,6 @@ import org.apache.sysml.runtime.matrix.data.LibMatrixDatagen;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.OutputInfo;
 import org.apache.sysml.runtime.matrix.data.RandomMatrixGenerator;
-import org.apache.sysml.runtime.transform.DataTransform;
 import org.apache.sysml.runtime.util.MapReduceTool;
 import org.apache.sysml.utils.Statistics;
 
@@ -305,25 +303,7 @@ public class RunMRJobs
 			case DATA_PARTITION:
 				ret = DataPartitionMR.runJob(inst, inputMatrices, shuffleInst, inst.getIv_resultIndices(), outputMatrices, inst.getIv_numReducers(), inst.getIv_replication());
 				break;
-				
-			case TRANSFORM:
-				
-				if(    ConfigurationManager.isDynamicRecompilation()
-						&& OptimizerUtils.ALLOW_TRANSFORM_RECOMPILE
-						&& DMLScript.rtplatform != RUNTIME_PLATFORM.HADOOP 
-						&& Recompiler.checkCPTransform( inst, inputMatrices ) ) 
-					{
-						// transform the data and generate output in CSV format
-						ret = executeInMemoryTransform(inst, inputMatrices, outputMatrices);
-						Statistics.decrementNoOfExecutedMRJobs();
-						execCP = true;
-					}
-					else 
-					{
-						ret = DataTransform.mrDataTransform(inst, inputMatrices, shuffleInst, otherInst, inst.getIv_resultIndices(), outputMatrices, inst.getIv_numReducers(), inst.getIv_replication());
-					}
-				break;
-				
+			
 			default:
 				throw new DMLRuntimeException("Invalid jobtype: " + inst.getJobType());
 			}
@@ -360,11 +340,12 @@ public class RunMRJobs
 						
 						outputMatrices[i].setHDFSFileExists(true);
 						
-						if ( inst.getJobType() != JobType.CSV_WRITE && inst.getJobType() != JobType.TRANSFORM) {
+						if ( inst.getJobType() != JobType.CSV_WRITE ) {
 							// write out metadata file
 							// Currently, valueType information in not stored in MR instruction, 
 							// since only DOUBLE matrices are supported ==> hard coded the value type information for now
-							MapReduceTool.writeMetaDataFile(fname + ".mtd", ValueType.DOUBLE,  ((MatrixDimensionsMetaData)ret.getMetaData(i)).getMatrixCharacteristics(), outinfo);
+							MapReduceTool.writeMetaDataFile(fname + ".mtd", ValueType.DOUBLE,  
+								((MatrixDimensionsMetaData)ret.getMetaData(i)).getMatrixCharacteristics(), outinfo);
 						}
 					}
 				}
@@ -558,11 +539,4 @@ public class RunMRJobs
 		
 		return  new JobReturn( mc, inst.getOutputInfos(), true);
 	}
-	
-	private static JobReturn executeInMemoryTransform( MRJobInstruction inst, MatrixObject[] inputMatrices, MatrixObject[] outputMatrices) throws IOException, DMLRuntimeException, IllegalArgumentException, JSONException {
-		return DataTransform.cpDataTransform(
-				inst.getIv_shuffleInstructions(), 
-				inputMatrices, 
-				outputMatrices);
-	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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 a40c736..75f11b5 100644
--- a/src/main/java/org/apache/sysml/parser/DMLTranslator.java
+++ b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
@@ -1920,13 +1920,6 @@ public class DMLTranslator
 			currBuiltinOp = new ReorgOp(target.getName(), target.getDataType(), target.getValueType(), ReOrgOp.SORT, inputs);
 			
 			break;
-			
-		case TRANSFORM:
-			currBuiltinOp = new ParameterizedBuiltinOp(
-									target.getName(), target.getDataType(), 
-									target.getValueType(), ParamBuiltinOp.TRANSFORM, 
-									paramHops);
-			break;	
 		
 		case TRANSFORMAPPLY:
 			currBuiltinOp = new ParameterizedBuiltinOp(

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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 01a0a77..9ee3fba 100644
--- a/src/main/java/org/apache/sysml/parser/Expression.java
+++ b/src/main/java/org/apache/sysml/parser/Expression.java
@@ -139,7 +139,7 @@ public abstract class Expression
 		GROUPEDAGG, RMEMPTY, REPLACE, ORDER, 
 		// Distribution Functions
 		CDF, INVCDF, PNORM, QNORM, PT, QT, PF, QF, PCHISQ, QCHISQ, PEXP, QEXP,
-		TRANSFORM, TRANSFORMAPPLY, TRANSFORMDECODE, TRANSFORMENCODE, TRANSFORMMETA,
+		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/systemml/blob/0cd3905f/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 aa888d3..4a3da28 100644
--- a/src/main/java/org/apache/sysml/parser/ParameterizedBuiltinFunctionExpression.java
+++ b/src/main/java/org/apache/sysml/parser/ParameterizedBuiltinFunctionExpression.java
@@ -25,7 +25,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 
 import org.apache.sysml.hops.Hop.ParamBuiltinOp;
-import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.parser.LanguageException.LanguageErrorCodes;
 
 
@@ -39,8 +38,6 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 	public static final String TF_FN_PARAM_MTD2 = "meta";
 	public static final String TF_FN_PARAM_SPEC = "spec";
 	public static final String TF_FN_PARAM_MTD = "transformPath"; //NOTE MB: for backwards compatibility
-	public static final String TF_FN_PARAM_APPLYMTD = "applyTransformPath";
-	public static final String TF_FN_PARAM_OUTNAMES = "outputNames";
 	
 	private static HashMap<String, Expression.ParameterizedBuiltinFunctionOp> opcodeMap;
 	static {
@@ -67,7 +64,6 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 		opcodeMap.put("qexp",	Expression.ParameterizedBuiltinFunctionOp.QEXP);
 
 		// data transformation functions
-		opcodeMap.put("transform",	Expression.ParameterizedBuiltinFunctionOp.TRANSFORM);
 		opcodeMap.put("transformapply",	Expression.ParameterizedBuiltinFunctionOp.TRANSFORMAPPLY);
 		opcodeMap.put("transformdecode", Expression.ParameterizedBuiltinFunctionOp.TRANSFORMDECODE);
 		opcodeMap.put("transformencode", Expression.ParameterizedBuiltinFunctionOp.TRANSFORMENCODE);
@@ -222,15 +218,11 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 		case ORDER:
 			validateOrder(output, conditional);
 			break;
-
-		case TRANSFORM:
-			validateTransform(output, conditional);
-			break;
 		
 		case TRANSFORMAPPLY:
 			validateTransformApply(output, conditional);
 			break;
-			
+		
 		case TRANSFORMDECODE:
 			validateTransformDecode(output, conditional);
 			break;	
@@ -289,64 +281,6 @@ public class ParameterizedBuiltinFunctionExpression extends DataIdentifier
 		return;
 	}
 	
-	// example: A = transform(data=D, txmtd="", txspec="")
-	private void validateTransform(DataIdentifier output, boolean conditional) 
-		throws LanguageException 
-	{
-		//validate data
-		checkDataType("transform", TF_FN_PARAM_DATA, DataType.FRAME, conditional);
-		
-		Expression txmtd = getVarParam(TF_FN_PARAM_MTD);
-		if( txmtd==null ) {
-			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_MTD + "' must be a string value (a scalar).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
-		}
-		
-		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_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 '" + TF_FN_PARAM_SPEC + "' must be a string value (a scalar).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
-		}	
-		
-		if ( applyMTD != null ) {
-			if( applyMTD.getOutput().getDataType() != DataType.SCALAR  || applyMTD.getOutput().getValueType() != ValueType.STRING ){	
-				raiseValidateError("Apply transformation metadata file'" + TF_FN_PARAM_APPLYMTD + "' must be a string value (a scalar).", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
-			}
-			
-			//NOTE: txspec can still be optionally specified; if specified it takes precedence over 
-			// specification persisted in txmtd during transform.
-		}
-		
-		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_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);
-		}
-		
-		// disable frame csv reblocks as transform operates directly over csv files
-		// (this is required to support both file-based transform and frame-based
-		// transform at the same time; hence, transform and frame-based transform
-		// functions over csv cannot be used in the same script; accordingly we
-		// give an appropriate warning)
-		OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = false;
-		raiseValidateError("Disable frame csv reblock to support file-based transform.", true);
-		
-		// Output is a matrix with same dims as input
-		output.setDataType(DataType.MATRIX);
-		output.setFormatType(FormatType.CSV);
-		output.setValueType(ValueType.DOUBLE);
-		// Output dimensions may not be known at compile time, for example when dummycoding.
-		output.setDimensions(-1, -1);
-	}
-	
 	// example: A = transformapply(target=X, meta=M, spec=s)
 	private void validateTransformApply(DataIdentifier output, boolean conditional) 
 		throws LanguageException 

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/functionobjects/ParameterizedBuiltin.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/functionobjects/ParameterizedBuiltin.java b/src/main/java/org/apache/sysml/runtime/functionobjects/ParameterizedBuiltin.java
index eb69068..8635833 100644
--- a/src/main/java/org/apache/sysml/runtime/functionobjects/ParameterizedBuiltin.java
+++ b/src/main/java/org/apache/sysml/runtime/functionobjects/ParameterizedBuiltin.java
@@ -46,7 +46,7 @@ public class ParameterizedBuiltin extends ValueFunction
 	
 	public enum ParameterizedBuiltinCode { 
 		CDF, INVCDF, RMEMPTY, REPLACE, REXPAND,
-		TRANSFORM, TRANSFORMAPPLY, TRANSFORMDECODE };
+		TRANSFORMAPPLY, TRANSFORMDECODE };
 	public enum ProbabilityDistributionCode { 
 		INVALID, NORMAL, EXP, CHISQ, F, T };
 	
@@ -62,7 +62,6 @@ public class ParameterizedBuiltin extends ValueFunction
 		String2ParameterizedBuiltinCode.put( "rmempty", ParameterizedBuiltinCode.RMEMPTY);
 		String2ParameterizedBuiltinCode.put( "replace", ParameterizedBuiltinCode.REPLACE);
 		String2ParameterizedBuiltinCode.put( "rexpand", ParameterizedBuiltinCode.REXPAND);
-		String2ParameterizedBuiltinCode.put( "transform", ParameterizedBuiltinCode.TRANSFORM);
 		String2ParameterizedBuiltinCode.put( "transformapply", ParameterizedBuiltinCode.TRANSFORMAPPLY);
 		String2ParameterizedBuiltinCode.put( "transformdecode", ParameterizedBuiltinCode.TRANSFORMDECODE);
 	}
@@ -169,9 +168,6 @@ public class ParameterizedBuiltin extends ValueFunction
 			case REXPAND:
 				return new ParameterizedBuiltin(ParameterizedBuiltinCode.REXPAND);
 			
-			case TRANSFORM:
-				return new ParameterizedBuiltin(ParameterizedBuiltinCode.TRANSFORM);
-			
 			case TRANSFORMAPPLY:
 				return new ParameterizedBuiltin(ParameterizedBuiltinCode.TRANSFORMAPPLY);
 			

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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 52c11e6..9126ab7 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
@@ -191,7 +191,6 @@ public class CPInstructionParser extends InstructionParser
 		String2CPInstructionType.put( "rmempty"	    , CPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2CPInstructionType.put( "replace"	    , CPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2CPInstructionType.put( "rexpand"	    , CPINSTRUCTION_TYPE.ParameterizedBuiltin);
-		String2CPInstructionType.put( "transform"	, CPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2CPInstructionType.put( "transformapply",CPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2CPInstructionType.put( "transformdecode",CPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2CPInstructionType.put( "transformencode",CPINSTRUCTION_TYPE.MultiReturnParameterizedBuiltin);

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java b/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
index 0b9cb7d..5ced2ab 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
@@ -288,7 +288,6 @@ public class MRInstructionParser extends InstructionParser
 		//dummy (pseudo instructions)
 		String2MRInstructionType.put( "sort", MRINSTRUCTION_TYPE.Sort);
 		String2MRInstructionType.put( "csvwrite", MRINSTRUCTION_TYPE.CSVWrite);
-		String2MRInstructionType.put( "transform", MRINSTRUCTION_TYPE.Transform);
 		
 		//parameterized builtins
 		String2MRInstructionType.put( "replace", MRINSTRUCTION_TYPE.ParameterizedBuiltin);

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java b/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java
index 5ca3847..a2b0ef1 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java
@@ -230,7 +230,6 @@ public class SPInstructionParser extends InstructionParser
 		String2SPInstructionType.put( "rmempty"	     , SPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2SPInstructionType.put( "replace"	     , SPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2SPInstructionType.put( "rexpand"	     , SPINSTRUCTION_TYPE.ParameterizedBuiltin);
-		String2SPInstructionType.put( "transform"    , SPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2SPInstructionType.put( "transformapply",SPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2SPInstructionType.put( "transformdecode",SPINSTRUCTION_TYPE.ParameterizedBuiltin);
 		String2SPInstructionType.put( "transformencode",SPINSTRUCTION_TYPE.MultiReturnBuiltin);

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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 a31fe94..847d5f9 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
@@ -33,12 +33,10 @@ import org.apache.sysml.runtime.functionobjects.ParameterizedBuiltin;
 import org.apache.sysml.runtime.functionobjects.ValueFunction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.mr.GroupedAggregateInstruction;
-import org.apache.sysml.runtime.matrix.JobReturn;
 import org.apache.sysml.runtime.matrix.data.FrameBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
-import org.apache.sysml.runtime.transform.DataTransform;
 import org.apache.sysml.runtime.transform.TfUtils;
 import org.apache.sysml.runtime.transform.decode.Decoder;
 import org.apache.sysml.runtime.transform.decode.DecoderFactory;
@@ -141,8 +139,7 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 			func = ParameterizedBuiltin.getParameterizedBuiltinFnObject(opcode);
 			return new ParameterizedBuiltinCPInstruction(new SimpleOperator(func), paramsMap, out, opcode, str);
 		}
-		else if (   opcode.equals("transform")
-				 || opcode.equals("transformapply")
+		else if (   opcode.equals("transformapply")
 				 || opcode.equals("transformdecode")
 				 || opcode.equals("transformmeta")) 
 		{
@@ -255,16 +252,6 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 			ec.setMatrixOutput(output.getName(), ret);
 			ec.releaseMatrixInput(params.get("target"));
 		}
-		else if ( opcode.equalsIgnoreCase("transform")) {
-			FrameObject fo = ec.getFrameObject(params.get("target"));
-			MatrixObject out = ec.getMatrixObject(output.getName());			
-			try {
-				JobReturn jt = DataTransform.cpDataTransform(this, new FrameObject[] { fo } , new MatrixObject[] {out} );
-				out.updateMatrixCharacteristics(jt.getMatrixCharacteristics(0));
-			} catch (Exception e) {
-				throw new DMLRuntimeException(e);
-			}
-		}
 		else if ( opcode.equalsIgnoreCase("transformapply")) {
 			//acquire locks
 			FrameBlock data = ec.getFrameInput(params.get("target"));

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
index 68132fc..ad98ad7 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
@@ -32,7 +32,7 @@ public abstract class MRInstruction extends Instruction
 {
 	
 	public enum MRINSTRUCTION_TYPE { INVALID, Append, Aggregate, ArithmeticBinary, ArithmeticBinary2, AggregateBinary, AggregateUnary, 
-		Rand, Seq, CSVReblock, CSVWrite, Transform,
+		Rand, Seq, CSVReblock, CSVWrite, 
 		Reblock, Reorg, Replicate, Unary, CombineBinary, CombineUnary, CombineTernary, PickByCount, Partition,
 		Ternary, Quaternary, CM_N_COV, MapGroupedAggregate, GroupedAggregate, RangeReIndex, ZeroOut, MMTSJ, PMMJ, MatrixReshape, ParameterizedBuiltin, Sort, MapMultChain,
 		CumsumAggregate, CumsumSplit, CumsumOffset, BinUaggChain, UaggOuterChain, RemoveEmpty}; 

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/instructions/spark/MultiReturnParameterizedBuiltinSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/MultiReturnParameterizedBuiltinSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/MultiReturnParameterizedBuiltinSPInstruction.java
index 5890bf9..254c3d7 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/MultiReturnParameterizedBuiltinSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/MultiReturnParameterizedBuiltinSPInstruction.java
@@ -56,12 +56,12 @@ import org.apache.sysml.runtime.matrix.data.InputInfo;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysml.runtime.matrix.operators.Operator;
-import org.apache.sysml.runtime.transform.MVImputeAgent;
-import org.apache.sysml.runtime.transform.MVImputeAgent.MVMethod;
-import org.apache.sysml.runtime.transform.RecodeAgent;
 import org.apache.sysml.runtime.transform.encode.Encoder;
 import org.apache.sysml.runtime.transform.encode.EncoderComposite;
 import org.apache.sysml.runtime.transform.encode.EncoderFactory;
+import org.apache.sysml.runtime.transform.encode.EncoderMVImpute;
+import org.apache.sysml.runtime.transform.encode.EncoderRecode;
+import org.apache.sysml.runtime.transform.encode.EncoderMVImpute.MVMethod;
 import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
 import org.apache.sysml.runtime.transform.meta.TfOffsetMap;
 
@@ -129,7 +129,7 @@ public class MultiReturnParameterizedBuiltinSPInstruction extends ComputationSPI
 					.distinct().groupByKey()
 					.flatMap(new TransformEncodeGroupFunction(accMax));
 			if( containsMVImputeEncoder(encoderBuild) ) {
-				MVImputeAgent mva = getMVImputeEncoder(encoderBuild);
+				EncoderMVImpute mva = getMVImputeEncoder(encoderBuild);
 				rcMaps = rcMaps.union(
 						in.mapPartitionsToPair(new TransformEncodeBuild2Function(mva))
 						  .groupByKey().flatMap(new TransformEncodeGroup2Function(mva)) );
@@ -149,7 +149,7 @@ public class MultiReturnParameterizedBuiltinSPInstruction extends ComputationSPI
 				omap = new TfOffsetMap(SparkUtils.toIndexedLong(in.mapToPair(
 					new RDDTransformApplyOffsetFunction(spec, colnames)).collect()));
 			}
-				
+			
 			//create encoder broadcast (avoiding replication per task) 
 			Encoder encoder = EncoderFactory.createEncoder(spec, colnames,
 					fo.getSchema(), (int)fo.getNumColumns(), meta);
@@ -176,16 +176,16 @@ public class MultiReturnParameterizedBuiltinSPInstruction extends ComputationSPI
 	private boolean containsMVImputeEncoder(Encoder encoder) {
 		if( encoder instanceof EncoderComposite )
 			for( Encoder cencoder : ((EncoderComposite)encoder).getEncoders() )
-				if( cencoder instanceof MVImputeAgent )
+				if( cencoder instanceof EncoderMVImpute )
 					return true;
 		return false;	
 	}
 
-	private MVImputeAgent getMVImputeEncoder(Encoder encoder) {
+	private EncoderMVImpute getMVImputeEncoder(Encoder encoder) {
 		if( encoder instanceof EncoderComposite )
 			for( Encoder cencoder : ((EncoderComposite)encoder).getEncoders() )
-				if( cencoder instanceof MVImputeAgent )
-					return (MVImputeAgent) cencoder;
+				if( cencoder instanceof EncoderMVImpute )
+					return (EncoderMVImpute) cencoder;
 		return null;	
 	}
 	
@@ -248,12 +248,12 @@ public class MultiReturnParameterizedBuiltinSPInstruction extends ComputationSPI
 	{
 		private static final long serialVersionUID = 6336375833412029279L;
 
-		private RecodeAgent _raEncoder = null;
+		private EncoderRecode _raEncoder = null;
 		
 		public TransformEncodeBuildFunction(Encoder encoder) {
 			for( Encoder cEncoder : ((EncoderComposite)encoder).getEncoders() )
-				if( cEncoder instanceof RecodeAgent )
-					_raEncoder = (RecodeAgent)cEncoder;
+				if( cEncoder instanceof EncoderRecode )
+					_raEncoder = (EncoderRecode)cEncoder;
 		}
 		
 		@Override
@@ -310,7 +310,7 @@ public class MultiReturnParameterizedBuiltinSPInstruction extends ComputationSPI
 				sb.append(' ');
 				sb.append(colID);
 				sb.append(' ');
-				sb.append(RecodeAgent.constructRecodeMapEntry(
+				sb.append(EncoderRecode.constructRecodeMapEntry(
 						iter.next().toString(), rowID));
 				ret.add(sb.toString());
 				sb.setLength(0); 
@@ -326,9 +326,9 @@ public class MultiReturnParameterizedBuiltinSPInstruction extends ComputationSPI
 	{
 		private static final long serialVersionUID = 6336375833412029279L;
 
-		private MVImputeAgent _encoder = null;
+		private EncoderMVImpute _encoder = null;
 		
-		public TransformEncodeBuild2Function(MVImputeAgent encoder) {
+		public TransformEncodeBuild2Function(EncoderMVImpute encoder) {
 			_encoder = encoder;
 		}
 		
@@ -370,9 +370,9 @@ public class MultiReturnParameterizedBuiltinSPInstruction extends ComputationSPI
 	{
 		private static final long serialVersionUID = 702100641492347459L;
 		
-		private MVImputeAgent _encoder = null;
+		private EncoderMVImpute _encoder = null;
 		
-		public TransformEncodeGroup2Function(MVImputeAgent encoder) {	
+		public TransformEncodeGroup2Function(EncoderMVImpute encoder) {	
 			_encoder = encoder;
 		}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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 2a80cfc..179ef9e 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
@@ -38,7 +38,6 @@ import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.parser.Statement;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.caching.FrameObject;
-import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject.UpdateType;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
@@ -73,7 +72,6 @@ import org.apache.sysml.runtime.matrix.operators.CMOperator;
 import org.apache.sysml.runtime.matrix.operators.CMOperator.AggregateOperationTypes;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
-import org.apache.sysml.runtime.transform.DataTransform;
 import org.apache.sysml.runtime.transform.TfUtils;
 import org.apache.sysml.runtime.transform.decode.Decoder;
 import org.apache.sysml.runtime.transform.decode.DecoderFactory;
@@ -173,7 +171,6 @@ public class ParameterizedBuiltinSPInstruction  extends ComputationSPInstruction
 			}
 			else if(   opcode.equalsIgnoreCase("rexpand") 
 					|| opcode.equalsIgnoreCase("replace")
-					|| opcode.equalsIgnoreCase("transform")
 					|| opcode.equalsIgnoreCase("transformapply")
 					|| opcode.equalsIgnoreCase("transformdecode")) 
 			{
@@ -432,17 +429,6 @@ public class ParameterizedBuiltinSPInstruction  extends ComputationSPInstruction
 			MatrixCharacteristics mcOut = sec.getMatrixCharacteristics(output.getName());
 			mcOut.set(dirRows?lmaxVal:mcIn.getRows(), dirRows?mcIn.getRows():lmaxVal, (int)brlen, (int)bclen, -1);
 		}
-		else if ( opcode.equalsIgnoreCase("transform") ) 
-		{
-			// perform data transform on Spark
-			try {
-				DataTransform.spDataTransform( this, 
-						new FrameObject[] { sec.getFrameObject(params.get("target")) }, 
-						new MatrixObject[] { sec.getMatrixObject(output.getName()) }, ec);
-			} catch (Exception e) {
-				throw new DMLRuntimeException(e);
-			}
-		}
 		else if ( opcode.equalsIgnoreCase("transformapply") ) 
 		{
 			//get input RDD and meta data

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java
index c30c85f..8bfee87 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java
@@ -56,10 +56,6 @@ public class WriteSPInstruction extends SPInstruction
 	private CPOperand input3 = null;
 	private CPOperand input4 = null;
 	private FileFormatProperties formatProperties;
-	
-	//scalars might occur for transform
-	// TODO remove once transform over frames supported
-	private boolean isInputMatrixBlock = true; 
 
 	public WriteSPInstruction(CPOperand in1, CPOperand in2, CPOperand in3, String opcode, String str) {
 		super(opcode, str);
@@ -100,10 +96,6 @@ public class WriteSPInstruction extends SPInstruction
 			boolean sparse = Boolean.parseBoolean(parts[6]);
 			FileFormatProperties formatProperties = new CSVFileFormatProperties(hasHeader, delim, sparse);
 			inst.setFormatProperties(formatProperties);
-			
-			boolean isInputMB = Boolean.parseBoolean(parts[7]);
-			inst.setInputMatrixBlock(isInputMB);
-
 			CPOperand in4 = new CPOperand(parts[8]);
 			inst.input4 = in4;
 		} else {
@@ -125,14 +117,6 @@ public class WriteSPInstruction extends SPInstruction
 		formatProperties = prop;
 	}
 	
-	public void setInputMatrixBlock(boolean isMB) {
-		isInputMatrixBlock = isMB;
-	}
-	
-	public boolean isInputMatrixBlock() {
-		return isInputMatrixBlock;
-	}
-	
 	@Override
 	public void processInstruction(ExecutionContext ec)
 		throws DMLRuntimeException 
@@ -180,7 +164,7 @@ public class WriteSPInstruction extends SPInstruction
 		{
 			//piggyback nnz maintenance on write
 			LongAccumulator aNnz = null;
-			if ( isInputMatrixBlock && !mc.nnzKnown() ) {
+			if( !mc.nnzKnown() ) {
 				aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
 				in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
 			}
@@ -202,57 +186,25 @@ public class WriteSPInstruction extends SPInstruction
 			else
 				customSaveTextFile(ijv, fname, false);
 			
-			if ( isInputMatrixBlock && !mc.nnzKnown() )
+			if( !mc.nnzKnown() )
 				mc.setNonZeros( aNnz.value() );
 		}
 		else if( oi == OutputInfo.CSVOutputInfo ) 
 		{
-			JavaRDD<String> out = null;
 			LongAccumulator aNnz = null;
 			
-			if ( isInputMatrixBlock ) {
-				//piggyback nnz computation on actual write
-				if( !mc.nnzKnown() ) {
-					aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
-					in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
-				}	
-				
-				out = RDDConverterUtils.binaryBlockToCsv(in1, mc, 
-						(CSVFileFormatProperties) formatProperties, true);
-			}
-			else 
-			{
-				// This case is applicable when the CSV output from transform() is written out
-				// TODO remove once transform over frames supported
-				@SuppressWarnings("unchecked")
-				JavaPairRDD<Long,String> rdd = (JavaPairRDD<Long, String>) (sec.getMatrixObject(input1.getName())).getRDDHandle().getRDD();
-				out = rdd.values(); 
-
-				String sep = ",";
-				boolean hasHeader = false;
-				if(formatProperties != null) {
-					sep = ((CSVFileFormatProperties) formatProperties).getDelim();
-					hasHeader = ((CSVFileFormatProperties) formatProperties).hasHeader();
-				}
-				
-				if(hasHeader) {
-					StringBuffer buf = new StringBuffer();
-		    		for(int j = 1; j < mc.getCols(); j++) {
-		    			if(j != 1) {
-		    				buf.append(sep);
-		    			}
-		    			buf.append("C" + j);
-		    		}
-		    		ArrayList<String> headerContainer = new ArrayList<String>(1);
-		    		headerContainer.add(0, buf.toString());
-		    		JavaRDD<String> header = sec.getSparkContext().parallelize(headerContainer);
-		    		out = header.union(out);
-				}
-			}
+			//piggyback nnz computation on actual write
+			if( !mc.nnzKnown() ) {
+				aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
+				in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
+			}	
 			
+			JavaRDD<String> out = RDDConverterUtils.binaryBlockToCsv(
+				in1, mc, (CSVFileFormatProperties) formatProperties, true);
+
 			customSaveTextFile(out, fname, false);
 			
-			if( isInputMatrixBlock && !mc.nnzKnown() )
+			if( !mc.nnzKnown() )
 				mc.setNonZeros((long)aNnz.value().longValue());
 		}
 		else if( oi == OutputInfo.BinaryBlockOutputInfo ) {

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/matrix/CSVReblockMR.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/CSVReblockMR.java b/src/main/java/org/apache/sysml/runtime/matrix/CSVReblockMR.java
index b976545..3bd9758 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/CSVReblockMR.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/CSVReblockMR.java
@@ -55,7 +55,6 @@ import org.apache.sysml.runtime.matrix.mapred.MRConfigurationNames;
 import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration;
 import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration.ConvertTarget;
 import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration.MatrixChar_N_ReducerGroups;
-import org.apache.sysml.runtime.transform.TfUtils;
 import org.apache.sysml.runtime.util.MapReduceTool;
 
 
@@ -281,17 +280,9 @@ public class CSVReblockMR
 				otherInstructionsInReducer, numReducers, replication, resultIndexes, outputs, outputInfos, ret1.counterFile, smallestFiles);
 		return ret;
 	}
-	
-	public static AssignRowIDMRReturn runAssignRowIDMRJob(String[] inputs, InputInfo[] inputInfos, int[] brlens, int[] bclens, 
-			String reblockInstructions, int replication, String[] smallestFiles) 
-	throws Exception
-	{
-		return runAssignRowIDMRJob(inputs, inputInfos, brlens, bclens, reblockInstructions, replication, smallestFiles, false, null, null);
-	}
-
 		
 	public static AssignRowIDMRReturn runAssignRowIDMRJob(String[] inputs, InputInfo[] inputInfos, int[] brlens, int[] bclens, 
-			String reblockInstructions, int replication, String[] smallestFiles, boolean transform, String naStrings, String spec) 
+			String reblockInstructions, int replication, String[] smallestFiles) 
 	throws Exception
 	{
 		AssignRowIDMRReturn ret=new AssignRowIDMRReturn();
@@ -347,16 +338,6 @@ public class CSVReblockMR
 		job.setOutputKeyClass(ByteWritable.class);
 		job.setOutputValueClass(OffsetCount.class);
 		
-		// setup properties relevant to transform
-		job.setBoolean(MRJobConfiguration.TF_TRANSFORM, transform);
-		if (transform)
-		{
-			if ( naStrings != null)
-				// Adding "dummy" string to handle the case of na_strings = ""
-				job.set(MRJobConfiguration.TF_NA_STRINGS, TfUtils.prepNAStrings(naStrings) );
-			job.set(MRJobConfiguration.TF_SPEC, spec);
-		}
-		
 		RunningJob runjob=JobClient.runJob(job);
 		
 		/* Process different counters */

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/matrix/mapred/CSVAssignRowIDMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/mapred/CSVAssignRowIDMapper.java b/src/main/java/org/apache/sysml/runtime/matrix/mapred/CSVAssignRowIDMapper.java
index 85b55cb..4f9877b 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/mapred/CSVAssignRowIDMapper.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/mapred/CSVAssignRowIDMapper.java
@@ -36,7 +36,6 @@ import org.apache.sysml.runtime.instructions.mr.CSVReblockInstruction;
 import org.apache.sysml.runtime.io.IOUtilFunctions;
 import org.apache.sysml.runtime.matrix.CSVReblockMR;
 import org.apache.sysml.runtime.matrix.CSVReblockMR.OffsetCount;
-import org.apache.sysml.runtime.transform.TfUtils;
 
 public class CSVAssignRowIDMapper extends MapReduceBase implements Mapper<LongWritable, Text, ByteWritable, OffsetCount>
 {	
@@ -51,9 +50,6 @@ public class CSVAssignRowIDMapper extends MapReduceBase implements Mapper<LongWr
 	private String filename = "";
 	private boolean headerFile = false;
 	
-	// members relevant to transform
-	private TfUtils _agents = null;
-	
 	@Override
 	public void map(LongWritable key, Text value,
 			OutputCollector<ByteWritable, OffsetCount> out, Reporter report)
@@ -69,7 +65,7 @@ public class CSVAssignRowIDMapper extends MapReduceBase implements Mapper<LongWr
 		if(key.get()==0 && headerFile) {
 			if(!ignoreFirstLine) {
 				report.incrCounter(CSVReblockMR.NUM_COLS_IN_MATRIX, outKey.toString(), value.toString().split(delim, -1).length);
-				num += omit(value.toString()) ? 0 : 1;
+				num++;
 			}
 			else
 				realFirstLine = true;
@@ -79,7 +75,7 @@ public class CSVAssignRowIDMapper extends MapReduceBase implements Mapper<LongWr
 				report.incrCounter(CSVReblockMR.NUM_COLS_IN_MATRIX, outKey.toString(), value.toString().split(delim, -1).length);
 				realFirstLine = false;
 			}
-			num += omit(value.toString()) ? 0 : 1;
+			num++;
 		}
 	}
 	
@@ -107,23 +103,12 @@ public class CSVAssignRowIDMapper extends MapReduceBase implements Mapper<LongWr
 					ignoreFirstLine = ins.hasHeader;
 					break;
 				}
-		
-			// load properties relevant to transform
-			boolean omit = job.getBoolean(MRJobConfiguration.TF_TRANSFORM, false);
-			if ( omit ) 
-				_agents = new TfUtils(job, true);
 		}
 		catch(Exception e) {
 			throw new RuntimeException(e);
 		}
 	}
 	
-	private boolean omit(String line) {
-		if(_agents == null)
-			return false;		
-		return _agents.omit( line.split(delim, -1) );
-	}
-	
 	@Override
 	public void close() throws IOException {
 		if( outCache != null ) //robustness empty splits

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java b/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java
index 50a7412..88512b6 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java
@@ -231,33 +231,6 @@ public class MRJobConfiguration
 	 */
 	public static final String NUM_NONZERO_CELLS="nonzeros";
 
-	public static final String TF_NUM_COLS 		= "transform.num.columns";
-	public static final String TF_HAS_HEADER 	= "transform.has.header";
-	public static final String TF_DELIM 		= "transform.field.delimiter";
-	public static final String TF_NA_STRINGS 	= "transform.na.strings";
-	public static final String TF_HEADER		= "transform.header.line";
-	public static final String TF_SPEC 	        = "transform.specification";
-	public static final String TF_TMP_LOC    	= "transform.temp.location";
-	public static final String TF_TRANSFORM     = "transform.omit.na.rows";
-	
-	public static final String TF_SMALLEST_FILE= "transform.smallest.file";
-	public static final String TF_OFFSETS_FILE = "transform.offsets.file";
-	public static final String TF_TXMTD_PATH   = "transform.txmtd.path";
-	
-	/*public static enum DataTransformJobProperty 
-	{
-		RCD_NUM_COLS("recode.num.columns");
-		
-		private final String name;
-		private DataTransformJobProperty(String n) {
-			name = n;
-		}
-	}*/
-	
-	public static enum DataTransformCounters { 
-		TRANSFORMED_NUM_ROWS
-	};
-	
 	public static final int getMiscMemRequired(JobConf job)
 	{
 		return job.getInt(MRConfigurationNames.IO_FILE_BUFFER_SIZE, 4096);

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/ApplyTfBBMR.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfBBMR.java b/src/main/java/org/apache/sysml/runtime/transform/ApplyTfBBMR.java
deleted file mode 100644
index e932d1e..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfBBMR.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * 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.runtime.transform;
-import java.util.HashSet;
-
-import org.apache.hadoop.filecache.DistributedCache;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.Counters.Group;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.sysml.conf.ConfigurationManager;
-import org.apache.sysml.runtime.instructions.InstructionParser;
-import org.apache.sysml.runtime.instructions.mr.CSVReblockInstruction;
-import org.apache.sysml.runtime.matrix.CSVReblockMR;
-import org.apache.sysml.runtime.matrix.CSVReblockMR.BlockRow;
-import org.apache.sysml.runtime.matrix.JobReturn;
-import org.apache.sysml.runtime.matrix.WriteCSVMR;
-import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
-import org.apache.sysml.runtime.matrix.data.InputInfo;
-import org.apache.sysml.runtime.matrix.data.OutputInfo;
-import org.apache.sysml.runtime.matrix.data.TaggedFirstSecondIndexes;
-import org.apache.sysml.runtime.matrix.mapred.CSVReblockReducer;
-import org.apache.sysml.runtime.matrix.mapred.MRConfigurationNames;
-import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration;
-import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration.ConvertTarget;
-import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration.MatrixChar_N_ReducerGroups;
-import org.apache.sysml.runtime.util.MapReduceTool;
-
-/**
- * MapReduce job that performs the actual data transformations, such as recoding
- * and binning. In contrast to ApplyTxCSVMR, this job generates the output in
- * BinaryBlock format. This job takes a data set as well as the transformation
- * metadata (which, for example, computed from GenTxMtdMR) as inputs.
- * 
- */
-
-@SuppressWarnings("deprecation")
-public class ApplyTfBBMR {
-	
-	public static JobReturn runJob(String inputPath, String rblkInst, String otherInst, String spec, String mapsPath, String tmpPath, String outputPath, String partOffsetsFile, CSVFileFormatProperties inputDataProperties, long numRows, long numColsBefore, long numColsAfter, int replication, String headerLine) throws Exception {
-		
-		CSVReblockInstruction rblk = (CSVReblockInstruction) InstructionParser.parseSingleInstruction(rblkInst);
-		
-		long[] rlens = new long[]{numRows};
-		long[] clens = new long[]{numColsAfter};
-		int[] brlens = new int[]{rblk.brlen};
-		int[] bclens = new int[]{rblk.bclen};
-		byte[] realIndexes = new byte[]{rblk.input};
-		byte[] resultIndexes = new byte[]{rblk.output};
-
-		JobConf job = new JobConf(ApplyTfBBMR.class);
-		job.setJobName("ApplyTfBB");
-
-		/* Setup MapReduce Job */
-		job.setJarByClass(ApplyTfBBMR.class);
-		
-		// set relevant classes
-		job.setMapperClass(ApplyTfBBMapper.class);
-	
-		MRJobConfiguration.setUpMultipleInputs(job, realIndexes, new String[]{inputPath}, new InputInfo[]{InputInfo.CSVInputInfo}, brlens, bclens, false, ConvertTarget.CELL);
-
-		MRJobConfiguration.setMatricesDimensions(job, realIndexes, rlens, clens);
-		MRJobConfiguration.setBlocksSizes(job, realIndexes, brlens, bclens);
-
-		MRJobConfiguration.setCSVReblockInstructions(job, rblkInst);
-		
-		//set up the instructions that will happen in the reducer, after the aggregation instrucions
-		MRJobConfiguration.setInstructionsInReducer(job, otherInst);
-
-		job.setInt(MRConfigurationNames.DFS_REPLICATION, replication);
-		
-		//set up preferred custom serialization framework for binary block format
-		if( MRJobConfiguration.USE_BINARYBLOCK_SERIALIZATION )
-			MRJobConfiguration.addBinaryBlockSerializationFramework( job );
-
-		//set up what matrices are needed to pass from the mapper to reducer
-		HashSet<Byte> mapoutputIndexes=MRJobConfiguration.setUpOutputIndexesForMapper(job, realIndexes,  null, 
-				rblkInst, null, otherInst, resultIndexes);
-
-		MatrixChar_N_ReducerGroups ret=MRJobConfiguration.computeMatrixCharacteristics(job, realIndexes, 
-				null, rblkInst, null, null, null, resultIndexes, mapoutputIndexes, false);
-
-		//set up the number of reducers
-		int numRed = WriteCSVMR.determineNumReducers(rlens, clens, ConfigurationManager.getNumReducers(), ret.numReducerGroups);
-		job.setNumReduceTasks( numRed );
-
-		//set up the multiple output files, and their format information
-		MRJobConfiguration.setUpMultipleOutputs(job, new byte[]{rblk.output}, new byte[]{0}, new String[]{outputPath}, new OutputInfo[]{OutputInfo.BinaryBlockOutputInfo}, true, false);
-		
-		// configure mapper and the mapper output key value pairs
-		job.setMapperClass(ApplyTfBBMapper.class);
-		job.setMapOutputKeyClass(TaggedFirstSecondIndexes.class);
-		job.setMapOutputValueClass(BlockRow.class);
-		
-		//configure reducer
-		job.setReducerClass(CSVReblockReducer.class);
-	
-		//turn off adaptivemr
-		job.setBoolean("adaptivemr.map.enable", false);
-
-		//set unique working dir
-		MRJobConfiguration.setUniqueWorkingDir(job);
-		
-		// Add transformation metadata file as well as partOffsetsFile to Distributed cache
-		DistributedCache.addCacheFile((new Path(mapsPath)).toUri(), job);
-		DistributedCache.createSymlink(job);
-		
-		Path cachefile=new Path(new Path(partOffsetsFile), "part-00000");
-		DistributedCache.addCacheFile(cachefile.toUri(), job);
-		DistributedCache.createSymlink(job);
-		
-		job.set(MRJobConfiguration.TF_HAS_HEADER, 	Boolean.toString(inputDataProperties.hasHeader()));
-		job.set(MRJobConfiguration.TF_DELIM, 		inputDataProperties.getDelim());
-		// Adding "dummy" string to handle the case of na_strings = ""
-		if( inputDataProperties.getNAStrings() != null )
-			job.set(MRJobConfiguration.TF_NA_STRINGS, TfUtils.prepNAStrings(inputDataProperties.getNAStrings()) );
-		job.set(MRJobConfiguration.TF_SPEC, spec);
-		job.set(MRJobConfiguration.TF_SMALLEST_FILE, CSVReblockMR.findSmallestFile(job, inputPath));
-		job.set(MRJobConfiguration.OUTPUT_MATRICES_DIRS_CONFIG, outputPath);
-		job.setLong(MRJobConfiguration.TF_NUM_COLS, numColsBefore);
-		job.set(MRJobConfiguration.TF_TXMTD_PATH, mapsPath);
-		job.set(MRJobConfiguration.TF_HEADER, headerLine);
-		job.set(CSVReblockMR.ROWID_FILE_NAME, cachefile.toString());
-		job.set(MRJobConfiguration.TF_TMP_LOC, tmpPath);
-
-		RunningJob runjob=JobClient.runJob(job);
-		
-		MapReduceTool.deleteFileIfExistOnHDFS(cachefile, job);
-		
-		Group group=runjob.getCounters().getGroup(MRJobConfiguration.NUM_NONZERO_CELLS);
-		for(int i=0; i<resultIndexes.length; i++) {
-			ret.stats[i].setNonZeros(group.getCounter(Integer.toString(i)));
-		}
-		return new JobReturn(ret.stats, runjob.isSuccessful());
-	}
-	
-}


[5/6] systemml git commit: [SYSTEMML-1300] Remove file-based transform from compiler/runtime

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/ApplyTfBBMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfBBMapper.java b/src/main/java/org/apache/sysml/runtime/transform/ApplyTfBBMapper.java
deleted file mode 100644
index 77c06ae..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfBBMapper.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * 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.runtime.transform;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.ByteWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.wink.json4j.JSONException;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.instructions.mr.CSVReblockInstruction;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.matrix.CSVReblockMR;
-import org.apache.sysml.runtime.matrix.CSVReblockMR.OffsetCount;
-import org.apache.sysml.runtime.matrix.data.TaggedFirstSecondIndexes;
-import org.apache.sysml.runtime.matrix.mapred.CSVReblockMapper;
-import org.apache.sysml.runtime.matrix.mapred.MRConfigurationNames;
-import org.apache.sysml.runtime.matrix.mapred.CSVReblockMapper.IndexedBlockRow;
-import org.apache.sysml.runtime.matrix.mapred.MapperBase;
-
-@SuppressWarnings("deprecation")
-public class ApplyTfBBMapper extends MapperBase implements Mapper<LongWritable, Text, TaggedFirstSecondIndexes, CSVReblockMR.BlockRow>{
-	
-	boolean _partFileWithHeader = false;
-	TfUtils tfmapper = null;
-	Reporter _reporter = null;
-	
-	// variables relevant to CSV Reblock
-	private IndexedBlockRow idxRow = null;
-	private long rowOffset=0;
-	private HashMap<Long, Long> offsetMap=new HashMap<Long, Long>();
-	private boolean _first = true;
-	private long num=0;
-	
-	@Override
-	public void configure(JobConf job) {
-		super.configure(job);
-		try {
-			_partFileWithHeader = TfUtils.isPartFileWithHeader(job);
-			tfmapper = new TfUtils(job);
-			tfmapper.loadTfMetadata(job, true);
-			
-			// Load relevant information for CSV Reblock
-			ByteWritable key=new ByteWritable();
-			OffsetCount value=new OffsetCount();
-			Path p=new Path(job.get(CSVReblockMR.ROWID_FILE_NAME));
-			
-			Path path=new Path(job.get(MRConfigurationNames.MR_MAP_INPUT_FILE));
-			FileSystem fs = IOUtilFunctions.getFileSystem(path, job);
-			String thisfile=path.makeQualified(fs).toString();
-
-			SequenceFile.Reader reader = null;
-			try {
-				reader = new SequenceFile.Reader(fs, p, job);
-				while (reader.next(key, value)) {
-					// "key" needn't be checked since the offset file has information about a single CSV input (the raw data file)
-					if(thisfile.equals(value.filename))
-						offsetMap.put(value.fileOffset, value.count);
-				}
-			}
-			finally {
-				IOUtilFunctions.closeSilently(reader);
-			}
-			
-			idxRow = new CSVReblockMapper.IndexedBlockRow();
-			int maxBclen=0;
-		
-			for(ArrayList<CSVReblockInstruction> insv: csv_reblock_instructions)
-				for(CSVReblockInstruction in: insv)
-				{	
-					if(maxBclen<in.bclen)
-						maxBclen=in.bclen;
-				}
-			
-			//always dense since common csv usecase
-			idxRow.getRow().data.reset(1, maxBclen, false);		
-
-		} catch (IOException e) { throw new RuntimeException(e); }
- 		 catch(JSONException e)  { throw new RuntimeException(e); }
-
-	}
-	
-	@Override
-	public void map(LongWritable rawKey, Text rawValue, OutputCollector<TaggedFirstSecondIndexes,CSVReblockMR.BlockRow> out, Reporter reporter) throws IOException  {
-		
-		if(_first) {
-			rowOffset=offsetMap.get(rawKey.get());
-			_reporter = reporter;
-			_first=false;
-		}
-		
-		// output the header line
-		if ( rawKey.get() == 0 && _partFileWithHeader ) 
-		{
-			tfmapper.processHeaderLine();
-			if ( tfmapper.hasHeader() )
-				return;
-		}
-		
-		// parse the input line and apply transformation
-		String[] words = tfmapper.getWords(rawValue);
-		
-		if(!tfmapper.omit(words))
-		{
-			words = tfmapper.apply(words);
-			try {
-				tfmapper.check(words);
-				
-				// Perform CSV Reblock
-				CSVReblockInstruction ins = csv_reblock_instructions.get(0).get(0);
-				idxRow = CSVReblockMapper.processRow(idxRow, words, rowOffset, num, ins.output, ins.brlen, ins.bclen, ins.fill, ins.fillValue, out);
-			}
-			catch(DMLRuntimeException e) {
-				throw new RuntimeException(e.getMessage() + ":" + rawValue.toString());
-			}
-			num++;
-		}
-	}
-
-	@Override
-	public void close() throws IOException {
-	}
-
-	@Override
-	protected void specialOperationsForActualMap(int index,
-			OutputCollector<Writable, Writable> out, Reporter reporter)
-			throws IOException {
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVMR.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVMR.java b/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVMR.java
deleted file mode 100644
index e2885d8..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVMR.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * 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.runtime.transform;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-
-import org.apache.hadoop.filecache.DistributedCache;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.matrix.CSVReblockMR;
-import org.apache.sysml.runtime.matrix.JobReturn;
-import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
-import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
-import org.apache.sysml.runtime.matrix.mapred.MRConfigurationNames;
-import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration;
-
-
-@SuppressWarnings("deprecation")
-public class ApplyTfCSVMR {
-	
-	public static JobReturn runJob(String inputPath, String spec, String mapsPath, String tmpPath, String outputPath, String partOffsetsFile, CSVFileFormatProperties inputDataProperties, long numCols, int replication, String headerLine) throws IOException, ClassNotFoundException, InterruptedException {
-		JobConf job = new JobConf(ApplyTfCSVMR.class);
-		job.setJobName("ApplyTfCSV");
-
-		/* Setup MapReduce Job */
-		job.setJarByClass(ApplyTfCSVMR.class);
-		
-		// set relevant classes
-		job.setMapperClass(ApplyTfCSVMapper.class);
-		job.setNumReduceTasks(0);
-	
-		// Add transformation metadata file as well as partOffsetsFile to Distributed cache
-		DistributedCache.addCacheFile((new Path(mapsPath)).toUri(), job);
-		DistributedCache.createSymlink(job);
-		
-		Path cachefile=new Path(partOffsetsFile);
-		DistributedCache.addCacheFile(cachefile.toUri(), job);
-		DistributedCache.createSymlink(job);
-		
-		// set input and output properties
-		job.setInputFormat(TextInputFormat.class);
-		job.setOutputFormat(TextOutputFormat.class);
-		
-		job.setMapOutputKeyClass(NullWritable.class);
-		job.setMapOutputValueClass(Text.class);
-		
-		job.setOutputKeyClass(NullWritable.class);
-		job.setOutputValueClass(Text.class);
-		
-		job.setInt(MRConfigurationNames.DFS_REPLICATION, replication);
-		
-		FileInputFormat.addInputPath(job, new Path(inputPath));
-		// delete outputPath, if exists already.
-		Path outPath = new Path(outputPath);
-		FileSystem fs = IOUtilFunctions.getFileSystem(outPath, job);
-		fs.delete(outPath, true);
-		FileOutputFormat.setOutputPath(job, outPath);
-
-		job.set(MRJobConfiguration.TF_HAS_HEADER, 	Boolean.toString(inputDataProperties.hasHeader()));
-		job.set(MRJobConfiguration.TF_DELIM, 		inputDataProperties.getDelim());
-		if ( inputDataProperties.getNAStrings() != null)
-			// Adding "dummy" string to handle the case of na_strings = ""
-			job.set(MRJobConfiguration.TF_NA_STRINGS, TfUtils.prepNAStrings(inputDataProperties.getNAStrings()) );
-		job.set(MRJobConfiguration.TF_SPEC, spec);
-		job.set(MRJobConfiguration.TF_SMALLEST_FILE, CSVReblockMR.findSmallestFile(job, inputPath));
-		job.set(MRJobConfiguration.OUTPUT_MATRICES_DIRS_CONFIG, outputPath);
-		job.setLong(MRJobConfiguration.TF_NUM_COLS, numCols);
-		job.set(MRJobConfiguration.TF_TXMTD_PATH, mapsPath);
-		job.set(MRJobConfiguration.TF_HEADER, headerLine);
-		job.set(CSVReblockMR.ROWID_FILE_NAME, cachefile.toString());
-		job.set(MRJobConfiguration.TF_TMP_LOC, tmpPath);
-		
-		//turn off adaptivemr
-		job.setBoolean("adaptivemr.map.enable", false);
-
-		// Run the job
-		RunningJob runjob = JobClient.runJob(job);
-		
-		// Since transform CSV produces part files w/ prefix transform-part-*,
-		// delete all the "default" part-..... files
-		deletePartFiles(fs, outPath);
-		
-		MatrixCharacteristics mc = new MatrixCharacteristics();
-		return new JobReturn(new MatrixCharacteristics[]{mc}, runjob.isSuccessful());
-	}
-	
-	private static void deletePartFiles(FileSystem fs, Path path) throws FileNotFoundException, IOException
-	{
-		PathFilter filter=new PathFilter(){
-			public boolean accept(Path file) {
-				return file.getName().startsWith("part-");
-	        }
-		};
-		FileStatus[] list = fs.listStatus(path, filter);
-		for(FileStatus stat : list) {
-			fs.delete(stat.getPath(), false);
-		}
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVMapper.java b/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVMapper.java
deleted file mode 100644
index 05b8a19..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVMapper.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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.runtime.transform;
-import java.io.BufferedWriter;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.wink.json4j.JSONException;
-
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-
-public class ApplyTfCSVMapper implements Mapper<LongWritable, Text, NullWritable, Text> {
-	
-	boolean _firstRecordInSplit = true;
-	boolean _partFileWithHeader = false;
-	
-	TfUtils tfmapper = null;
-	Reporter _reporter = null;
-	BufferedWriter br = null;
-	JobConf _rJob = null;
-	
-	@Override
-	public void configure(JobConf job) {
-		try {
-			_rJob = job;
-			_partFileWithHeader = TfUtils.isPartFileWithHeader(job);
-			tfmapper = new TfUtils(job);
-			
-			tfmapper.loadTfMetadata(job, true);
-			
-		} catch (IOException e) { throw new RuntimeException(e); }
-		catch(JSONException e)  { throw new RuntimeException(e); }
-
-	}
-	
-	@Override
-	public void map(LongWritable rawKey, Text rawValue, OutputCollector<NullWritable, Text> out, Reporter reporter) throws IOException  {
-		
-		if(_firstRecordInSplit)
-		{
-			_firstRecordInSplit = false;
-			_reporter = reporter;
-			
-			// generate custom output paths so that order of rows in the 
-			// output (across part files) matches w/ that from input data set
-			String partFileSuffix = tfmapper.getPartFileID(_rJob, rawKey.get());
-			Path mapOutputPath = new Path(tfmapper.getOutputPath() + "/transform-part-" + partFileSuffix);
-			
-			// setup the writer for mapper's output
-			// the default part-..... files will be deleted later once the job finishes 
-			FileSystem fs = IOUtilFunctions.getFileSystem(mapOutputPath);
-			br = new BufferedWriter(new OutputStreamWriter(fs.create( mapOutputPath, true)));
-		}
-		
-		// output the header line
-		if ( rawKey.get() == 0 && _partFileWithHeader ) 
-		{
-			_reporter = reporter;
-			tfmapper.processHeaderLine();
-			if ( tfmapper.hasHeader() )
-				return;
-		}
-		
-		// parse the input line and apply transformation
-		String[] words = tfmapper.getWords(rawValue);
-		
-		if(!tfmapper.omit(words))
-		{
-			try {
-				words = tfmapper.apply(words);
-				String outStr = tfmapper.checkAndPrepOutputString(words);
-				//out.collect(NullWritable.get(), new Text(outStr));
-				br.write(outStr + "\n");
-			} 
-			catch(DMLRuntimeException e) {
-				throw new RuntimeException(e.getMessage() + ": " + rawValue.toString());
-			}
-		}
-	}
-
-	@Override
-	public void close() throws IOException {
-		IOUtilFunctions.closeSilently(br);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVSPARK.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVSPARK.java b/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVSPARK.java
deleted file mode 100644
index b820449..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/ApplyTfCSVSPARK.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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.runtime.transform;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.function.Function2;
-import org.apache.spark.api.java.function.PairFunction;
-import org.apache.spark.broadcast.Broadcast;
-import org.apache.wink.json4j.JSONException;
-import org.apache.wink.json4j.JSONObject;
-
-import scala.Tuple2;
-
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
-import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
-
-
-public class ApplyTfCSVSPARK {
-	
-	/**
-	 * Apply transformation metadata and generate the result in CSV format, as a
-	 * JavaRDD of Strings.
-	 * 
-	 * @param sec spark execution context
-	 * @param inputRDD input rdd
-	 * @param tfMtdPath transform metadata path
-	 * @param spec transform specification as json string
-	 * @param tmpPath temporary file path
-	 * @param prop csv file format properties
-	 * @param numCols number of columns
-	 * @param headerLine header line
-	 * @return JavaPairRDD of long-strings
-	 * @throws IOException if IOException occurs
-	 * @throws ClassNotFoundException if ClassNotFoundException occurs
-	 * @throws InterruptedException if InterruptedException occurs
-	 * @throws IllegalArgumentException if IllegalArgumentException occurs
-	 * @throws JSONException if JSONException occurs
-	 */
-	public static JavaPairRDD<Long, String> runSparkJob(
-			SparkExecutionContext sec, JavaRDD<Tuple2<LongWritable, Text>> inputRDD, 
-			String tfMtdPath, String spec, String tmpPath, CSVFileFormatProperties prop, 
-			int numCols, String headerLine) 
-		throws IOException, ClassNotFoundException, InterruptedException, IllegalArgumentException, JSONException 
-	{
-		// Load transformation metadata and broadcast it
-		String[] naStrings = TfUtils.parseNAStrings(prop.getNAStrings());
-		JSONObject jspec = new JSONObject(spec);
-		TfUtils _tfmapper = new TfUtils(headerLine, prop.hasHeader(), prop.getDelim(), naStrings, jspec, numCols, tfMtdPath, null, tmpPath);
-		
-		_tfmapper.loadTfMetadata();
-
-		Broadcast<TfUtils> bcast_tf = sec.getSparkContext().broadcast(_tfmapper);
-		
-		/*
-		 * Construct transformation metadata (map-side) -- the logic is similar
-		 * to GTFMTDMapper
-		 * 
-		 * Note: The result of mapPartitionsWithIndex is cached so that the
-		 * transformed data is not redundantly computed multiple times
-		 */
-		JavaPairRDD<Long, String> applyRDD = inputRDD
-				.mapPartitionsWithIndex( new ApplyTfCSVMap(bcast_tf),  true)
-				.mapToPair(
-						new PairFunction<String,Long,String>(){
-							private static final long serialVersionUID = 3868143093999082931L;
-							@Override
-							public Tuple2<Long, String> call(String t) throws Exception {
-								return new Tuple2<Long, String>(new Long(1), t);
-							}
-						}
-				).cache();
-
-		/*
-		 * An action to force execution of apply()
-		 * 
-		 * We need to trigger the execution of this RDD so as to ensure the
-		 * creation of a few metadata files (headers, dummycoded information,
-		 * etc.), which are referenced in the caller function.
-		 */
-		applyRDD.count();
-		
-		return applyRDD;
-	}
-
-	public static class ApplyTfCSVMap implements Function2<Integer, Iterator<Tuple2<LongWritable, Text>>, Iterator<String>> {
-
-		private static final long serialVersionUID = 1496686437276906911L;
-
-		TfUtils _tfmapper = null;
-		
-		ApplyTfCSVMap(Broadcast<TfUtils> tf) throws IllegalArgumentException, IOException, JSONException {
-			_tfmapper = tf.getValue();
-		}
-		
-		@Override
-		public Iterator<String> call(Integer partitionID,
-				Iterator<Tuple2<LongWritable, Text>> csvLines) throws Exception {
-			
-			boolean first = true;
-			Tuple2<LongWritable, Text> rec = null;
-			ArrayList<String> outLines = new ArrayList<String>();
-			
-			while(csvLines.hasNext()) {
-				rec = csvLines.next();
-				
-				if (first && partitionID == 0) {
-					first = false;
-					
-					_tfmapper.processHeaderLine();
-					
-					if (_tfmapper.hasHeader() ) {
-						continue; 
-					}
-				}
-				
-				// parse the input line and apply transformation
-			
-				String[] words = _tfmapper.getWords(rec._2());
-				
-				if(!_tfmapper.omit(words))
-				{
-					try {
-						words = _tfmapper.apply(words);
-						String outStr = _tfmapper.checkAndPrepOutputString(words);
-						outLines.add(outStr);
-					} 
-					catch(DMLRuntimeException e) {
-						throw new RuntimeException(e.getMessage() + ": " + rec._2().toString());
-					}
-				}
-			}
-			
-			return outLines.iterator();
-		}
-		
-	}
-
-	
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/BinAgent.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/BinAgent.java b/src/main/java/org/apache/sysml/runtime/transform/BinAgent.java
deleted file mode 100644
index 8878ff0..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/BinAgent.java
+++ /dev/null
@@ -1,382 +0,0 @@
-/*
- * 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.runtime.transform;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.nio.charset.CharacterCodingException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.wink.json4j.JSONArray;
-import org.apache.wink.json4j.JSONException;
-import org.apache.wink.json4j.JSONObject;
-import org.apache.sysml.lops.Lop;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.matrix.data.FrameBlock;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.matrix.data.Pair;
-import org.apache.sysml.runtime.transform.MVImputeAgent.MVMethod;
-import org.apache.sysml.runtime.transform.encode.Encoder;
-import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
-import org.apache.sysml.runtime.util.UtilFunctions;
-
-public class BinAgent extends Encoder 
-{	
-	private static final long serialVersionUID = 1917445005206076078L;
-
-	public static final String MIN_PREFIX = "min";
-	public static final String MAX_PREFIX = "max";
-	public static final String NBINS_PREFIX = "nbins";
-
-	private int[] _numBins = null;
-	private double[] _min=null, _max=null;	// min and max among non-missing values
-	private double[] _binWidths = null;		// width of a bin for each attribute
-	
-	//frame transform-apply attributes
-	private double[][] _binMins = null;
-	private double[][] _binMaxs = null;
-	
-	public BinAgent(JSONObject parsedSpec, String[] colnames, int clen) 
-		throws JSONException, IOException 
-	{
-		this(parsedSpec, colnames, clen, false);
-	}
-
-	public BinAgent(JSONObject parsedSpec, String[] colnames, int clen, boolean colsOnly) 
-		throws JSONException, IOException 
-	{
-		super( null, clen );		
-		if ( !parsedSpec.containsKey(TfUtils.TXMETHOD_BIN) )
-			return;
-		
-		if( colsOnly ) {
-			List<Integer> collist = TfMetaUtils.parseBinningColIDs(parsedSpec, colnames);
-			initColList(ArrayUtils.toPrimitive(collist.toArray(new Integer[0])));
-		}
-		else 
-		{
-			JSONObject obj = (JSONObject) parsedSpec.get(TfUtils.TXMETHOD_BIN);		
-			JSONArray attrs = (JSONArray) obj.get(TfUtils.JSON_ATTRS);
-			JSONArray nbins = (JSONArray) obj.get(TfUtils.JSON_NBINS);
-			initColList(attrs);
-			
-			_numBins = new int[attrs.size()];
-			for(int i=0; i < _numBins.length; i++)
-				_numBins[i] = UtilFunctions.toInt(nbins.get(i)); 
-			
-			// initialize internal transformation metadata
-			_min = new double[_colList.length];
-			Arrays.fill(_min, Double.MAX_VALUE);
-			_max = new double[_colList.length];
-			Arrays.fill(_max, -Double.MAX_VALUE);
-			
-			_binWidths = new double[_colList.length];
-		}
-	}
-
-	public int[] getNumBins() { return _numBins; }
-	public double[] getMin()  { return _min; }
-	public double[] getBinWidths() { return _binWidths; }
-	
-	public void prepare(String[] words, TfUtils agents) {
-		if ( !isApplicable() )
-			return;
-		
-		for(int i=0; i <_colList.length; i++) {
-			int colID = _colList[i];
-			
-			String w = null;
-			double d = 0;
-				
-			// equi-width
-			w = UtilFunctions.unquote(words[colID-1].trim());
-			if(!TfUtils.isNA(agents.getNAStrings(),w)) {
-				d = UtilFunctions.parseToDouble(w);
-				if(d < _min[i])
-					_min[i] = d;
-				if(d > _max[i])
-					_max[i] = d;
-			}
-		}
-	}
-	
-	private DistinctValue prepMinOutput(int idx) throws CharacterCodingException {
-		String s =  MIN_PREFIX + Double.toString(_min[idx]);
-		return  new DistinctValue(s, -1L);
-	}
-	
-	private DistinctValue prepMaxOutput(int idx) throws CharacterCodingException {
-		String s =  MAX_PREFIX + Double.toString(_max[idx]);
-		return  new DistinctValue(s, -1L);
-	}
-	
-	private DistinctValue prepNBinsOutput(int idx) throws CharacterCodingException {
-		String s =  NBINS_PREFIX + Double.toString(_numBins[idx]);
-		return  new DistinctValue(s, -1L);
-	}
-	
-	/**
-	 * Method to output transformation metadata from the mappers. 
-	 * This information is collected and merged by the reducers.
-	 */
-	@Override
-	public void mapOutputTransformationMetadata(OutputCollector<IntWritable, DistinctValue> out, int taskID, TfUtils agents) throws IOException {
-		if( !isApplicable() )
-			return;
-		
-		try { 
-			for(int i=0; i < _colList.length; i++) {
-				int colID = _colList[i];
-				IntWritable iw = new IntWritable(-colID);
-				
-				out.collect(iw,  prepMinOutput(i));
-				out.collect(iw,  prepMaxOutput(i));
-				out.collect(iw,  prepNBinsOutput(i));
-			}
-		} catch(Exception e) {
-			throw new IOException(e);
-		}
-	}
-	
-	public ArrayList<Pair<Integer, DistinctValue>> mapOutputTransformationMetadata(int taskID, ArrayList<Pair<Integer, DistinctValue>> list, TfUtils agents) throws IOException {
-		if ( !isApplicable() )
-			return list;
-		
-		try { 
-			for(int i=0; i < _colList.length; i++) {
-				int colID = _colList[i];
-				Integer iw = -colID;
-				
-				list.add( new Pair<Integer,DistinctValue>(iw, prepMinOutput(i)) );
-				list.add( new Pair<Integer,DistinctValue>(iw, prepMaxOutput(i)) );
-				list.add( new Pair<Integer,DistinctValue>(iw, prepNBinsOutput(i)) );
-			}
-		} catch(Exception e) {
-			throw new IOException(e);
-		}
-		return list;
-	}
-
-	private void writeTfMtd(int colID, String min, String max, String binwidth, String nbins, String tfMtdDir, FileSystem fs, TfUtils agents) throws IOException 
-	{
-		Path pt = new Path(tfMtdDir+"/Bin/"+ agents.getName(colID) + TfUtils.TXMTD_BIN_FILE_SUFFIX);
-		BufferedWriter br = null;
-		try {
-			br = new BufferedWriter(new OutputStreamWriter(fs.create(pt,true)));
-			br.write(colID + TfUtils.TXMTD_SEP + min + TfUtils.TXMTD_SEP + max + TfUtils.TXMTD_SEP + binwidth + TfUtils.TXMTD_SEP + nbins + "\n");
-		}
-		finally {
-			IOUtilFunctions.closeSilently(br);
-		}
-	}
-
-	/** 
-	 * Method to merge map output transformation metadata.
-	 */
-	@Override
-	public void mergeAndOutputTransformationMetadata(Iterator<DistinctValue> values, String outputDir, int colID, FileSystem fs, TfUtils agents) throws IOException {
-		double min = Double.MAX_VALUE;
-		double max = -Double.MAX_VALUE;
-		int nbins = 0;
-		
-		DistinctValue val = new DistinctValue();
-		String w = null;
-		double d;
-		while(values.hasNext()) {
-			val.reset();
-			val = values.next();
-			w = val.getWord();
-			
-			if(w.startsWith(MIN_PREFIX)) {
-				d = UtilFunctions.parseToDouble(w.substring( MIN_PREFIX.length() ));
-				if ( d < min )
-					min = d;
-			}
-			else if(w.startsWith(MAX_PREFIX)) {
-				d = UtilFunctions.parseToDouble(w.substring( MAX_PREFIX.length() ));
-				if ( d > max )
-					max = d;
-			}
-			else if (w.startsWith(NBINS_PREFIX)) {
-				nbins = (int) UtilFunctions.parseToLong( w.substring(NBINS_PREFIX.length() ) );
-			}
-			else
-				throw new RuntimeException("MVImputeAgent: Invalid prefix while merging map output: " + w);
-		}
-		
-		// write merged metadata
-		double binwidth = (max-min)/nbins;
-		writeTfMtd(colID, Double.toString(min), Double.toString(max), Double.toString(binwidth), Integer.toString(nbins), outputDir, fs, agents);
-	}
-	
-	
-	public void outputTransformationMetadata(String outputDir, FileSystem fs, TfUtils agents) throws IOException {
-		if( !isApplicable() )
-			return;
-		
-		MVImputeAgent mvagent = agents.getMVImputeAgent();
-		for(int i=0; i < _colList.length; i++) {
-			int colID = _colList[i];
-			
-			// If the column is imputed with a constant, then adjust min and max based the value of the constant.
-			if ( mvagent.isApplicable(colID) != -1 && mvagent.getMethod(colID) == MVMethod.CONSTANT ) 
-			{
-				double cst = UtilFunctions.parseToDouble( mvagent.getReplacement(colID) );
-				if ( cst < _min[i])
-					_min[i] = cst;
-				if ( cst > _max[i])
-					_max[i] = cst;
-			}
-			
-			double binwidth = (_max[i] - _min[i])/_numBins[i];
-			writeTfMtd(colID, Double.toString(_min[i]), Double.toString(_max[i]), Double.toString(binwidth), Integer.toString(_numBins[i]), outputDir, fs, agents);
-		}
-	}
-	
-	// ------------------------------------------------------------------------------------------------
-
-	/**
-	 * Method to load transform metadata for all attributes
-	 */
-	@Override
-	public void loadTxMtd(JobConf job, FileSystem fs, Path txMtdDir, TfUtils agents) throws IOException {
-		if( !isApplicable() )
-			return;
-		
-		if(fs.isDirectory(txMtdDir)) {
-			for(int i=0; i<_colList.length;i++) {
-				int colID = _colList[i];
-				
-				Path path = new Path( txMtdDir + "/Bin/" + agents.getName(colID) + TfUtils.TXMTD_BIN_FILE_SUFFIX);
-				TfUtils.checkValidInputFile(fs, path, true); 
-					
-				BufferedReader br = null;
-				try {
-					br = new BufferedReader(new InputStreamReader(fs.open(path)));
-					// format: colID,min,max,nbins
-					String[] fields = br.readLine().split(TfUtils.TXMTD_SEP);
-					double min = UtilFunctions.parseToDouble(fields[1]);
-					//double max = UtilFunctions.parseToDouble(fields[2]);
-					double binwidth = UtilFunctions.parseToDouble(fields[3]);
-					int nbins = UtilFunctions.parseToInt(fields[4]);
-					
-					_numBins[i] = nbins;
-					_min[i] = min;
-					_binWidths[i] = binwidth; // (max-min)/nbins;
-				}
-				finally {
-					IOUtilFunctions.closeSilently(br);
-				}
-			}
-		}
-		else {
-			throw new RuntimeException("Path to recode maps must be a directory: " + txMtdDir);
-		}
-	}
-	
-
-	@Override
-	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
-		build(in);
-		return apply(in, out);
-	}
-
-	@Override
-	public void build(FrameBlock in) {
-		// TODO Auto-generated method stub
-	}
-	
-	/**
-	 * Method to apply transformations.
-	 */
-	@Override
-	public String[] apply(String[] words) {
-		if( !isApplicable() )
-			return words;
-	
-		for(int i=0; i < _colList.length; i++) {
-			int colID = _colList[i];
-			try {
-				double val = UtilFunctions.parseToDouble(words[colID-1]);
-				int binid = 1;
-				double tmp = _min[i] + _binWidths[i];
-				while(val > tmp && binid < _numBins[i]) {
-					tmp += _binWidths[i];
-					binid++;
-				}
-				words[colID-1] = Integer.toString(binid);
-			} 
-			catch(NumberFormatException e) {
-				throw new RuntimeException("Encountered \"" + words[colID-1] + "\" in column ID \"" + colID + "\", when expecting a numeric value. Consider adding \"" + words[colID-1] + "\" to na.strings, along with an appropriate imputation method.");
-			}
-		}
-		
-		return words;
-	}
-
-	@Override
-	public MatrixBlock apply(FrameBlock in, MatrixBlock out) {
-		for(int j=0; j<_colList.length; j++) {
-			int colID = _colList[j];
-			for( int i=0; i<in.getNumRows(); i++ ) {
-				double inVal = UtilFunctions.objectToDouble(
-						in.getSchema()[colID-1], in.get(i, colID-1));
-				int ix = Arrays.binarySearch(_binMaxs[j], inVal);
-				int binID = ((ix < 0) ? Math.abs(ix+1) : ix) + 1;		
-				out.quickSetValue(i, colID-1, binID);
-			}	
-		}
-		return out;
-	}
-
-	@Override
-	public FrameBlock getMetaData(FrameBlock meta) {
-		return meta;
-	}
-	
-	@Override
-	public void initMetaData(FrameBlock meta) {
-		_binMins = new double[_colList.length][];
-		_binMaxs = new double[_colList.length][];
-		for( int j=0; j<_colList.length; j++ ) {
-			int colID = _colList[j]; //1-based
-			int nbins = (int)meta.getColumnMetadata()[colID-1].getNumDistinct();
-			_binMins[j] = new double[nbins];
-			_binMaxs[j] = new double[nbins];
-			for( int i=0; i<nbins; i++ ) {
-				String[] tmp = meta.get(i, colID-1).toString().split(Lop.DATATYPE_PREFIX);
-				_binMins[j][i] = Double.parseDouble(tmp[0]);
-				_binMaxs[j][i] = Double.parseDouble(tmp[1]);
-			}
-		}
-	}
-}


[2/6] systemml git commit: [SYSTEMML-1300] Remove file-based transform from compiler/runtime

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/TfUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/TfUtils.java b/src/main/java/org/apache/sysml/runtime/transform/TfUtils.java
index 9e30f5c..7743b61 100644
--- a/src/main/java/org/apache/sysml/runtime/transform/TfUtils.java
+++ b/src/main/java/org/apache/sysml/runtime/transform/TfUtils.java
@@ -19,40 +19,13 @@
 
 package org.apache.sysml.runtime.transform;
 
-import java.io.EOFException;
-import java.io.IOException;
 import java.io.Serializable;
-import java.util.Arrays;
 import java.util.regex.Pattern;
 
-import org.apache.hadoop.filecache.DistributedCache;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.ByteWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.wink.json4j.JSONException;
-import org.apache.wink.json4j.JSONObject;
-import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.lops.Lop;
-import org.apache.sysml.parser.DataExpression;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.io.MatrixReader;
-import org.apache.sysml.runtime.matrix.CSVReblockMR;
-import org.apache.sysml.runtime.matrix.CSVReblockMR.OffsetCount;
-import org.apache.sysml.runtime.matrix.mapred.MRConfigurationNames;
-import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration;
-import org.apache.sysml.runtime.util.MapReduceTool;
-import org.apache.sysml.runtime.util.UtilFunctions;
 
-@SuppressWarnings("deprecation")
-public class TfUtils implements Serializable{
-	
+public class TfUtils implements Serializable
+{	
 	private static final long serialVersionUID = 526252850872633125L;
 
 	protected enum ColumnTypes { 
@@ -89,9 +62,7 @@ public class TfUtils implements Serializable{
 	
 	//transform meta data constants (old file-based transform)
 	public static final String TXMTD_SEP         = ",";
-	public static final String TXMTD_COLTYPES    = "coltypes.csv";	
 	public static final String TXMTD_COLNAMES    = "column.names";
-	public static final String TXMTD_DC_COLNAMES = "dummycoded.column.names";	
 	public static final String TXMTD_RCD_MAP_SUFFIX      = ".map";
 	public static final String TXMTD_RCD_DISTINCT_SUFFIX = ".ndistinct";
 	public static final String TXMTD_BIN_FILE_SUFFIX     = ".bin";
@@ -101,184 +72,14 @@ public class TfUtils implements Serializable{
 	public static final String JSON_MTHD 	= "methods"; 
 	public static final String JSON_CONSTS = "constants"; 
 	public static final String JSON_NBINS 	= "numbins"; 		
-	protected static final String MODE_FILE_SUFFIX 		= ".mode";
-	protected static final String SCALE_FILE_SUFFIX		= ".scale";
-	protected static final String DCD_FILE_NAME 		= "dummyCodeMaps.csv";	
-	protected static final String DCD_NAME_SEP 	= "_";
-	
-	
-	private OmitAgent _oa = null;
-	private MVImputeAgent _mia = null;
-	private RecodeAgent _ra = null;	
-	private BinAgent _ba = null;
-	private DummycodeAgent _da = null;
-	
-	private long _numRecordsInPartFile;		// Total number of records in the data file
-	private long _numValidRecords;			// (_numRecordsInPartFile - #of omitted records)
-	private long _numTransformedRows; 		// Number of rows after applying transformations
-	private long _numTransformedColumns; 	// Number of columns after applying transformations
 
 	private String _headerLine = null;
 	private boolean _hasHeader;
 	private Pattern _delim = null;
 	private String _delimString = null;
 	private String[] _NAstrings = null;
-	private String[] _outputColumnNames = null;
 	private int _numInputCols = -1;
 	
-	private String _tfMtdDir = null;
-	private String _spec = null;
-	private String _offsetFile = null;
-	private String _tmpDir = null;
-	private String _outputPath = null;
-	
-	public TfUtils(JobConf job, boolean minimal) 
-		throws IOException, JSONException 
-	{
-		if( !InfrastructureAnalyzer.isLocalMode(job) ) {
-			ConfigurationManager.setCachedJobConf(job);
-		}		
-		_NAstrings = TfUtils.parseNAStrings(job);
-		_spec = job.get(MRJobConfiguration.TF_SPEC);
-		_oa = new OmitAgent(new JSONObject(_spec), null, -1);
-	}
-	
-	// called from GenTFMtdMapper, ApplyTf (Hadoop)
-	public TfUtils(JobConf job) 
-		throws IOException, JSONException 
-	{
-		if( !InfrastructureAnalyzer.isLocalMode(job) ) {
-			ConfigurationManager.setCachedJobConf(job);
-		}
-		
-		boolean hasHeader = Boolean.parseBoolean(job.get(MRJobConfiguration.TF_HAS_HEADER));
-		String[] naStrings = TfUtils.parseNAStrings(job);
-		long numCols = UtilFunctions.parseToLong( job.get(MRJobConfiguration.TF_NUM_COLS) ); // #cols input data
-		String spec = job.get(MRJobConfiguration.TF_SPEC);
-		String offsetFile = job.get(MRJobConfiguration.TF_OFFSETS_FILE);
-		String tmpPath = job.get(MRJobConfiguration.TF_TMP_LOC);
-		String outputPath = FileOutputFormat.getOutputPath(job).toString();
-		JSONObject jspec = new JSONObject(spec);
-		
-		init(job.get(MRJobConfiguration.TF_HEADER), hasHeader, job.get(MRJobConfiguration.TF_DELIM), naStrings, jspec, numCols, offsetFile, tmpPath, outputPath);
-	}
-	
-	// called from GenTfMtdReducer 
-	public TfUtils(JobConf job, String tfMtdDir) throws IOException, JSONException 
-	{
-		this(job);
-		_tfMtdDir = tfMtdDir;
-	}
-	
-	// called from GenTFMtdReducer and ApplyTf (Spark)
-	public TfUtils(String headerLine, boolean hasHeader, String delim, String[] naStrings, JSONObject spec, long ncol, String tfMtdDir, String offsetFile, String tmpPath) throws IOException, JSONException {
-		init (headerLine, hasHeader, delim, naStrings, spec, ncol, offsetFile, tmpPath, null);
-		_tfMtdDir = tfMtdDir;
-	}
-
-	protected static boolean checkValidInputFile(FileSystem fs, Path path, boolean err)
-			throws IOException {
-		// check non-existing file
-		if (!fs.exists(path))
-			if ( err )
-				throw new IOException("File " + path.toString() + " does not exist on HDFS/LFS.");
-			else
-				return false;
-
-		// check for empty file
-		if( MapReduceTool.isFileEmpty(fs, path) )
-			if ( err )
-				throw new EOFException("Empty input file " + path.toString() + ".");
-			else
-				return false;
-		
-		return true;
-	}
-	
-	public static String getPartFileName(JobConf job) throws IOException {
-		Path path = new Path(job.get(MRConfigurationNames.MR_MAP_INPUT_FILE));
-		FileSystem fs = IOUtilFunctions.getFileSystem(path, job);
-		path = path.makeQualified(fs);
-		return path.toString();
-	}
-	
-	public static boolean isPartFileWithHeader(JobConf job) throws IOException {
-		String thisfile=getPartFileName(job);
-		Path path = new Path(job.get(MRJobConfiguration.TF_SMALLEST_FILE));
-		FileSystem fs = IOUtilFunctions.getFileSystem(path, job);
-		path = path.makeQualified(fs);
-		return thisfile.toString().equals(path.toString());
-	}
-	
-	/**
-	 * Prepare NA strings so that they can be sent to workers via JobConf.
-	 * A "dummy" string is added at the end to handle the case of empty strings.
-	 * @param na NA string
-	 * @return NA string concatenated with NA string separator concatenated with "dummy"
-	 */
-	public static String prepNAStrings(String na) {
-		return na  + DataExpression.DELIM_NA_STRING_SEP + "dummy";
-	}
-	
-	public static String[] parseNAStrings(String na) 
-	{
-		if ( na == null )
-			return null;
-		
-		String[] tmp = Pattern.compile(Pattern.quote(DataExpression.DELIM_NA_STRING_SEP)).split(na, -1);
-		return tmp; //Arrays.copyOf(tmp, tmp.length-1);
-	}
-	
-	public static String[] parseNAStrings(JobConf job) 
-	{
-		return parseNAStrings(job.get(MRJobConfiguration.TF_NA_STRINGS));
-	}
-	
-	private void createAgents(JSONObject spec, String[] naStrings) 
-		throws IOException, JSONException 
-	{
-		_oa = new OmitAgent(spec, _outputColumnNames, _numInputCols);
-		_mia = new MVImputeAgent(spec, null, naStrings, _numInputCols);
-		_ra = new RecodeAgent(spec, _outputColumnNames, _numInputCols);
-		_ba = new BinAgent(spec, _outputColumnNames, _numInputCols);
-		_da = new DummycodeAgent(spec, _outputColumnNames, _numInputCols);
-	}
-	
-	private void parseColumnNames() {
-		_outputColumnNames = _delim.split(_headerLine, -1);
-		for(int i=0; i < _outputColumnNames.length; i++)
-			_outputColumnNames[i] = UtilFunctions.unquote(_outputColumnNames[i]);
-	}
-	
-	private void init(String headerLine, boolean hasHeader, String delim, String[] naStrings, JSONObject spec, long numCols, String offsetFile, String tmpPath, String outputPath) throws IOException, JSONException
-	{
-		_numRecordsInPartFile = 0;
-		_numValidRecords = 0;
-		_numTransformedRows = 0;
-		_numTransformedColumns = 0;
-		
-		//TODO: fix hard-wired header propagation to meta data column names
-		
-		_headerLine = headerLine;
-		_hasHeader = hasHeader;
-		_delimString = delim;
-		_delim = Pattern.compile(Pattern.quote(delim));
-		_NAstrings = naStrings;
-		_numInputCols = (int)numCols;
-		_offsetFile = offsetFile;
-		_tmpDir = tmpPath;
-		_outputPath = outputPath;
-		
-		parseColumnNames();		
-		createAgents(spec, naStrings);
-	}
-	
-	public void incrValid() { _numValidRecords++; }
-	public long getValid()  { return _numValidRecords; }
-	public long getTotal()  { return _numRecordsInPartFile; }
-	public long getNumTransformedRows() 	{ return _numTransformedRows; }
-	public long getNumTransformedColumns() 	{ return _numTransformedColumns; }
-	
 	public String getHeader() 		{ return _headerLine; }
 	public boolean hasHeader() 		{ return _hasHeader; }
 	public String getDelimString() 	{ return _delimString; }
@@ -286,24 +87,6 @@ public class TfUtils implements Serializable{
 	public String[] getNAStrings() 	{ return _NAstrings; }
 	public long getNumCols() 		{ return _numInputCols; }
 	
-	public String getSpec() 	{ return _spec; }
-	public String getTfMtdDir() 	{ return _tfMtdDir; }
-	public String getOffsetFile() 	{ return _offsetFile; }
-	public String getTmpDir() 		{ return _tmpDir; }
-	public String getOutputPath()	{ return _outputPath; }
-	
-	public String getName(int colID) { return _outputColumnNames[colID-1]; }
-	
-	public void setValid(long n) { _numValidRecords = n;}
-	public void incrTotal() { _numRecordsInPartFile++; }
-	public void setTotal(long n) { _numRecordsInPartFile = n;}
-	
-	public OmitAgent 	  getOmitAgent() 	{ 	return _oa; }
-	public MVImputeAgent  getMVImputeAgent(){ 	return _mia;}
-	public RecodeAgent 	  getRecodeAgent() 	{ 	return _ra; }
-	public BinAgent 	  getBinAgent() 	{ 	return _ba; }
-	public DummycodeAgent getDummycodeAgent() { return _da; }
-	
 	/**
 	 * Function that checks if the given string is one of NA strings.
 	 * 
@@ -321,229 +104,4 @@ public class TfUtils implements Serializable{
 		}
 		return false;
 	}
-	
-	public String[] getWords(Text line) {
-		return getWords(line.toString());
-	}
-	
-
-	public String[] getWords(String line) {
-		return getDelim().split(line.trim(), -1);
-	}
-	
-	/**
-	 * Process a given row to construct transformation metadata.
-	 * 
-	 * @param line string to break into words
-	 * @return string array of words from the line
-	 * @throws IOException if IOException occurs
-	 */
-	public String[] prepareTfMtd(String line) throws IOException {
-		String[] words = getWords(line);
-		if(!getOmitAgent().omit(words, this))
-		{
-			getMVImputeAgent().prepare(words);
-			getRecodeAgent().prepare(words, this);
-			getBinAgent().prepare(words, this);
-			incrValid();
-		}
-		incrTotal();
-		
-		return words;
-	}
-	
-	public void loadTfMetadata() throws IOException 
-	{
-		JobConf job = ConfigurationManager.getCachedJobConf();
-		loadTfMetadata(job, false);
-	}
-	
-	public void loadTfMetadata(JobConf job, boolean fromLocalFS) throws IOException
-	{
-		Path tfMtdDir = null; 
-		FileSystem fs = null;
-		
-		if(fromLocalFS) {
-			// metadata must be read from local file system (e.g., distributed cache in the case of Hadoop)
-			tfMtdDir = (DistributedCache.getLocalCacheFiles(job))[0];
-			fs = FileSystem.getLocal(job);
-		}
-		else {
-			tfMtdDir = new Path(getTfMtdDir());
-			fs = IOUtilFunctions.getFileSystem(tfMtdDir, job);
-		}
-		
-		// load transformation metadata 
-		getMVImputeAgent().loadTxMtd(job, fs, tfMtdDir, this);
-		getRecodeAgent().loadTxMtd(job, fs, tfMtdDir, this);
-		getBinAgent().loadTxMtd(job, fs, tfMtdDir, this);
-		
-		// associate recode maps and bin definitions with dummycoding agent,
-		// as recoded and binned columns are typically dummycoded
-		getDummycodeAgent().setRecodeMaps( getRecodeAgent().getRecodeMaps() );
-		getDummycodeAgent().setNumBins(getBinAgent().getColList(), getBinAgent().getNumBins());
-		getDummycodeAgent().loadTxMtd(job, fs, tfMtdDir, this);
-
-	}
-
-	public String processHeaderLine() throws IOException 
-	{
-		//TODO: fix hard-wired header propagation to meta data column names
-		
-		FileSystem fs = FileSystem.get(ConfigurationManager.getCachedJobConf());
-		String dcdHeader = getDummycodeAgent().constructDummycodedHeader(getHeader(), getDelim());
-		getDummycodeAgent().genDcdMapsAndColTypes(fs, getTmpDir(), (int) getNumCols(), this);
-		
-		// write header information (before and after transformation) to temporary path
-		// these files are copied into txMtdPath, once the ApplyTf job is complete.
-		DataTransform.generateHeaderFiles(fs, getTmpDir(), getHeader(), dcdHeader);
-
-		return dcdHeader;
-		//_numTransformedColumns = getDelim().split(dcdHeader, -1).length; 
-		//return _numTransformedColumns;
-	}
-
-	public boolean omit(String[] words) {
-		if(getOmitAgent() == null)
-			return false;
-		return getOmitAgent().omit(words, this);
-	}
-	
-	/**
-	 * Function to apply transformation metadata on a given row.
-	 * 
-	 * @param words string array of words
-	 * @return string array of transformed words
-	 */
-	public String[] apply( String[] words ) {
-		words = getMVImputeAgent().apply(words);
-		words = getRecodeAgent().apply(words);
-		words = getBinAgent().apply(words);
-		words = getDummycodeAgent().apply(words);		
-		_numTransformedRows++;
-		
-		return words;
-	}
-	
-	public void check(String []words) throws DMLRuntimeException 
-	{
-		boolean checkEmptyString = ( getNAStrings() != null );
-		if ( checkEmptyString ) 
-		{
-			final String msg = "When na.strings are provided, empty string \"\" is considered as a missing value, and it must be imputed appropriately. Encountered an unhandled empty string in column ID: ";
-			for(int i=0; i<words.length; i++) 
-				if ( words[i] != null && words[i].equals(""))
-					throw new DMLRuntimeException(msg + getDummycodeAgent().mapDcdColumnID(i+1));
-		}
-	}
-	
-	public String checkAndPrepOutputString(String []words) throws DMLRuntimeException {
-		return checkAndPrepOutputString(words, new StringBuilder());
-	}
-	
-	public String checkAndPrepOutputString(String []words, StringBuilder sb) throws DMLRuntimeException 
-	{
-		/*
-		 * Check if empty strings ("") have to be handled.
-		 * 
-		 * Unless na.strings are provided, empty strings are (implicitly) considered as value zero.
-		 * When na.strings are provided, then "" is considered a missing value indicator, and the 
-		 * user is expected to provide an appropriate imputation method. Therefore, when na.strings 
-		 * are provided, "" encountered in any column (after all transformations are applied) 
-		 * denotes an erroneous condition.  
-		 */
-		boolean checkEmptyString = ( getNAStrings() != null ); //&& !MVImputeAgent.isNA("", TransformationAgent.NAstrings) ) {
-		
-		//StringBuilder sb = new StringBuilder();
-		sb.setLength(0);
-		int i =0;
-		
-		if ( checkEmptyString ) 
-		{
-			final String msg = "When na.strings are provided, empty string \"\" is considered as a missing value, and it must be imputed appropriately. Encountered an unhandled empty string in column ID: ";
-			if ( words[0] != null ) 
-				if ( words[0].equals("") )
-					throw new DMLRuntimeException( msg + getDummycodeAgent().mapDcdColumnID(1));
-				else 
-					sb.append(words[0]);
-			else
-				sb.append("0");
-			
-			for(i=1; i<words.length; i++) 
-			{
-				sb.append(_delimString);
-				
-				if ( words[i] != null ) 
-					if ( words[i].equals("") )
-						throw new DMLRuntimeException(msg + getDummycodeAgent().mapDcdColumnID(i+1));
-					else 
-						sb.append(words[i]);
-				else
-					sb.append("0");
-			}
-		}
-		else 
-		{
-			sb.append(words[0] != null ? words[0] : "0");
-			for(i=1; i<words.length; i++) 
-			{
-				sb.append(_delimString);
-				sb.append(words[i] != null ? words[i] : "0");
-			}
-		}
-		
-		return sb.toString();
-	}
-
-	private Reader initOffsetsReader(JobConf job) throws IOException 
-	{
-		Path path=new Path(job.get(CSVReblockMR.ROWID_FILE_NAME));
-		FileSystem fs = IOUtilFunctions.getFileSystem(path, job);
-		Path[] files = MatrixReader.getSequenceFilePaths(fs, path);
-		if ( files.length != 1 )
-			throw new IOException("Expecting a single file under counters file: " + path.toString());
-		
-		Reader reader = new SequenceFile.Reader(fs, files[0], job);
-		
-		return reader;
-	}
-	
-	/**
-	 * Function to generate custom file names (transform-part-.....) for
-	 * mappers' output for ApplyTfCSV job. The idea is to find the index 
-	 * of (thisfile, fileoffset) in the list of all offsets from the 
-	 * counters/offsets file, which was generated from either GenTfMtdMR
-	 * or AssignRowIDMR job.
-	 * 
-	 * @param job job configuration
-	 * @param offset file offset
-	 * @return part file id (ie, 00001, 00002, etc)
-	 * @throws IOException if IOException occurs
-	 */
-	public String getPartFileID(JobConf job, long offset) throws IOException
-	{
-		Reader reader = null;
-		int id = 0;
-		try {
-			reader = initOffsetsReader(job);
-			ByteWritable key=new ByteWritable();
-			OffsetCount value=new OffsetCount();
-			String thisFile = TfUtils.getPartFileName(job);
-			while (reader.next(key, value)) {
-				if ( thisFile.equals(value.filename) && value.fileOffset == offset ) 
-					break;
-				id++;
-			}
-		}
-		finally {
-			IOUtilFunctions.closeSilently(reader);
-		}
-		
-		String sid = Integer.toString(id);
-		char[] carr = new char[5-sid.length()];
-		Arrays.fill(carr, '0');
-		String ret = (new String(carr)).concat(sid);
-		
-		return ret;
-	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/encode/Encoder.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/encode/Encoder.java b/src/main/java/org/apache/sysml/runtime/transform/encode/Encoder.java
index a3f01a1..304dcdb 100644
--- a/src/main/java/org/apache/sysml/runtime/transform/encode/Encoder.java
+++ b/src/main/java/org/apache/sysml/runtime/transform/encode/Encoder.java
@@ -19,20 +19,10 @@
 
 package org.apache.sysml.runtime.transform.encode;
 
-import java.io.IOException;
 import java.io.Serializable;
 import java.util.Arrays;
-import java.util.Iterator;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.sysml.runtime.matrix.data.FrameBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.transform.DistinctValue;
-import org.apache.sysml.runtime.transform.TfUtils;
 import org.apache.sysml.runtime.util.UtilFunctions;
 import org.apache.wink.json4j.JSONArray;
 
@@ -152,11 +142,4 @@ public abstract class Encoder implements Serializable
 	 * @param meta frame block
 	 */
 	public abstract void initMetaData(FrameBlock meta);
-	
-	
-	//OLD API: kept for a transition phase only
-	//TODO stage 2: refactor data and meta data IO into minimal set of ultility functions
-	abstract public void mapOutputTransformationMetadata(OutputCollector<IntWritable, DistinctValue> out, int taskID, TfUtils agents) throws IOException;
-	abstract public void mergeAndOutputTransformationMetadata(Iterator<DistinctValue> values, String outputDir, int colID, FileSystem fs, TfUtils agents) throws IOException;
-	abstract public void loadTxMtd(JobConf job, FileSystem fs, Path txMtdDir, TfUtils agents) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderBin.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderBin.java b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderBin.java
new file mode 100644
index 0000000..fbe6994
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderBin.java
@@ -0,0 +1,188 @@
+/*
+ * 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.runtime.transform.encode;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.wink.json4j.JSONArray;
+import org.apache.wink.json4j.JSONException;
+import org.apache.wink.json4j.JSONObject;
+import org.apache.sysml.lops.Lop;
+import org.apache.sysml.runtime.matrix.data.FrameBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.transform.TfUtils;
+import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
+import org.apache.sysml.runtime.util.UtilFunctions;
+
+public class EncoderBin extends Encoder 
+{	
+	private static final long serialVersionUID = 1917445005206076078L;
+
+	public static final String MIN_PREFIX = "min";
+	public static final String MAX_PREFIX = "max";
+	public static final String NBINS_PREFIX = "nbins";
+
+	private int[] _numBins = null;
+	private double[] _min=null, _max=null;	// min and max among non-missing values
+	private double[] _binWidths = null;		// width of a bin for each attribute
+	
+	//frame transform-apply attributes
+	private double[][] _binMins = null;
+	private double[][] _binMaxs = null;
+	
+	public EncoderBin(JSONObject parsedSpec, String[] colnames, int clen) 
+		throws JSONException, IOException 
+	{
+		this(parsedSpec, colnames, clen, false);
+	}
+
+	public EncoderBin(JSONObject parsedSpec, String[] colnames, int clen, boolean colsOnly) 
+		throws JSONException, IOException 
+	{
+		super( null, clen );		
+		if ( !parsedSpec.containsKey(TfUtils.TXMETHOD_BIN) )
+			return;
+		
+		if( colsOnly ) {
+			List<Integer> collist = TfMetaUtils.parseBinningColIDs(parsedSpec, colnames);
+			initColList(ArrayUtils.toPrimitive(collist.toArray(new Integer[0])));
+		}
+		else 
+		{
+			JSONObject obj = (JSONObject) parsedSpec.get(TfUtils.TXMETHOD_BIN);		
+			JSONArray attrs = (JSONArray) obj.get(TfUtils.JSON_ATTRS);
+			JSONArray nbins = (JSONArray) obj.get(TfUtils.JSON_NBINS);
+			initColList(attrs);
+			
+			_numBins = new int[attrs.size()];
+			for(int i=0; i < _numBins.length; i++)
+				_numBins[i] = UtilFunctions.toInt(nbins.get(i)); 
+			
+			// initialize internal transformation metadata
+			_min = new double[_colList.length];
+			Arrays.fill(_min, Double.MAX_VALUE);
+			_max = new double[_colList.length];
+			Arrays.fill(_max, -Double.MAX_VALUE);
+			
+			_binWidths = new double[_colList.length];
+		}
+	}
+
+	public void prepare(String[] words, TfUtils agents) {
+		if ( !isApplicable() )
+			return;
+		
+		for(int i=0; i <_colList.length; i++) {
+			int colID = _colList[i];
+			
+			String w = null;
+			double d = 0;
+				
+			// equi-width
+			w = UtilFunctions.unquote(words[colID-1].trim());
+			if(!TfUtils.isNA(agents.getNAStrings(),w)) {
+				d = UtilFunctions.parseToDouble(w);
+				if(d < _min[i])
+					_min[i] = d;
+				if(d > _max[i])
+					_max[i] = d;
+			}
+		}
+	}
+		
+	@Override
+	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
+		build(in);
+		return apply(in, out);
+	}
+
+	@Override
+	public void build(FrameBlock in) {
+		// nothing to do
+	}
+	
+	/**
+	 * Method to apply transformations.
+	 */
+	@Override
+	public String[] apply(String[] words) {
+		if( !isApplicable() )
+			return words;
+	
+		for(int i=0; i < _colList.length; i++) {
+			int colID = _colList[i];
+			try {
+				double val = UtilFunctions.parseToDouble(words[colID-1]);
+				int binid = 1;
+				double tmp = _min[i] + _binWidths[i];
+				while(val > tmp && binid < _numBins[i]) {
+					tmp += _binWidths[i];
+					binid++;
+				}
+				words[colID-1] = Integer.toString(binid);
+			} 
+			catch(NumberFormatException e) {
+				throw new RuntimeException("Encountered \"" + words[colID-1] + "\" in column ID \"" + colID + "\", when expecting a numeric value. Consider adding \"" + words[colID-1] + "\" to na.strings, along with an appropriate imputation method.");
+			}
+		}
+		
+		return words;
+	}
+
+	@Override
+	public MatrixBlock apply(FrameBlock in, MatrixBlock out) {
+		for(int j=0; j<_colList.length; j++) {
+			int colID = _colList[j];
+			for( int i=0; i<in.getNumRows(); i++ ) {
+				double inVal = UtilFunctions.objectToDouble(
+						in.getSchema()[colID-1], in.get(i, colID-1));
+				int ix = Arrays.binarySearch(_binMaxs[j], inVal);
+				int binID = ((ix < 0) ? Math.abs(ix+1) : ix) + 1;		
+				out.quickSetValue(i, colID-1, binID);
+			}	
+		}
+		return out;
+	}
+
+	@Override
+	public FrameBlock getMetaData(FrameBlock meta) {
+		return meta;
+	}
+	
+	@Override
+	public void initMetaData(FrameBlock meta) {
+		_binMins = new double[_colList.length][];
+		_binMaxs = new double[_colList.length][];
+		for( int j=0; j<_colList.length; j++ ) {
+			int colID = _colList[j]; //1-based
+			int nbins = (int)meta.getColumnMetadata()[colID-1].getNumDistinct();
+			_binMins[j] = new double[nbins];
+			_binMaxs[j] = new double[nbins];
+			for( int i=0; i<nbins; i++ ) {
+				String[] tmp = meta.get(i, colID-1).toString().split(Lop.DATATYPE_PREFIX);
+				_binMins[j][i] = Double.parseDouble(tmp[0]);
+				_binMaxs[j][i] = Double.parseDouble(tmp[1]);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderComposite.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderComposite.java b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderComposite.java
index 9efbc19..deff887 100644
--- a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderComposite.java
+++ b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderComposite.java
@@ -19,20 +19,11 @@
 
 package org.apache.sysml.runtime.transform.encode;
 
-import java.io.IOException;
-import java.util.Iterator;
 import java.util.List;
 
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.matrix.data.FrameBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.transform.DistinctValue;
-import org.apache.sysml.runtime.transform.TfUtils;
 
 /**
  * Simple composite encoder that applies a list of encoders 
@@ -90,7 +81,6 @@ public class EncoderComposite extends Encoder
 			encoder.build(in);
 	}
 
-
 	@Override
 	public String[] apply(String[] in) {
 		for( Encoder encoder : _encoders )
@@ -119,19 +109,4 @@ public class EncoderComposite extends Encoder
 		for( Encoder encoder : _encoders )
 			encoder.initMetaData(out);
 	}
-
-	@Override
-	public void mapOutputTransformationMetadata(OutputCollector<IntWritable, DistinctValue> out, int taskID, TfUtils agents) throws IOException {
-		throw new RuntimeException("File-based api not supported.");
-	}
-
-	@Override
-	public void mergeAndOutputTransformationMetadata(Iterator<DistinctValue> values, String outputDir, int colID, FileSystem fs, TfUtils agents) throws IOException {
-		throw new RuntimeException("File-based api not supported.");	
-	}
-
-	@Override
-	public void loadTxMtd(JobConf job, FileSystem fs, Path txMtdDir, TfUtils agents) throws IOException {
-		throw new RuntimeException("File-based api not supported.");
-	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderDummycode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderDummycode.java b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderDummycode.java
new file mode 100644
index 0000000..743381a
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderDummycode.java
@@ -0,0 +1,139 @@
+/*
+ * 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.runtime.transform.encode;
+
+import org.apache.sysml.runtime.matrix.data.FrameBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.transform.TfUtils;
+import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
+import org.apache.sysml.runtime.util.UtilFunctions;
+import org.apache.wink.json4j.JSONException;
+import org.apache.wink.json4j.JSONObject;
+
+public class EncoderDummycode extends Encoder 
+{		
+	private static final long serialVersionUID = 5832130477659116489L;
+
+	private int[] _domainSizes = null;			// length = #of dummycoded columns
+	private long _dummycodedLength = 0;			// #of columns after dummycoded
+
+	public EncoderDummycode(JSONObject parsedSpec, String[] colnames, int clen) throws JSONException {
+		super(null, clen);
+		
+		if ( parsedSpec.containsKey(TfUtils.TXMETHOD_DUMMYCODE) ) {
+			int[] collist = TfMetaUtils.parseJsonIDList(parsedSpec, colnames, TfUtils.TXMETHOD_DUMMYCODE);
+			initColList(collist);
+		}
+	}
+	
+	@Override
+	public int getNumCols() {
+		return (int)_dummycodedLength;
+	}
+	
+	@Override
+	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
+		return apply(in, out);
+	}
+
+	@Override
+	public void build(FrameBlock in) {
+		//do nothing
+	}
+	
+	/**
+	 * Method to apply transformations.
+	 * 
+	 * @param words array of strings
+	 * @return array of transformed strings
+	 */
+	@Override
+	public String[] apply(String[] words) 
+	{
+		if( !isApplicable() )
+			return words;
+		
+		String[] nwords = new String[(int)_dummycodedLength];
+		int rcdVal = 0;
+		
+		for(int colID=1, idx=0, ncolID=1; colID <= words.length; colID++) {
+			if(idx < _colList.length && colID==_colList[idx]) {
+				// dummycoded columns
+				try {
+					rcdVal = UtilFunctions.parseToInt(UtilFunctions.unquote(words[colID-1]));
+					nwords[ ncolID-1+rcdVal-1 ] = "1";
+					ncolID += _domainSizes[idx];
+					idx++;
+				} 
+				catch (Exception e) {
+					throw new RuntimeException("Error in dummycoding: colID="+colID + ", rcdVal=" + rcdVal+", word="+words[colID-1] 
+							+ ", domainSize=" + _domainSizes[idx] + ", dummyCodedLength=" + _dummycodedLength);
+				}
+			}
+			else {
+				nwords[ncolID-1] = words[colID-1];
+				ncolID++;
+			}
+		}
+		
+		return nwords;
+	}
+	
+	@Override
+	public MatrixBlock apply(FrameBlock in, MatrixBlock out) 
+	{
+		MatrixBlock ret = new MatrixBlock(out.getNumRows(), (int)_dummycodedLength, false);
+		
+		for( int i=0; i<out.getNumRows(); i++ ) {
+			for(int colID=1, idx=0, ncolID=1; colID <= out.getNumColumns(); colID++) {
+				double val = out.quickGetValue(i, colID-1);
+				if(idx < _colList.length && colID==_colList[idx]) {
+					ret.quickSetValue(i, ncolID-1+(int)val-1, 1);
+					ncolID += _domainSizes[idx];
+					idx++;
+				}
+				else {
+					double ptval = UtilFunctions.objectToDouble(in.getSchema()[colID-1], in.get(i, colID-1));
+					ret.quickSetValue(i, ncolID-1, ptval);
+					ncolID++;
+				}
+			}
+		}
+		
+		return ret;
+	}
+
+	@Override
+	public FrameBlock getMetaData(FrameBlock out) {
+		return out;
+	}
+	
+	@Override
+	public void initMetaData(FrameBlock meta) {
+		//initialize domain sizes and output num columns
+		_domainSizes = new int[_colList.length];
+		_dummycodedLength = _clen;
+		for( int j=0; j<_colList.length; j++ ) {
+			int colID = _colList[j]; //1-based
+			_domainSizes[j] = (int)meta.getColumnMetadata()[colID-1].getNumDistinct();
+			_dummycodedLength +=  _domainSizes[j]-1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderFactory.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderFactory.java b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderFactory.java
index f7ceefd..13b2810 100644
--- a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderFactory.java
+++ b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderFactory.java
@@ -28,11 +28,6 @@ import org.apache.commons.lang.ArrayUtils;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.matrix.data.FrameBlock;
-import org.apache.sysml.runtime.transform.BinAgent;
-import org.apache.sysml.runtime.transform.DummycodeAgent;
-import org.apache.sysml.runtime.transform.MVImputeAgent;
-import org.apache.sysml.runtime.transform.OmitAgent;
-import org.apache.sysml.runtime.transform.RecodeAgent;
 import org.apache.sysml.runtime.transform.TfUtils;
 import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
 import org.apache.sysml.runtime.util.UtilFunctions;
@@ -40,7 +35,6 @@ import org.apache.wink.json4j.JSONObject;
 
 public class EncoderFactory 
 {
-
 	public static Encoder createEncoder(String spec, String[] colnames, int clen, FrameBlock meta) throws DMLRuntimeException {
 		return createEncoder(spec, colnames, UtilFunctions.nCopies(clen, ValueType.STRING), meta);
 	}
@@ -79,7 +73,7 @@ public class EncoderFactory
 			
 			//create individual encoders
 			if( !rcIDs.isEmpty() ) {
-				RecodeAgent ra = new RecodeAgent(jSpec, colnames, clen);
+				EncoderRecode ra = new EncoderRecode(jSpec, colnames, clen);
 				ra.setColList(ArrayUtils.toPrimitive(rcIDs.toArray(new Integer[0])));
 				lencoders.add(ra);	
 			}
@@ -87,13 +81,13 @@ public class EncoderFactory
 				lencoders.add(new EncoderPassThrough(
 						ArrayUtils.toPrimitive(ptIDs.toArray(new Integer[0])), clen));	
 			if( !dcIDs.isEmpty() )
-				lencoders.add(new DummycodeAgent(jSpec, colnames, schema.length));
+				lencoders.add(new EncoderDummycode(jSpec, colnames, schema.length));
 			if( !binIDs.isEmpty() )
-				lencoders.add(new BinAgent(jSpec, colnames, schema.length, true));
+				lencoders.add(new EncoderBin(jSpec, colnames, schema.length, true));
 			if( !oIDs.isEmpty() )
-				lencoders.add(new OmitAgent(jSpec, colnames, schema.length));
+				lencoders.add(new EncoderOmit(jSpec, colnames, schema.length));
 			if( !mvIDs.isEmpty() ) {
-				MVImputeAgent ma = new MVImputeAgent(jSpec, colnames, schema.length);
+				EncoderMVImpute ma = new EncoderMVImpute(jSpec, colnames, schema.length);
 				ma.initRecodeIDList(rcIDs);
 				lencoders.add(ma);
 			}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderMVImpute.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderMVImpute.java b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderMVImpute.java
new file mode 100644
index 0000000..55a0bde
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderMVImpute.java
@@ -0,0 +1,422 @@
+/*
+ * 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.runtime.transform.encode;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.wink.json4j.JSONArray;
+import org.apache.wink.json4j.JSONException;
+import org.apache.wink.json4j.JSONObject;
+import org.apache.sysml.runtime.functionobjects.CM;
+import org.apache.sysml.runtime.functionobjects.Mean;
+import org.apache.sysml.runtime.instructions.cp.CM_COV_Object;
+import org.apache.sysml.runtime.instructions.cp.KahanObject;
+import org.apache.sysml.runtime.matrix.data.FrameBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.CMOperator.AggregateOperationTypes;
+import org.apache.sysml.runtime.transform.TfUtils;
+import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
+import org.apache.sysml.runtime.util.UtilFunctions;
+
+public class EncoderMVImpute extends Encoder 
+{	
+	private static final long serialVersionUID = 9057868620144662194L;
+
+	public enum MVMethod { INVALID, GLOBAL_MEAN, GLOBAL_MODE, CONSTANT };
+	
+	private MVMethod[] _mvMethodList = null;
+	private MVMethod[] _mvscMethodList = null;	// scaling methods for attributes that are imputed and also scaled
+	
+	private BitSet _isMVScaled = null;
+	private CM _varFn = CM.getCMFnObject(AggregateOperationTypes.VARIANCE);		// function object that understands variance computation
+	
+	// objects required to compute mean and variance of all non-missing entries 
+	private Mean _meanFn = Mean.getMeanFnObject();	// function object that understands mean computation
+	private KahanObject[] _meanList = null; 		// column-level means, computed so far
+	private long[] _countList = null;				// #of non-missing values
+	
+	private CM_COV_Object[] _varList = null;		// column-level variances, computed so far (for scaling)
+
+	private int[] 			_scnomvList = null;			// List of attributes that are scaled but not imputed
+	private MVMethod[]		_scnomvMethodList = null;	// scaling methods: 0 for invalid; 1 for mean-subtraction; 2 for z-scoring
+	private KahanObject[] 	_scnomvMeanList = null;		// column-level means, for attributes scaled but not imputed
+	private long[] 			_scnomvCountList = null;	// #of non-missing values, for attributes scaled but not imputed
+	private CM_COV_Object[] _scnomvVarList = null;		// column-level variances, computed so far
+	
+	private String[] _replacementList = null;		// replacements: for global_mean, mean; and for global_mode, recode id of mode category
+	private String[] _NAstrings = null;
+	private List<Integer> _rcList = null; 
+	private HashMap<Integer,HashMap<String,Long>> _hist = null;
+	
+	public String[] getReplacements() { return _replacementList; }
+	public KahanObject[] getMeans()   { return _meanList; }
+	public CM_COV_Object[] getVars()  { return _varList; }
+	public KahanObject[] getMeans_scnomv()   { return _scnomvMeanList; }
+	public CM_COV_Object[] getVars_scnomv()  { return _scnomvVarList; }
+	
+	public EncoderMVImpute(JSONObject parsedSpec, String[] colnames, int clen) 
+		throws JSONException
+	{
+		super(null, clen);
+		
+		//handle column list
+		int[] collist = TfMetaUtils.parseJsonObjectIDList(parsedSpec, colnames, TfUtils.TXMETHOD_IMPUTE);
+		initColList(collist);
+	
+		//handle method list
+		parseMethodsAndReplacments(parsedSpec);
+		
+		//create reuse histograms
+		_hist = new HashMap<Integer, HashMap<String,Long>>();
+	}
+			
+	public EncoderMVImpute(JSONObject parsedSpec, String[] colnames, String[] NAstrings, int clen)
+		throws JSONException 
+	{
+		super(null, clen);	
+		boolean isMV = parsedSpec.containsKey(TfUtils.TXMETHOD_IMPUTE);
+		boolean isSC = parsedSpec.containsKey(TfUtils.TXMETHOD_SCALE);		
+		_NAstrings = NAstrings;
+		
+		if(!isMV) {
+			// MV Impute is not applicable
+			_colList = null;
+			_mvMethodList = null;
+			_meanList = null;
+			_countList = null;
+			_replacementList = null;
+		}
+		else {
+			JSONObject mvobj = (JSONObject) parsedSpec.get(TfUtils.TXMETHOD_IMPUTE);
+			JSONArray mvattrs = (JSONArray) mvobj.get(TfUtils.JSON_ATTRS);
+			JSONArray mvmthds = (JSONArray) mvobj.get(TfUtils.JSON_MTHD);
+			int mvLength = mvattrs.size();
+			
+			_colList = new int[mvLength];
+			_mvMethodList = new MVMethod[mvLength];
+			
+			_meanList = new KahanObject[mvLength];
+			_countList = new long[mvLength];
+			_varList = new CM_COV_Object[mvLength];
+			
+			_isMVScaled = new BitSet(_colList.length);
+			_isMVScaled.clear();
+			
+			for(int i=0; i < _colList.length; i++) {
+				_colList[i] = UtilFunctions.toInt(mvattrs.get(i));
+				_mvMethodList[i] = MVMethod.values()[UtilFunctions.toInt(mvmthds.get(i))]; 
+				_meanList[i] = new KahanObject(0, 0);
+			}
+			
+			_replacementList = new String[mvLength]; 	// contains replacements for all columns (scale and categorical)
+			
+			JSONArray constants = (JSONArray)mvobj.get(TfUtils.JSON_CONSTS);
+			for(int i=0; i < constants.size(); i++) {
+				if ( constants.get(i) == null )
+					_replacementList[i] = "NaN";
+				else
+					_replacementList[i] = constants.get(i).toString();
+			}
+		}
+		
+		// Handle scaled attributes
+		if ( !isSC )
+		{
+			// scaling is not applicable
+			_scnomvCountList = null;
+			_scnomvMeanList = null;
+			_scnomvVarList = null;
+		}
+		else
+		{
+			if ( _colList != null ) 
+				_mvscMethodList = new MVMethod[_colList.length];
+			
+			JSONObject scobj = (JSONObject) parsedSpec.get(TfUtils.TXMETHOD_SCALE);
+			JSONArray scattrs = (JSONArray) scobj.get(TfUtils.JSON_ATTRS);
+			JSONArray scmthds = (JSONArray) scobj.get(TfUtils.JSON_MTHD);
+			int scLength = scattrs.size();
+			
+			int[] _allscaled = new int[scLength];
+			int scnomv = 0, colID;
+			byte mthd;
+			for(int i=0; i < scLength; i++)
+			{
+				colID = UtilFunctions.toInt(scattrs.get(i));
+				mthd = (byte) UtilFunctions.toInt(scmthds.get(i)); 
+						
+				_allscaled[i] = colID;
+				
+				// check if the attribute is also MV imputed
+				int mvidx = isApplicable(colID);
+				if(mvidx != -1)
+				{
+					_isMVScaled.set(mvidx);
+					_mvscMethodList[mvidx] = MVMethod.values()[mthd];
+					_varList[mvidx] = new CM_COV_Object();
+				}
+				else
+					scnomv++;	// count of scaled but not imputed 
+			}
+			
+			if(scnomv > 0)
+			{
+				_scnomvList = new int[scnomv];			
+				_scnomvMethodList = new MVMethod[scnomv];	
+	
+				_scnomvMeanList = new KahanObject[scnomv];
+				_scnomvCountList = new long[scnomv];
+				_scnomvVarList = new CM_COV_Object[scnomv];
+				
+				for(int i=0, idx=0; i < scLength; i++)
+				{
+					colID = UtilFunctions.toInt(scattrs.get(i));
+					mthd = (byte)UtilFunctions.toInt(scmthds.get(i)); 
+							
+					if(isApplicable(colID) == -1)
+					{	// scaled but not imputed
+						_scnomvList[idx] = colID;
+						_scnomvMethodList[idx] = MVMethod.values()[mthd];
+						_scnomvMeanList[idx] = new KahanObject(0, 0);
+						_scnomvVarList[idx] = new CM_COV_Object();
+						idx++;
+					}
+				}
+			}
+		}
+	}
+
+	private void parseMethodsAndReplacments(JSONObject parsedSpec) throws JSONException {
+		JSONArray mvspec = (JSONArray) parsedSpec.get(TfUtils.TXMETHOD_IMPUTE);
+		_mvMethodList = new MVMethod[mvspec.size()];
+		_replacementList = new String[mvspec.size()];
+		_meanList = new KahanObject[mvspec.size()];
+		_countList = new long[mvspec.size()];
+		for(int i=0; i < mvspec.size(); i++) {
+			JSONObject mvobj = (JSONObject)mvspec.get(i);
+			_mvMethodList[i] = MVMethod.valueOf(mvobj.get("method").toString().toUpperCase()); 
+			if( _mvMethodList[i] == MVMethod.CONSTANT ) {
+				_replacementList[i] = mvobj.getString("value").toString();
+			}
+			_meanList[i] = new KahanObject(0, 0);
+		}
+	}
+		
+	public void prepare(String[] words) throws IOException {
+		
+		try {
+			String w = null;
+			if(_colList != null)
+			for(int i=0; i <_colList.length; i++) {
+				int colID = _colList[i];
+				w = UtilFunctions.unquote(words[colID-1].trim());
+				
+				try {
+				if(!TfUtils.isNA(_NAstrings, w)) {
+					_countList[i]++;
+					
+					boolean computeMean = (_mvMethodList[i] == MVMethod.GLOBAL_MEAN || _isMVScaled.get(i) );
+					if(computeMean) {
+						// global_mean
+						double d = UtilFunctions.parseToDouble(w);
+						_meanFn.execute2(_meanList[i], d, _countList[i]);
+						
+						if (_isMVScaled.get(i) && _mvscMethodList[i] == MVMethod.GLOBAL_MODE)
+							_varFn.execute(_varList[i], d);
+					}
+					else {
+						// global_mode or constant
+						// Nothing to do here. Mode is computed using recode maps.
+					}
+				}
+				} catch (NumberFormatException e) 
+				{
+					throw new RuntimeException("Encountered \"" + w + "\" in column ID \"" + colID + "\", when expecting a numeric value. Consider adding \"" + w + "\" to na.strings, along with an appropriate imputation method.");
+				}
+			}
+			
+			// Compute mean and variance for attributes that are scaled but not imputed
+			if(_scnomvList != null)
+			for(int i=0; i < _scnomvList.length; i++) 
+			{
+				int colID = _scnomvList[i];
+				w = UtilFunctions.unquote(words[colID-1].trim());
+				double d = UtilFunctions.parseToDouble(w);
+				_scnomvCountList[i]++; 		// not required, this is always equal to total #records processed
+				_meanFn.execute2(_scnomvMeanList[i], d, _scnomvCountList[i]);
+				if(_scnomvMethodList[i] == MVMethod.GLOBAL_MODE)
+					_varFn.execute(_scnomvVarList[i], d);
+			}
+		} catch(Exception e) {
+			throw new IOException(e);
+		}
+	}
+	
+	public MVMethod getMethod(int colID) {
+		int idx = isApplicable(colID);		
+		if(idx == -1)
+			return MVMethod.INVALID;
+		else
+			return _mvMethodList[idx];
+	}
+	
+	public long getNonMVCount(int colID) {
+		int idx = isApplicable(colID);
+		return (idx == -1) ? 0 : _countList[idx];
+	}
+	
+	public String getReplacement(int colID)  {
+		int idx = isApplicable(colID);		
+		return (idx == -1) ? null : _replacementList[idx];
+	}
+	
+	@Override
+	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
+		build(in);
+		return apply(in, out);
+	}
+	
+	@Override
+	public void build(FrameBlock in) {
+		try {
+			for( int j=0; j<_colList.length; j++ ) {
+				int colID = _colList[j];
+				if( _mvMethodList[j] == MVMethod.GLOBAL_MEAN ) {
+					//compute global column mean (scale)
+					long off = _countList[j];
+					for( int i=0; i<in.getNumRows(); i++ )
+						_meanFn.execute2(_meanList[j], UtilFunctions.objectToDouble(
+							in.getSchema()[colID-1], in.get(i, colID-1)), off+i+1);
+					_replacementList[j] = String.valueOf(_meanList[j]._sum);
+					_countList[j] += in.getNumRows();
+				}
+				else if( _mvMethodList[j] == MVMethod.GLOBAL_MODE ) {
+					//compute global column mode (categorical), i.e., most frequent category
+					HashMap<String,Long> hist = _hist.containsKey(colID) ? 
+							_hist.get(colID) : new HashMap<String,Long>();
+					for( int i=0; i<in.getNumRows(); i++ ) {
+						String key = String.valueOf(in.get(i, colID-1));
+						if( key != null && !key.isEmpty() ) {
+							Long val = hist.get(key);
+							hist.put(key, (val!=null) ? val+1 : 1);
+						}	
+					}
+					_hist.put(colID, hist);
+					long max = Long.MIN_VALUE; 
+					for( Entry<String, Long> e : hist.entrySet() ) 
+						if( e.getValue() > max  ) {
+							_replacementList[j] = e.getKey();
+							max = e.getValue();
+						}
+				}
+			}
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+
+	@Override
+	public String[] apply(String[] words) 
+	{	
+		if( isApplicable() )
+			for(int i=0; i < _colList.length; i++) {
+				int colID = _colList[i];
+				String w = UtilFunctions.unquote(words[colID-1]);
+				if(TfUtils.isNA(_NAstrings, w))
+					w = words[colID-1] = _replacementList[i];
+				
+				if ( _isMVScaled.get(i) )
+					if ( _mvscMethodList[i] == MVMethod.GLOBAL_MEAN )
+						words[colID-1] = Double.toString( UtilFunctions.parseToDouble(w) - _meanList[i]._sum );
+					else
+						words[colID-1] = Double.toString( (UtilFunctions.parseToDouble(w) - _meanList[i]._sum) / _varList[i].mean._sum );
+			}
+		
+		if(_scnomvList != null)
+		for(int i=0; i < _scnomvList.length; i++)
+		{
+			int colID = _scnomvList[i];
+			if ( _scnomvMethodList[i] == MVMethod.GLOBAL_MEAN )
+				words[colID-1] = Double.toString( UtilFunctions.parseToDouble(words[colID-1]) - _scnomvMeanList[i]._sum );
+			else
+				words[colID-1] = Double.toString( (UtilFunctions.parseToDouble(words[colID-1]) - _scnomvMeanList[i]._sum) / _scnomvVarList[i].mean._sum );
+		}
+			
+		return words;
+	}
+	
+	@Override
+	public MatrixBlock apply(FrameBlock in, MatrixBlock out) {
+		for(int i=0; i<in.getNumRows(); i++) {
+			for(int j=0; j<_colList.length; j++) {
+				int colID = _colList[j];
+				if( Double.isNaN(out.quickGetValue(i, colID-1)) )
+					out.quickSetValue(i, colID-1, Double.parseDouble(_replacementList[j]));
+			}
+		}
+		return out;
+	}
+	
+	@Override
+	public FrameBlock getMetaData(FrameBlock out) {
+		for( int j=0; j<_colList.length; j++ ) {
+			out.getColumnMetadata(_colList[j]-1)
+			   .setMvValue(_replacementList[j]);
+		}
+		return out;
+	}
+
+	public void initMetaData(FrameBlock meta) {
+		//init replacement lists, replace recoded values to
+		//apply mv imputation potentially after recoding
+		for( int j=0; j<_colList.length; j++ ) {
+			int colID = _colList[j];	
+			String mvVal = UtilFunctions.unquote(meta.getColumnMetadata(colID-1).getMvValue()); 
+			if( _rcList.contains(colID) ) {
+				Long mvVal2 = meta.getRecodeMap(colID-1).get(mvVal);
+				if( mvVal2 == null)
+					throw new RuntimeException("Missing recode value for impute value '"+mvVal+"' (colID="+colID+").");
+				_replacementList[j] = mvVal2.toString();
+			}
+			else {
+				_replacementList[j] = mvVal;
+			}
+		}
+	}
+
+	public void initRecodeIDList(List<Integer> rcList) {
+		_rcList = rcList;
+	}
+	
+	/**
+	 * Exposes the internal histogram after build.
+	 * 
+	 * @param colID column ID
+	 * @return histogram (map of string keys and long values)
+	 */
+	public HashMap<String,Long> getHistogram( int colID ) {
+		return _hist.get(colID);
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderOmit.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderOmit.java b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderOmit.java
new file mode 100644
index 0000000..af09cee
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderOmit.java
@@ -0,0 +1,123 @@
+/*
+ * 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.runtime.transform.encode;
+
+import org.apache.wink.json4j.JSONException;
+import org.apache.wink.json4j.JSONObject;
+import org.apache.sysml.runtime.matrix.data.FrameBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.transform.TfUtils;
+import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
+import org.apache.sysml.runtime.util.UtilFunctions;
+
+public class EncoderOmit extends Encoder 
+{	
+	private static final long serialVersionUID = 1978852120416654195L;
+
+	private int _rmRows = 0;
+
+	public EncoderOmit(JSONObject parsedSpec, String[] colnames, int clen) 
+		throws JSONException 
+	{
+		super(null, clen);
+		if (!parsedSpec.containsKey(TfUtils.TXMETHOD_OMIT))
+			return;
+		int[] collist = TfMetaUtils.parseJsonIDList(parsedSpec, colnames, TfUtils.TXMETHOD_OMIT);
+		initColList(collist);
+	}
+	
+	public int getNumRemovedRows() {
+		return _rmRows;
+	}
+	
+	public boolean omit(String[] words, TfUtils agents) 
+	{
+		if( !isApplicable() )
+			return false;
+		
+		for(int i=0; i<_colList.length; i++) {
+			int colID = _colList[i];
+			if(TfUtils.isNA(agents.getNAStrings(),UtilFunctions.unquote(words[colID-1].trim())))
+				return true;
+		}
+		return false;
+	}
+
+	@Override
+	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
+		return apply(in, out);
+	}
+	
+	@Override
+	public void build(FrameBlock in) {	
+		//do nothing
+	}
+	
+	@Override
+	public String[] apply(String[] words) {
+		return null;
+	}
+	
+	@Override
+	public MatrixBlock apply(FrameBlock in, MatrixBlock out) 
+	{
+		//determine output size
+		int numRows = 0;
+		for(int i=0; i<out.getNumRows(); i++) {
+			boolean valid = true;
+			for(int j=0; j<_colList.length; j++)
+				valid &= !Double.isNaN(out.quickGetValue(i, _colList[j]-1));
+			numRows += valid ? 1 : 0;
+		}
+		
+		//copy over valid rows into the output
+		MatrixBlock ret = new MatrixBlock(numRows, out.getNumColumns(), false);
+		int pos = 0;
+		for(int i=0; i<in.getNumRows(); i++) {
+			//determine if valid row or omit
+			boolean valid = true;
+			for(int j=0; j<_colList.length; j++)
+				valid &= !Double.isNaN(out.quickGetValue(i, _colList[j]-1));
+			//copy row if necessary
+			if( valid ) {
+				for(int j=0; j<out.getNumColumns(); j++)
+					ret.quickSetValue(pos, j, out.quickGetValue(i, j));
+				pos++;
+			}
+		}
+	
+		//keep info an remove rows
+		_rmRows = out.getNumRows() - pos;
+		
+		return ret; 
+	}
+
+	@Override
+	public FrameBlock getMetaData(FrameBlock out) {
+		//do nothing
+		return out;
+	}
+	
+	@Override
+	public void initMetaData(FrameBlock meta) {
+		//do nothing
+	}
+}
+ 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderPassThrough.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderPassThrough.java b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderPassThrough.java
index 08722fd..d84ea0d 100644
--- a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderPassThrough.java
+++ b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderPassThrough.java
@@ -19,19 +19,10 @@
 
 package org.apache.sysml.runtime.transform.encode;
 
-import java.io.IOException;
-import java.util.Iterator;
 
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.matrix.data.FrameBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.transform.DistinctValue;
-import org.apache.sysml.runtime.transform.TfUtils;
 import org.apache.sysml.runtime.util.UtilFunctions;
 
 /**
@@ -89,20 +80,4 @@ public class EncoderPassThrough extends Encoder
 	public void initMetaData(FrameBlock meta) {
 		//do nothing
 	}
-	
-	
-	@Override
-	public void mapOutputTransformationMetadata(OutputCollector<IntWritable, DistinctValue> out, int taskID, TfUtils agents) throws IOException {
-		throw new RuntimeException("File-based api not supported.");
-	}
-
-	@Override
-	public void mergeAndOutputTransformationMetadata(Iterator<DistinctValue> values, String outputDir, int colID, FileSystem fs, TfUtils agents) throws IOException {
-		throw new RuntimeException("File-based api not supported.");	
-	}
-
-	@Override
-	public void loadTxMtd(JobConf job, FileSystem fs, Path txMtdDir, TfUtils agents) throws IOException {
-		throw new RuntimeException("File-based api not supported.");
-	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderRecode.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderRecode.java b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderRecode.java
new file mode 100644
index 0000000..bb8592c
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/transform/encode/EncoderRecode.java
@@ -0,0 +1,253 @@
+/*
+ * 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.runtime.transform.encode;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+import org.apache.sysml.lops.Lop;
+import org.apache.sysml.runtime.matrix.data.FrameBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.transform.TfUtils;
+import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
+import org.apache.sysml.runtime.util.UtilFunctions;
+import org.apache.wink.json4j.JSONException;
+import org.apache.wink.json4j.JSONObject;
+
+public class EncoderRecode extends Encoder 
+{	
+	private static final long serialVersionUID = 8213163881283341874L;
+
+	private int[] _mvrcdList = null;
+	private int[] _fullrcdList = null;
+	
+	//recode maps and custom map for partial recode maps 
+	private HashMap<Integer, HashMap<String, Long>> _rcdMaps  = new HashMap<Integer, HashMap<String, Long>>();
+	private HashMap<Integer, HashMap<String,String>> _finalMaps = null;
+	private HashMap<Integer, HashSet<Object>> _rcdMapsPart = null;
+	
+	public EncoderRecode(JSONObject parsedSpec, String[] colnames, int clen)
+		throws JSONException 
+	{
+		super(null, clen);
+		int rcdCount = 0;
+		
+		if( parsedSpec.containsKey(TfUtils.TXMETHOD_RECODE) ) {
+			int[] collist = TfMetaUtils.parseJsonIDList(parsedSpec, colnames, TfUtils.TXMETHOD_RECODE);
+			rcdCount = initColList(collist);
+		}
+		
+		if ( parsedSpec.containsKey(TfUtils.TXMETHOD_MVRCD)) {
+			_mvrcdList = TfMetaUtils.parseJsonIDList(parsedSpec, colnames, TfUtils.TXMETHOD_MVRCD);
+			rcdCount += _mvrcdList.length;
+		}
+		
+		if ( rcdCount > 0 ) {
+			_fullrcdList = new int[rcdCount];
+			int idx = -1;
+			if(_colList != null)
+				for(int i=0; i < _colList.length; i++)
+					_fullrcdList[++idx] = _colList[i]; 
+			
+			if(_mvrcdList != null)
+				for(int i=0; i < _mvrcdList.length; i++)
+					_fullrcdList[++idx] = _mvrcdList[i]; 
+		}
+	}
+	
+	public HashMap<Integer, HashMap<String,Long>> getCPRecodeMaps() { 
+		return _rcdMaps; 
+	}
+	
+	public HashMap<Integer, HashSet<Object>> getCPRecodeMapsPartial() { 
+		return _rcdMapsPart; 
+	}
+	
+	public HashMap<Integer, HashMap<String,String>> getRecodeMaps() {
+		return _finalMaps;
+	}
+	
+	private String lookupRCDMap(int colID, String key) {
+		if( _finalMaps!=null )
+			return _finalMaps.get(colID).get(key);
+		else { //used for cp
+			Long tmp = _rcdMaps.get(colID).get(key);
+			return (tmp!=null) ? Long.toString(tmp) : null;
+		}
+	}
+	
+	@Override
+	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
+		if( !isApplicable() )
+			return out;
+		
+		//build and apply recode maps 
+		build(in);
+		apply(in, out);
+		
+		return out;
+	}
+
+	@Override
+	public void build(FrameBlock in) {
+		if( !isApplicable() )
+			return;		
+
+		Iterator<String[]> iter = in.getStringRowIterator();
+		while( iter.hasNext() ) {
+			String[] row = iter.next(); 
+			for( int j=0; j<_colList.length; j++ ) {
+				int colID = _colList[j]; //1-based
+				//allocate column map if necessary
+				if( !_rcdMaps.containsKey(colID) ) 
+					_rcdMaps.put(colID, new HashMap<String,Long>());
+				//probe and build column map
+				HashMap<String,Long> map = _rcdMaps.get(colID);
+				String key = row[colID-1];
+				if( key!=null && !key.isEmpty() && !map.containsKey(key) )
+					map.put(key, Long.valueOf(map.size()+1));
+			}
+		}
+	}
+
+	public void buildPartial(FrameBlock in) {
+		if( !isApplicable() )
+			return;		
+
+		//ensure allocated partial recode map
+		if( _rcdMapsPart == null )
+			_rcdMapsPart = new HashMap<Integer, HashSet<Object>>();
+		
+		//construct partial recode map (tokens w/o codes)
+		//iterate over columns for sequential access
+		for( int j=0; j<_colList.length; j++ ) {
+			int colID = _colList[j]; //1-based
+			//allocate column map if necessary
+			if( !_rcdMapsPart.containsKey(colID) ) 
+				_rcdMapsPart.put(colID, new HashSet<Object>());
+			HashSet<Object> map = _rcdMapsPart.get(colID);
+			//probe and build column map
+			for( int i=0; i<in.getNumRows(); i++ )
+				map.add(in.get(i, colID-1));
+			//cleanup unnecessary entries once
+			map.remove(null);
+			map.remove("");
+		}
+	}
+	
+	/**
+	 * Method to apply transformations.
+	 */
+	@Override
+	public String[] apply(String[] words) 
+	{
+		if( !isApplicable() )
+			return words;
+		
+		//apply recode maps on relevant columns of given row
+		for(int i=0; i < _colList.length; i++) {
+			//prepare input and get code
+			int colID = _colList[i];
+			String key = UtilFunctions.unquote(words[colID-1].trim());
+			String val = lookupRCDMap(colID, key);			
+			// replace unseen keys with NaN 
+			words[colID-1] = (val!=null) ? val : "NaN";
+		}
+			
+		return words;
+	}
+	
+	@Override
+	public MatrixBlock apply(FrameBlock in, MatrixBlock out) {
+		//apply recode maps column wise
+		for( int j=0; j<_colList.length; j++ ) {
+			int colID = _colList[j];
+			for( int i=0; i<in.getNumRows(); i++ ) {
+				Object okey = in.get(i, colID-1);
+				String key = (okey!=null) ? okey.toString() : null;
+				String val = lookupRCDMap(colID, key);			
+				out.quickSetValue(i, colID-1, (val!=null) ? 
+						Double.parseDouble(val) : Double.NaN);
+			}
+		}
+		
+		return out;
+	}
+
+	@Override
+	public FrameBlock getMetaData(FrameBlock meta) {
+		if( !isApplicable() )
+			return meta;
+		
+		//inverse operation to initRecodeMaps
+		
+		//allocate output rows
+		int maxDistinct = 0;
+		for( int j=0; j<_colList.length; j++ )
+			if( _rcdMaps.containsKey(_colList[j]) )
+				maxDistinct = Math.max(maxDistinct, _rcdMaps.get(_colList[j]).size());
+		meta.ensureAllocatedColumns(maxDistinct);
+		
+		//create compact meta data representation
+		for( int j=0; j<_colList.length; j++ ) {
+			int colID = _colList[j]; //1-based
+			int rowID = 0;
+			if( _rcdMaps.containsKey(_colList[j]) )
+				for( Entry<String, Long> e : _rcdMaps.get(colID).entrySet() ) {
+					String tmp = constructRecodeMapEntry(e.getKey(), e.getValue());
+					meta.set(rowID++, colID-1, tmp); 
+				}
+			meta.getColumnMetadata(colID-1).setNumDistinct(
+					_rcdMaps.get(colID).size());
+		}
+		
+		return meta;
+	}
+	
+
+	/**
+	 * Construct the recodemaps from the given input frame for all 
+	 * columns registered for recode.
+	 * 
+	 * @param meta frame block
+	 */
+	public void initMetaData( FrameBlock meta ) {
+		if( meta == null || meta.getNumRows()<=0 )
+			return;
+		
+		for( int j=0; j<_colList.length; j++ ) {
+			int colID = _colList[j]; //1-based
+			_rcdMaps.put(colID, meta.getRecodeMap(colID-1));
+		}
+	}
+	
+	/**
+	 * Returns the Recode map entry which consists of concatenation of code, delimiter and token. 
+	 * @param token	is part of Recode map
+	 * @param code  is code for token 
+	 * @return the concatenation of code and token with delimiter in between
+	 */
+	public static String constructRecodeMapEntry(String token, Long code) {
+		return token + Lop.DATATYPE_PREFIX + code.toString();
+	}
+}
+ 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/meta/TfMetaUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/meta/TfMetaUtils.java b/src/main/java/org/apache/sysml/runtime/transform/meta/TfMetaUtils.java
index 62b90b4..afb7ee9 100644
--- a/src/main/java/org/apache/sysml/runtime/transform/meta/TfMetaUtils.java
+++ b/src/main/java/org/apache/sysml/runtime/transform/meta/TfMetaUtils.java
@@ -50,7 +50,6 @@ import org.apache.wink.json4j.JSONObject;
 
 public class TfMetaUtils 
 {
-
 	public static boolean isIDSpecification(String spec) throws DMLRuntimeException {
 		try {
 			JSONObject jSpec = new JSONObject(spec);

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameFunctionTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameFunctionTest.java b/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameFunctionTest.java
index af2e75f..b506444 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameFunctionTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameFunctionTest.java
@@ -88,9 +88,7 @@ public class FrameFunctionTest extends AutomatedTestBase
 			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
 	
 		boolean oldIPA = OptimizerUtils.ALLOW_INTER_PROCEDURAL_ANALYSIS;
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
 		OptimizerUtils.ALLOW_INTER_PROCEDURAL_ANALYSIS = IPA;
-		OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
 		
 		try
 		{
@@ -126,7 +124,6 @@ public class FrameFunctionTest extends AutomatedTestBase
 			rtplatform = platformOld;
 			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
 			OptimizerUtils.ALLOW_INTER_PROCEDURAL_ANALYSIS = oldIPA;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMatrixReblockTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMatrixReblockTest.java b/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMatrixReblockTest.java
index ecc958b..c629eee 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMatrixReblockTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMatrixReblockTest.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.lops.LopProperties.ExecType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.io.FrameWriter;
@@ -201,10 +200,6 @@ public class FrameMatrixReblockTest extends AutomatedTestBase
 		if( rtplatform == RUNTIME_PLATFORM.SPARK )
 			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
 		
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
-		if( ofmt.equals("csv") )
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
-		
 		try
 		{
 			int cols = multColBlks ? cols2 : cols1;
@@ -235,7 +230,6 @@ public class FrameMatrixReblockTest extends AutomatedTestBase
 		finally {
 			rtplatform = platformOld;
 			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMetaReadWriteTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMetaReadWriteTest.java b/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMetaReadWriteTest.java
index 5066582..ceeec07 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMetaReadWriteTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/frame/FrameMetaReadWriteTest.java
@@ -21,7 +21,6 @@ package org.apache.sysml.test.integration.functions.frame;
 
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.lops.LopProperties.ExecType;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.io.FrameReaderFactory;
@@ -101,10 +100,6 @@ public class FrameMetaReadWriteTest extends AutomatedTestBase
 			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
 	
 		String ofmt = OutputInfo.outputInfoToStringExternal(oinfo);
-
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
-		if( ofmt.equals("csv") )
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
 		
 		try
 		{
@@ -148,7 +143,6 @@ public class FrameMetaReadWriteTest extends AutomatedTestBase
 		finally {
 			rtplatform = platformOld;
 			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/FrameCSVReadWriteTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/FrameCSVReadWriteTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/FrameCSVReadWriteTest.java
index 35078f3..056e619 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/FrameCSVReadWriteTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/transform/FrameCSVReadWriteTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.functions.transform;
 import org.junit.Test;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.io.FrameReader;
 import org.apache.sysml.runtime.io.FrameReaderFactory;
 import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
@@ -75,7 +74,6 @@ public class FrameCSVReadWriteTest extends AutomatedTestBase
 	{
 		//set runtime platform
 		RUNTIME_PLATFORM rtold = rtplatform;
-		boolean csvReblockOld = OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK;
 		rtplatform = rt;
 
 		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
@@ -94,7 +92,6 @@ public class FrameCSVReadWriteTest extends AutomatedTestBase
 			programArgs = new String[]{"-explain","-args", 
 				HOME + "input/" + DATASET, output("R") };
 	
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = true;
 			runTest(true, false, null, -1); 
 			
 			//read input/output and compare
@@ -113,7 +110,6 @@ public class FrameCSVReadWriteTest extends AutomatedTestBase
 		finally {
 			rtplatform = rtold;
 			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-			OptimizerUtils.ALLOW_FRAME_CSV_REBLOCK = csvReblockOld;
 		}
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/test/java/org/apache/sysml/test/integration/functions/transform/RunTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/transform/RunTest.java b/src/test/java/org/apache/sysml/test/integration/functions/transform/RunTest.java
deleted file mode 100644
index 81c0bab..0000000
--- a/src/test/java/org/apache/sysml/test/integration/functions/transform/RunTest.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * 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.transform;
-
-import java.io.IOException;
-
-import org.junit.Test;
-
-import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.test.integration.AutomatedTestBase;
-import org.apache.sysml.test.integration.TestConfiguration;
-import org.apache.sysml.test.utils.TestUtils;
-
-/**
- * 
- * 
- */
-public class RunTest extends AutomatedTestBase 
-{
-	
-	private final static String TEST_NAME1 = "Transform";
-	private final static String TEST_NAME2 = "Apply";
-	private final static String TEST_DIR = "functions/transform/";
-	private final static String TEST_CLASS_DIR = TEST_DIR + RunTest.class.getSimpleName() + "/";
-	
-	private final static String HOMES_DATASET 	= "homes/homes.csv";
-	//private final static String HOMES_SPEC 		= "homes/homes.tfspec.json";
-	private final static String HOMES_SPEC2 	= "homes/homes.tfspec2.json";
-	//private final static String HOMES_IDSPEC 	= "homes/homes.tfidspec.json";
-	//private final static String HOMES_TFDATA 	= "homes/homes.transformed.csv";
-	//private final static String HOMES_COLNAMES 	= "homes/homes.csv.colnames";
-	
-	private final static String HOMES_NAN_DATASET 	= "homes/homesNAN.csv";
-	private final static String HOMES_NAN_SPEC 		= "homes/homesNAN.tfspec.json";
-	//private final static String HOMES_NAN_IDSPEC 	= "homes/homesNAN.tfidspec.json";
-	private final static String HOMES_NAN_COLNAMES 	= "homes/homesNAN.colnames.csv";
-	
-	private final static String HOMES_MISSING_DATASET 	= "homes/homesAllMissing.csv";
-	private final static String HOMES_MISSING_SPEC 		= "homes/homesAllMissing.tfspec.json";
-	private final static String HOMES_MISSING_IDSPEC 	= "homes/homesAllMissing.tfidspec.json";
-	
-	@Override
-	public void setUp() 
-	{
-		TestUtils.clearAssertionInformation();
-		addTestConfiguration(TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1,new String[]{"R"}));
-	}
-	
-	// ---- NAN BinaryBlock ----
-	
-	@Test
-	public void runTestWithNAN_HybridBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_NAN_DATASET, HOMES_NAN_SPEC, HOMES_NAN_COLNAMES, false, RUNTIME_PLATFORM.HYBRID, "binary");
-	}
-
-	@Test
-	public void runTestWithNAN_SPHybridBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_NAN_DATASET, HOMES_NAN_SPEC, HOMES_NAN_COLNAMES, false, RUNTIME_PLATFORM.HYBRID_SPARK, "binary");
-	}
-
-	@Test
-	public void runTestWithNAN_HadoopBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_NAN_DATASET, HOMES_NAN_SPEC, HOMES_NAN_COLNAMES, false, RUNTIME_PLATFORM.HADOOP, "binary");
-	}
-
-	@Test
-	public void runTestWithNAN_SparkBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_NAN_DATASET, HOMES_NAN_SPEC, HOMES_NAN_COLNAMES, false, RUNTIME_PLATFORM.SPARK, "binary");
-	}
-
-	// ---- NAN CSV ----
-	
-	@Test
-	public void runTestWithNAN_HybridCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_NAN_DATASET, HOMES_NAN_SPEC, HOMES_NAN_COLNAMES, false, RUNTIME_PLATFORM.HYBRID, "csv");
-	}
-
-	@Test
-	public void runTestWithNAN_SPHybridCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_NAN_DATASET, HOMES_NAN_SPEC, HOMES_NAN_COLNAMES, false, RUNTIME_PLATFORM.HYBRID_SPARK, "csv");
-	}
-
-	@Test
-	public void runTestWithNAN_HadoopCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_NAN_DATASET, HOMES_NAN_SPEC, HOMES_NAN_COLNAMES, false, RUNTIME_PLATFORM.HADOOP, "csv");
-	}
-
-	@Test
-	public void runTestWithNAN_SparkCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_NAN_DATASET, HOMES_NAN_SPEC, HOMES_NAN_COLNAMES, false, RUNTIME_PLATFORM.SPARK, "csv");
-	}
-
-	// ---- Test2 BinaryBlock ----
-	
-	@Test
-	public void runTest2_HybridBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_DATASET, HOMES_SPEC2, null, false, RUNTIME_PLATFORM.HYBRID, "binary");
-	}
-
-	@Test
-	public void runTest2_SPHybridBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_DATASET, HOMES_SPEC2, null, false, RUNTIME_PLATFORM.HYBRID_SPARK, "binary");
-	}
-
-	@Test
-	public void runTest2_HadoopBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_DATASET, HOMES_SPEC2, null, false, RUNTIME_PLATFORM.HADOOP, "binary");
-	}
-
-	@Test
-	public void runTest2_SparkBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_DATASET, HOMES_SPEC2, null, false, RUNTIME_PLATFORM.SPARK, "binary");
-	}
-
-	// ---- Test2 CSV ----
-	
-	@Test
-	public void runTest2_HybridCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_DATASET, HOMES_SPEC2, null, false, RUNTIME_PLATFORM.HYBRID, "csv");
-	}
-
-	@Test
-	public void runTest2_SPHybridCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_DATASET, HOMES_SPEC2, null, false, RUNTIME_PLATFORM.HYBRID_SPARK, "csv");
-	}
-
-	@Test
-	public void runTest2_HadoopCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_DATASET, HOMES_SPEC2, null, false, RUNTIME_PLATFORM.HADOOP, "csv");
-	}
-
-	@Test
-	public void runTest2_SparkCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_DATASET, HOMES_SPEC2, null, false, RUNTIME_PLATFORM.SPARK, "csv");
-	}
-
-	// ---- HomesMissing BinaryBlock ----
-	
-	@Test
-	public void runAllMissing_HybridBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_MISSING_DATASET, HOMES_MISSING_SPEC, null, true, RUNTIME_PLATFORM.HYBRID, "binary");
-	}
-
-	@Test
-	public void runAllMissing_SPHybridBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_MISSING_DATASET, HOMES_MISSING_SPEC, null, true, RUNTIME_PLATFORM.HYBRID_SPARK, "binary");
-	}
-
-	@Test
-	public void runAllMissing_HadoopBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_MISSING_DATASET, HOMES_MISSING_SPEC, null, true, RUNTIME_PLATFORM.HADOOP, "binary");
-	}
-
-	@Test
-	public void runAllMissing_SparkBB() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_MISSING_DATASET, HOMES_MISSING_SPEC, null, true, RUNTIME_PLATFORM.SPARK, "binary");
-	}
-
-	// ---- HomesMissing CSV ----
-	
-	@Test
-	public void runAllMissing_HybridCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_MISSING_DATASET, HOMES_MISSING_IDSPEC, null, true, RUNTIME_PLATFORM.HYBRID, "csv");
-	}
-
-	@Test
-	public void runAllMissing_SPHybridCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_MISSING_DATASET, HOMES_MISSING_IDSPEC, null, true, RUNTIME_PLATFORM.HYBRID_SPARK, "csv");
-	}
-
-	@Test
-	public void runAllMissing_HadoopCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_MISSING_DATASET, HOMES_MISSING_IDSPEC, null, true, RUNTIME_PLATFORM.HADOOP, "csv");
-	}
-
-	@Test
-	public void runAllMissing_SparkCSV() throws DMLRuntimeException, IOException {
-		runScalingTest( HOMES_MISSING_DATASET, HOMES_MISSING_IDSPEC, null, true, RUNTIME_PLATFORM.SPARK, "csv");
-	}
-
-	// ------------------
-	
-	/**
-	 * 
-	 * @param sparseM1
-	 * @param sparseM2
-	 * @param instType
-	 * @throws IOException 
-	 * @throws DMLRuntimeException 
-	 */
-	private void runScalingTest( String dataset, String spec, String colnames, boolean exception, RUNTIME_PLATFORM rt, String ofmt) throws IOException, DMLRuntimeException
-	{
-		RUNTIME_PLATFORM platformOld = rtplatform;
-		rtplatform = rt;
-	
-		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
-		if( rtplatform == RUNTIME_PLATFORM.SPARK  || rtplatform == RUNTIME_PLATFORM.HYBRID_SPARK)
-			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
-
-		try
-		{
-			getAndLoadTestConfiguration(TEST_NAME1);
-			
-			/* This is for running the junit test the new way, i.e., construct the arguments directly */
-			String HOME = SCRIPT_DIR + TEST_DIR;
-			fullDMLScriptName = null;
-			
-			if (colnames == null) {
-				fullDMLScriptName  = HOME + TEST_NAME1 + ".dml";
-				programArgs = new String[]{"-nvargs", 
-											"DATA=" + HOME + "input/" + dataset,
-											"TFSPEC=" + HOME + "input/" + spec,
-											"TFMTD=" + output("tfmtd"),
-											"TFDATA=" + output("tfout"),
-											"OFMT=" + ofmt };
-			}
-			else {
-				fullDMLScriptName  = HOME + TEST_NAME1 + "_colnames.dml";
-				programArgs = new String[]{"-nvargs", 
-											"DATA=" + HOME + "input/" + dataset,
-											"TFSPEC=" + HOME + "input/" + spec,
-											"COLNAMES=" + HOME + "input/" + colnames,
-											"TFMTD=" + output("tfmtd"),
-											"TFDATA=" + output("tfout"),
-											"OFMT=" + ofmt };
-			}
-	
-			boolean exceptionExpected = exception;
-			runTest(true, exceptionExpected, null, -1); 
-			
-			fullDMLScriptName = HOME + TEST_NAME2 + ".dml";
-			programArgs = new String[]{"-nvargs", 
-											"DATA=" + HOME + "input/" + dataset,
-											"APPLYMTD=" + output("tfmtd"),  // generated above
-											"TFMTD=" + output("test_tfmtd"),
-											"TFDATA=" + output("test_tfout"),
-											"OFMT=" + ofmt };
-	
-			exceptionExpected = exception;
-			runTest(true, exceptionExpected, null, -1); 
-			
-		}
-		finally
-		{
-			rtplatform = platformOld;
-			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-		}
-	}	
-}
\ No newline at end of file


[4/6] systemml git commit: [SYSTEMML-1300] Remove file-based transform from compiler/runtime

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/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
deleted file mode 100644
index 4a7839d..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/DataTransform.java
+++ /dev/null
@@ -1,1496 +0,0 @@
-/*
- * 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.runtime.transform;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.wink.json4j.JSONArray;
-import org.apache.wink.json4j.JSONException;
-import org.apache.wink.json4j.JSONObject;
-
-import scala.Tuple2;
-
-import org.apache.sysml.conf.ConfigurationManager;
-import org.apache.sysml.conf.DMLConfig;
-import org.apache.sysml.lops.CSVReBlock;
-import org.apache.sysml.lops.Lop;
-import org.apache.sysml.lops.LopProperties.ExecType;
-import org.apache.sysml.parser.Expression.DataType;
-import org.apache.sysml.parser.Expression.ValueType;
-import org.apache.sysml.parser.ParameterizedBuiltinFunctionExpression;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.controlprogram.caching.CacheableData;
-import org.apache.sysml.runtime.controlprogram.caching.FrameObject;
-import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
-import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
-import org.apache.sysml.runtime.instructions.InstructionParser;
-import org.apache.sysml.runtime.instructions.MRJobInstruction;
-import org.apache.sysml.runtime.instructions.cp.ParameterizedBuiltinCPInstruction;
-import org.apache.sysml.runtime.instructions.mr.CSVReblockInstruction;
-import org.apache.sysml.runtime.instructions.spark.ParameterizedBuiltinSPInstruction;
-import org.apache.sysml.runtime.instructions.spark.data.RDDObject;
-import org.apache.sysml.runtime.instructions.spark.utils.RDDConverterUtils;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.matrix.CSVReblockMR;
-import org.apache.sysml.runtime.matrix.CSVReblockMR.AssignRowIDMRReturn;
-import org.apache.sysml.runtime.matrix.JobReturn;
-import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
-import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
-import org.apache.sysml.runtime.matrix.data.FileFormatProperties;
-import org.apache.sysml.runtime.matrix.data.InputInfo;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.matrix.data.OutputInfo;
-import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration;
-import org.apache.sysml.runtime.util.MapReduceTool;
-import org.apache.sysml.runtime.util.UtilFunctions;
-import org.apache.sysml.utils.JSONHelper;
-
-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.
-	 * 
-	 * @param fs file system
-	 * @param prop csv file format properties
-	 * @param smallestFile file name
-	 * @return header line
-	 * @throws IOException if IOException occurs
-	 */
-	private static String readHeaderLine(FileSystem fs, CSVFileFormatProperties prop, String smallestFile) throws IOException {
-		String line = null;
-		try( BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(new Path(smallestFile)))) ) {
-			line = br.readLine();
-		}
-		if(prop.hasHeader()) {
-			; // nothing here
-		}
-		else 
-		{
-			// construct header with default column names, V1, V2, etc.
-			int ncol = Pattern.compile( Pattern.quote(prop.getDelim()) ).split(line, -1).length;
-			line = null;
-			
-			StringBuilder sb = new StringBuilder();
-			sb.append("V1");
-			for(int i=2; i <= ncol; i++)
-				sb.append(prop.getDelim() + "V" + i);
-			line = sb.toString();
-		}
-		return line;
-	}
-	
-	/**
-	 * Method to construct a mapping between column names and their
-	 * corresponding column IDs. The mapping is used to prepare the
-	 * specification file in <code>processSpecFile()</code>.
-	 * 
-	 * @param fs file system
-	 * @param prop csv file format properties
-	 * @param headerLine header line
-	 * @param smallestFile file name
-	 * @return map of column names and their column IDs
-	 * @throws IllegalArgumentException if IllegalArgumentException occurs
-	 * @throws IOException if IOException occurs
-	 */
-	private static HashMap<String, Integer> processColumnNames(FileSystem fs, CSVFileFormatProperties prop, String headerLine, String smallestFile) throws IllegalArgumentException, IOException {
-		HashMap<String, Integer> colNames = new HashMap<String,Integer>();
-		
-		String escapedDelim = Pattern.quote(prop.getDelim());
-		Pattern compiledDelim = Pattern.compile(escapedDelim);
-		String[] names = compiledDelim.split(headerLine, -1);
-			
-		for(int i=0; i< names.length; i++)
-			colNames.put(UtilFunctions.unquote(names[i].trim()), i+1);
-
-		return colNames;
-	}
-	
-	/**
-	 * In-place permutation of list, mthd, and cst arrays based on indices,
-	 * by navigating through cycles in the permutation. 
-	 * 
-	 * @param list ?
-	 * @param mthd ?
-	 * @param cst ?
-	 * @param indices ?
-	 */
-	private static void inplacePermute(int[] list, byte[] mthd, Object[] cst, Integer[] indices) 
-	{
-		int x;
-		byte xb = 0;
-		Object xo = null;
-		
-		int j, k;
-		for(int i=0; i < list.length; i++) 
-		{
-		    x = list[i];
-		    xb = mthd[i];
-		    if ( cst != null )  xo = cst[i];
-		    
-		    j = i;
-		    while(true) {
-		        k = indices[j];
-		        indices[j] = j;
-		        
-		        if (k == i)
-		            break;
-		        
-		        list[j] = list[k];
-		        mthd[j] = mthd[k]; 
-		        if ( cst != null )  cst[j] = cst[k]; 
-		        j = k;
-		    }
-		    list[j] = x;
-	        mthd[j] = xb; 
-	        if ( cst != null )  cst[j] = xo; 
-		}
-
-	}
-	
-	/**
-	 * Convert input transformation specification file with column names into a
-	 * specification with corresponding column Ids. This file is sent to all the
-	 * relevant MR jobs.
-	 * 
-	 * @param fs file system
-	 * @param inputPath input file path
-	 * @param smallestFile file name
-	 * @param colNames column names
-	 * @param prop csv file format properties
-	 * @param specFileWithNames ?
-	 * @return specification as a JSONObject
-	 * @throws IllegalArgumentException if IllegalArgumentException occurs
-	 * @throws IOException if IOException occurs
-	 * @throws JSONException if JSONException occurs
-	 */
-	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";
-		final String METHOD = "method";
-		final String VALUE = "value";
-		final String MV_METHOD_MEAN = "global_mean";
-		final String MV_METHOD_MODE = "global_mode";
-		final String MV_METHOD_CONSTANT = "constant";
-		final String BIN_METHOD_WIDTH = "equi-width";
-		final String BIN_METHOD_HEIGHT = "equi-height";
-		final String SCALE_METHOD_Z = "z-score";
-		final String SCALE_METHOD_M = "mean-subtraction";
-		final String JSON_BYPOS = "ids";
-		
-		String stmp = null;
-		JSONObject entry = null;
-		byte btmp = 0;
-		
-		final int[] mvList;
-		int[] rcdList, dcdList, omitList;
-		final int[] binList;
-		final int[] scaleList;
-		byte[] mvMethods = null, binMethods=null, scaleMethods=null;
-		Object[] numBins = null;
-		Object[] mvConstants = null;
-		
-		boolean byPositions = (inputSpec.containsKey(JSON_BYPOS) && ((Boolean)inputSpec.get(JSON_BYPOS)).booleanValue() == true);
-		
-		// --------------------------------------------------------------------------
-		// Omit
-		if( inputSpec.containsKey(TfUtils.TXMETHOD_OMIT) ) {
-			JSONArray arrtmp = (JSONArray) inputSpec.get(TfUtils.TXMETHOD_OMIT);
-			omitList = new int[arrtmp.size()];
-			for(int i=0; i<arrtmp.size(); i++) {
-				if(byPositions)
-					omitList[i] = UtilFunctions.toInt( arrtmp.get(i) );
-				else {
-					stmp = UtilFunctions.unquote( (String)arrtmp.get(i) );
-					omitList[i] = colNames.get(stmp);
-				}
-			}
-			Arrays.sort(omitList);
-		}
-		else
-			omitList = null;
-		// --------------------------------------------------------------------------
-		// Missing value imputation
-		if( inputSpec.containsKey(TfUtils.TXMETHOD_IMPUTE) ) {
-			JSONArray arrtmp = (JSONArray) inputSpec.get(TfUtils.TXMETHOD_IMPUTE);
-			
-			mvList = new int[arrtmp.size()];
-			mvMethods = new byte[arrtmp.size()];
-			mvConstants = new Object[arrtmp.size()];
-			
-			for(int i=0; i<arrtmp.size(); i++) {
-				entry = (JSONObject)arrtmp.get(i);
-				if (byPositions) {
-					mvList[i] = UtilFunctions.toInt(entry.get(ID));
-				}
-				else {
-					stmp = UtilFunctions.unquote((String) entry.get(NAME));
-					mvList[i] = colNames.get(stmp);
-				}
-				
-				stmp = UtilFunctions.unquote((String) entry.get(METHOD));
-				if(stmp.equals(MV_METHOD_MEAN))
-					btmp = (byte)1;
-				else if ( stmp.equals(MV_METHOD_MODE))
-					btmp = (byte)2;
-				else if ( stmp.equals(MV_METHOD_CONSTANT))
-					btmp = (byte)3;
-				else
-					throw new IOException("Unknown missing value imputation method (" + stmp + ") in transformation specification: " + specWithNames);
-				mvMethods[i] = btmp;
-				
-				//txMethods.add( btmp );
-				
-				mvConstants[i] = null;
-				if ( entry.containsKey(VALUE) )
-					mvConstants[i] = entry.get(VALUE);
-			}
-			
-			Integer[] idx = new Integer[mvList.length];
-			for(int i=0; i < mvList.length; i++)
-				idx[i] = i;
-			Arrays.sort(idx, new Comparator<Integer>() {
-				@Override
-				public int compare(Integer o1, Integer o2) {
-					return (mvList[o1]-mvList[o2]);
-				}
-			});
-			
-			// rearrange mvList, mvMethods, and mvConstants according to permutation idx
-			inplacePermute(mvList, mvMethods, mvConstants, idx);
-		}
-		else
-			mvList = null;
-		// --------------------------------------------------------------------------
-		// Recoding
-		if( inputSpec.containsKey(TfUtils.TXMETHOD_RECODE) ) {
-			JSONArray arrtmp = (JSONArray) inputSpec.get(TfUtils.TXMETHOD_RECODE);
-			rcdList = new int[arrtmp.size()];
-			for(int i=0; i<arrtmp.size(); i++) {
-				if (byPositions)
-					rcdList[i] = UtilFunctions.toInt(arrtmp.get(i));
-				else {
-					stmp = UtilFunctions.unquote( (String)arrtmp.get(i) );
-					rcdList[i] = colNames.get(stmp);
-				}
-			}
-			Arrays.sort(rcdList);
-		}
-		else
-			rcdList = null;
-		// --------------------------------------------------------------------------
-		// Binning
-		if( inputSpec.containsKey(TfUtils.TXMETHOD_BIN) ) {
-			JSONArray arrtmp = (JSONArray) inputSpec.get(TfUtils.TXMETHOD_BIN);
-			
-			binList = new int[arrtmp.size()];
-			binMethods = new byte[arrtmp.size()];
-			numBins = new Object[arrtmp.size()];
-			
-			for(int i=0; i<arrtmp.size(); i++) {
-				entry = (JSONObject)arrtmp.get(i);
-				
-				if (byPositions) {
-					binList[i] = UtilFunctions.toInt(entry.get(ID));
-				}
-				else {
-					stmp = UtilFunctions.unquote((String) entry.get(NAME));
-					binList[i] = colNames.get(stmp);
-				}
-				stmp = UtilFunctions.unquote((String) entry.get(METHOD));
-				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: " + specWithNames);
-				else
-					throw new IOException("Unknown missing value imputation method (" + stmp + ") in transformation specification: " + specWithNames);
-				binMethods[i] = btmp;
-				
-				numBins[i] = entry.get(TfUtils.JSON_NBINS);
-				if ( ((Integer) numBins[i]).intValue() <= 1 ) 
-					throw new IllegalArgumentException("Invalid transformation on column \"" + (String) entry.get(NAME) + "\". Number of bins must be greater than 1.");
-			}
-			
-			Integer[] idx = new Integer[binList.length];
-			for(int i=0; i < binList.length; i++)
-				idx[i] = i;
-			Arrays.sort(idx, new Comparator<Integer>() {
-				@Override
-				public int compare(Integer o1, Integer o2) {
-					return (binList[o1]-binList[o2]);
-				}
-			});
-			
-			// rearrange binList and binMethods according to permutation idx
-			inplacePermute(binList, binMethods, numBins, idx);
-		}
-		else
-			binList = null;
-		// --------------------------------------------------------------------------
-		// Dummycoding
-		if( inputSpec.containsKey(TfUtils.TXMETHOD_DUMMYCODE) ) {
-			JSONArray arrtmp = (JSONArray) inputSpec.get(TfUtils.TXMETHOD_DUMMYCODE);
-			dcdList = new int[arrtmp.size()];
-			for(int i=0; i<arrtmp.size(); i++) {
-				if (byPositions)
-					dcdList[i] = UtilFunctions.toInt(arrtmp.get(i));
-				else {
-					stmp = UtilFunctions.unquote( (String)arrtmp.get(i) );
-					dcdList[i] = colNames.get(stmp);
-				}
-			}
-			Arrays.sort(dcdList);
-		}
-		else
-			dcdList = null;
-		// --------------------------------------------------------------------------
-		// Scaling
-		if(inputSpec.containsKey(TfUtils.TXMETHOD_SCALE) ) {
-			JSONArray arrtmp = (JSONArray) inputSpec.get(TfUtils.TXMETHOD_SCALE);
-			
-			scaleList = new int[arrtmp.size()];
-			scaleMethods = new byte[arrtmp.size()];
-			
-			for(int i=0; i<arrtmp.size(); i++) {
-				entry = (JSONObject)arrtmp.get(i);
-				
-				if (byPositions) {
-					scaleList[i] = UtilFunctions.toInt(entry.get(ID));
-				}
-				else {
-					stmp = UtilFunctions.unquote((String) entry.get(NAME));
-					scaleList[i] = colNames.get(stmp);
-				}
-				stmp = UtilFunctions.unquote((String) entry.get(METHOD));
-				if(stmp.equals(SCALE_METHOD_M))
-					btmp = (byte)1;
-				else if ( stmp.equals(SCALE_METHOD_Z))
-					btmp = (byte)2;
-				else
-					throw new IOException("Unknown missing value imputation method (" + stmp + ") in transformation specification: " + specWithNames);
-				scaleMethods[i] = btmp;
-			}
-			
-			Integer[] idx = new Integer[scaleList.length];
-			for(int i=0; i < scaleList.length; i++)
-				idx[i] = i;
-			Arrays.sort(idx, new Comparator<Integer>() {
-				@Override
-				public int compare(Integer o1, Integer o2) {
-					return (scaleList[o1]-scaleList[o2]);
-				}
-			});
-			
-			// rearrange scaleList and scaleMethods according to permutation idx
-			inplacePermute(scaleList, scaleMethods, null, idx);
-		}
-		else
-			scaleList = null;
-		// --------------------------------------------------------------------------
-		
-		// check for column IDs that are imputed with mode, but not recoded
-		// These columns have be handled separately, because the computation of mode 
-		// requires the computation of distinct values (i.e., recode maps)
-		ArrayList<Integer> tmpList = new ArrayList<Integer>();
-		if(mvList != null)
-		for(int i=0; i < mvList.length; i++) {
-			int colID = mvList[i];
-			if(mvMethods[i] == 2 && (rcdList == null || Arrays.binarySearch(rcdList, colID) < 0) )
-				tmpList.add(colID);
-		}
-		
-		int[] mvrcdList = null;
-		if ( tmpList.size() > 0 ) {
-			mvrcdList = new int[tmpList.size()];
-			for(int i=0; i < tmpList.size(); i++)
-				mvrcdList[i] = tmpList.get(i);
-		}
-		// Perform Validity Checks
-		
-		/*
-			      OMIT MVI RCD BIN DCD SCL
-			OMIT     -  x   *   *   *   *
-			MVI      x  -   *   *   *   *
-			RCD      *  *   -   x   *   x
-			BIN      *  *   x   -   *   x
-			DCD      *  *   *   *   -   x
-			SCL      *  *   x   x   x   -
-		 */
-		
-		if(mvList != null)
-			for(int i=0; i < mvList.length; i++) 
-			{
-				int colID = mvList[i];
-
-				if ( omitList != null && Arrays.binarySearch(omitList, colID) >= 0 ) 
-					throw new IllegalArgumentException("Invalid transformations on column ID " + colID + ". A column can not be both omitted and imputed.");
-				
-				if(mvMethods[i] == 1) 
-				{
-					if ( rcdList != null && Arrays.binarySearch(rcdList, colID) >= 0 ) 
-						throw new IllegalArgumentException("Invalid transformations on column ID " + colID + ". A numeric column can not be recoded.");
-					
-					if ( dcdList != null && Arrays.binarySearch(dcdList, colID) >= 0 )
-						// throw an error only if the column is not binned
-						if ( binList == null || Arrays.binarySearch(binList, colID) < 0 )
-							throw new IllegalArgumentException("Invalid transformations on column ID " + colID + ". A numeric column can not be dummycoded.");
-				}
-			}
-		
-		if(scaleList != null)
-		for(int i=0; i < scaleList.length; i++) 
-		{
-			int colID = scaleList[i];
-			if ( rcdList != null && Arrays.binarySearch(rcdList, colID) >= 0 ) 
-				throw new IllegalArgumentException("Invalid transformations on column ID " + colID + ". A column can not be recoded and scaled.");
-			if ( binList != null && Arrays.binarySearch(binList, colID) >= 0 ) 
-				throw new IllegalArgumentException("Invalid transformations on column ID " + colID + ". A column can not be binned and scaled.");
-			if ( dcdList != null && Arrays.binarySearch(dcdList, colID) >= 0 ) 
-				throw new IllegalArgumentException("Invalid transformations on column ID " + colID + ". A column can not be dummycoded and scaled.");
-		}
-		
-		if(rcdList != null)
-		for(int i=0; i < rcdList.length; i++) 
-		{
-			int colID = rcdList[i];
-			if ( binList != null && Arrays.binarySearch(binList, colID) >= 0 ) 
-				throw new IllegalArgumentException("Invalid transformations on column ID " + colID + ". A column can not be recoded and binned.");
-		}
-		
-		// Check if dummycoded columns are either recoded or binned.
-		// If not, add them to recode list.
-		ArrayList<Integer> addToRcd = new ArrayList<Integer>();
-		if(dcdList != null)
-		for(int i=0; i < dcdList.length; i++) 
-		{
-			int colID = dcdList[i];
-			boolean isRecoded = (rcdList != null && Arrays.binarySearch(rcdList, colID) >= 0);
-			boolean isBinned = (binList != null && Arrays.binarySearch(binList, colID) >= 0);
-			// If colID is neither recoded nor binned, then, add it to rcdList.
-			if ( !isRecoded && !isBinned )
-				addToRcd.add(colID);
-		}
-		if ( addToRcd.size() > 0 ) 
-		{
-			int[] newRcdList = null;
-			if ( rcdList != null)  
-				newRcdList = Arrays.copyOf(rcdList, rcdList.length + addToRcd.size());
-			else
-				newRcdList = new int[addToRcd.size()];
-			
-			int i = (rcdList != null ? rcdList.length : 0);
-			for(int idx=0; i < newRcdList.length; i++, idx++)
-				newRcdList[i] = addToRcd.get(idx);
-			Arrays.sort(newRcdList);
-			rcdList = newRcdList;
-		}
-		// -----------------------------------------------------------------------------
-		
-		// Prepare output spec
-		JSONObject outputSpec = new JSONObject();
-
-		if (omitList != null)
-		{
-			JSONObject rcdSpec = new JSONObject();
-			rcdSpec.put(TfUtils.JSON_ATTRS, toJSONArray(omitList));
-			outputSpec.put(TfUtils.TXMETHOD_OMIT, rcdSpec);
-		}
-		
-		if (mvList != null)
-		{
-			JSONObject mvSpec = new JSONObject();
-			mvSpec.put(TfUtils.JSON_ATTRS, toJSONArray(mvList));
-			mvSpec.put(TfUtils.JSON_MTHD, toJSONArray(mvMethods));
-			mvSpec.put(TfUtils.JSON_CONSTS, toJSONArray(mvConstants));
-			outputSpec.put(TfUtils.TXMETHOD_IMPUTE, mvSpec);
-		}
-		
-		if (rcdList != null)
-		{
-			JSONObject rcdSpec = new JSONObject();
-			rcdSpec.put(TfUtils.JSON_ATTRS, toJSONArray(rcdList));
-			outputSpec.put(TfUtils.TXMETHOD_RECODE, rcdSpec);
-		}
-		
-		if (binList != null)
-		{
-			JSONObject binSpec = new JSONObject();
-			binSpec.put(TfUtils.JSON_ATTRS, toJSONArray(binList));
-			binSpec.put(TfUtils.JSON_MTHD, toJSONArray(binMethods));
-			binSpec.put(TfUtils.JSON_NBINS, toJSONArray(numBins));
-			outputSpec.put(TfUtils.TXMETHOD_BIN, binSpec);
-		}
-		
-		if (dcdList != null)
-		{
-			JSONObject dcdSpec = new JSONObject();
-			dcdSpec.put(TfUtils.JSON_ATTRS, toJSONArray(dcdList));
-			outputSpec.put(TfUtils.TXMETHOD_DUMMYCODE, dcdSpec);
-		}
-		
-		if (scaleList != null)
-		{
-			JSONObject scaleSpec = new JSONObject();
-			scaleSpec.put(TfUtils.JSON_ATTRS, toJSONArray(scaleList));
-			scaleSpec.put(TfUtils.JSON_MTHD, toJSONArray(scaleMethods));
-			outputSpec.put(TfUtils.TXMETHOD_SCALE, scaleSpec);
-		}
-		
-		if (mvrcdList != null)
-		{
-			JSONObject mvrcd = new JSONObject();
-			mvrcd.put(TfUtils.JSON_ATTRS, toJSONArray(mvrcdList));
-			outputSpec.put(TfUtils.TXMETHOD_MVRCD, mvrcd);
-		}
-		
-		// return output spec with IDs
-		return outputSpec.toString();
-	}
-	
-	private static JSONArray toJSONArray(int[] list) 
-	{
-		JSONArray ret = new JSONArray(list.length);
-		for(int i=0; i < list.length; i++)
-			ret.add(list[i]);
-		return ret;
-	}
-
-	private static JSONArray toJSONArray(byte[] list) 
-	{
-		JSONArray ret = new JSONArray(list.length);
-		for(int i=0; i < list.length; i++)
-			ret.add(list[i]);
-		return ret;
-	}
-
-	private static JSONArray toJSONArray(Object[] list) 
-		throws JSONException 
-	{
-		return new JSONArray(list);
-	}
-	
-	/**
-	 * Helper function to move transformation metadata files from a temporary
-	 * location to permanent location. These files (e.g., header before and
-	 * after transformation) are generated by a single mapper, while applying
-	 * data transformations. Note that, these files must be ultimately be placed
-	 * under the existing metadata directory (txMtdPath), which is
-	 * simultaneously read by other mappers. If they are not created at a
-	 * temporary location, then MR tasks fail due to changing timestamps on
-	 * txMtdPath.
-	 * 
-	 * @param fs file system
-	 * @param tmpPath temporary location (directory) of transformation metadata files
-	 * @param txMtdPath directory where to place transformation metadata files
-	 * @throws IllegalArgumentException if IllegalArgumentException occurs
-	 * @throws IOException if IOException occurs
-	 */
-	private static void moveFilesFromTmp(FileSystem fs, String tmpPath, String txMtdPath) throws IllegalArgumentException, IOException 
-	{
-		// move files from temporary location to txMtdPath
-		MapReduceTool.renameFileOnHDFS(tmpPath + "/" + TfUtils.TXMTD_COLNAMES, txMtdPath + "/" + TfUtils.TXMTD_COLNAMES);
-		MapReduceTool.renameFileOnHDFS(tmpPath + "/" + TfUtils.TXMTD_DC_COLNAMES, txMtdPath + "/" + TfUtils.TXMTD_DC_COLNAMES);
-		MapReduceTool.renameFileOnHDFS(tmpPath + "/" + TfUtils.TXMTD_COLTYPES, txMtdPath + "/" + TfUtils.TXMTD_COLTYPES);
-		
-		if ( fs.exists(new Path(tmpPath +"/Dummycode/" + TfUtils.DCD_FILE_NAME)) ) 
-		{
-			if ( !fs.exists( new Path(txMtdPath + "/Dummycode/") )) 
-				fs.mkdirs(new Path(txMtdPath + "/Dummycode/"));
-			MapReduceTool.renameFileOnHDFS( tmpPath + "/Dummycode/" + TfUtils.DCD_FILE_NAME, txMtdPath + "/Dummycode/" + TfUtils.DCD_FILE_NAME);
-		}
-	}
-	
-	/**
-	 * Helper function to determine the number of columns after applying
-	 * transformations. Note that dummycoding changes the number of columns.
-	 * 
-	 * @param fs file system
-	 * @param header header line
-	 * @param delim delimiter
-	 * @param tfMtdPath transform metadata path
-	 * @return number of columns after applying transformations
-	 * @throws IllegalArgumentException if IllegalArgumentException occurs
-	 * @throws IOException if IOException occurs
-	 * @throws DMLRuntimeException if DMLRuntimeException occurs
-	 * @throws JSONException  if JSONException occurs
-	 */
-	private static int getNumColumnsTf(FileSystem fs, String header, String delim, String tfMtdPath) throws IllegalArgumentException, IOException, DMLRuntimeException, JSONException {
-		String[] columnNames = Pattern.compile(Pattern.quote(delim)).split(header, -1);
-		int ret = columnNames.length;
-		
-		JSONObject spec = null;
-		try(BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(new Path(tfMtdPath + "/spec.json"))))) {
-			spec = JSONHelper.parse(br);
-		}
-		
-		// fetch relevant attribute lists
-		if ( !spec.containsKey(TfUtils.TXMETHOD_DUMMYCODE) )
-			return ret;
-		
-		JSONArray dcdList = (JSONArray) ((JSONObject)spec.get(TfUtils.TXMETHOD_DUMMYCODE)).get(TfUtils.JSON_ATTRS);
-
-		// look for numBins among binned columns
-		for(Object o : dcdList) 
-		{
-			int id = UtilFunctions.toInt(o);
-			
-			Path binpath = new Path( tfMtdPath + "/Bin/" + UtilFunctions.unquote(columnNames[id-1]) + TfUtils.TXMTD_BIN_FILE_SUFFIX);
-			Path rcdpath = new Path( tfMtdPath + "/Recode/" + UtilFunctions.unquote(columnNames[id-1]) + TfUtils.TXMTD_RCD_DISTINCT_SUFFIX);
-			
-			if ( TfUtils.checkValidInputFile(fs, binpath, false ) )
-			{
-				int nbins = -1;
-				try( BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(binpath)))) {
-					nbins = UtilFunctions.parseToInt(br.readLine().split(TfUtils.TXMTD_SEP)[4]);
-				}
-				ret += (nbins-1);
-			}
-			else if ( TfUtils.checkValidInputFile(fs, rcdpath, false ) )
-			{
-				int ndistinct = -1;
-				try( BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(rcdpath))) ) {
-					ndistinct = UtilFunctions.parseToInt(br.readLine());
-				}
-				ret += (ndistinct-1);
-			}
-			else
-				throw new DMLRuntimeException("Relevant transformation metadata for column (id=" + id + ", name=" + columnNames[id-1] + ") is not found.");
-		}
-		
-		return ret;
-	}
-	
-	/**
-	 * Main method to create and/or apply transformation metdata using MapReduce.
-	 * 
-	 * @param jobinst MR job instruction
-	 * @param inputs array of input matrices
-	 * @param shuffleInst shuffle instructions
-	 * @param otherInst other instructions
-	 * @param resultIndices byte array of result indices
-	 * @param outputs array of output matrices
-	 * @param numReducers number of reducers
-	 * @param replication ?
-	 * @return MR job result
-	 * @throws Exception if IOException occurs
-	 */
-	public static JobReturn mrDataTransform(MRJobInstruction jobinst, MatrixObject[] inputs, String shuffleInst, String otherInst, byte[] resultIndices, MatrixObject[] outputs, int numReducers, int replication) throws Exception {
-		
-		String[] insts = shuffleInst.split(Instruction.INSTRUCTION_DELIM);
-		
-		// Parse transform instruction (the first instruction) to obtain relevant fields
-		TransformOperands oprnds = new TransformOperands(insts[0], inputs[0]);
-		
-		JobConf job = new JobConf(ConfigurationManager.getCachedJobConf());
-		
-		// find the first file in alphabetical ordering of part files in directory inputPath 
-		String smallestFile = CSVReblockMR.findSmallestFile(job, oprnds.inputPath);
-		
-		// find column names
-		FileSystem fs = IOUtilFunctions.getFileSystem(smallestFile);
-		String headerLine = readHeaderLine(fs, oprnds.inputCSVProperties, smallestFile);
-		HashMap<String, Integer> colNamesToIds = processColumnNames(fs, oprnds.inputCSVProperties, headerLine, smallestFile);
-		String outHeader = getOutputHeader(fs, headerLine, oprnds);
-		int numColumns = colNamesToIds.size();
-		
-		int numColumnsTf = 0;
-		long numRowsTf = 0;
-		
-		ArrayList<Integer> csvoutputs= new ArrayList<Integer>();
-		ArrayList<Integer> bboutputs = new ArrayList<Integer>();
-		
-		// divide output objects based on output format (CSV or BinaryBlock)
-		for(int i=0; i < outputs.length; i++) 
-		{
-			if(outputs[i].getFileFormatProperties() != null 
-					&& outputs[i].getFileFormatProperties().getFileFormat() == FileFormatProperties.FileFormat.CSV)
-				csvoutputs.add(i);
-			else
-				bboutputs.add(i);
-		}
-		boolean isCSV = (csvoutputs.size() > 0);
-		boolean isBB  = (bboutputs.size()  > 0);
-		String tmpPath = MRJobConfiguration.constructTempOutputFilename();
-		
-		checkIfOutputOverlapsWithTxMtd(outputs, oprnds, isCSV, isBB, csvoutputs, bboutputs, fs);
-		
-		JobReturn retCSV = null, retBB = null;
-		
-		if (!oprnds.isApply) {
-			// build specification file with column IDs insteadof column names
-			String specWithIDs = 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.
-			// Also, generate part offsets file (counters file), which is to be used in csv-reblock
-			
-			String partOffsetsFile =  MRJobConfiguration.constructTempOutputFilename();
-			numRowsTf = GenTfMtdMR.runJob(oprnds.inputPath, oprnds.txMtdPath, specWithIDs, smallestFile, 
-					partOffsetsFile, oprnds.inputCSVProperties, numColumns, replication, outHeader);
-			
-			if ( numRowsTf == 0 )
-				throw new DMLRuntimeException(ERROR_MSG_ZERO_ROWS);
-			
-			// store the specFileWithIDs as transformation metadata
-			MapReduceTool.writeStringToHDFS(specWithIDs, oprnds.txMtdPath + "/" + "spec.json");
-			
-			numColumnsTf = getNumColumnsTf(fs, outHeader, oprnds.inputCSVProperties.getDelim(), oprnds.txMtdPath);
-			
-			// Apply transformation metadata, and perform actual transformation 
-			if(isCSV)
-				retCSV = ApplyTfCSVMR.runJob(oprnds.inputPath, specWithIDs, oprnds.txMtdPath, tmpPath, 
-					outputs[csvoutputs.get(0)].getFileName(), partOffsetsFile, 
-					oprnds.inputCSVProperties, numColumns, replication, outHeader);
-			
-			if(isBB)
-			{
-				DMLConfig conf = ConfigurationManager.getDMLConfig();
-				int blockSize = conf.getIntValue(DMLConfig.DEFAULT_BLOCK_SIZE);
-				CSVReblockInstruction rblk = prepDummyReblockInstruction(oprnds.inputCSVProperties, blockSize);
-				
-				AssignRowIDMRReturn ret1 = CSVReblockMR.runAssignRowIDMRJob(new String[]{oprnds.inputPath}, 
-							new InputInfo[]{InputInfo.CSVInputInfo}, new int[]{blockSize}, new int[]{blockSize}, 
-							rblk.toString(), replication, new String[]{smallestFile}, true, 
-							oprnds.inputCSVProperties.getNAStrings(), specWithIDs);
-				if ( ret1.rlens[0] == 0 )
-					throw new DMLRuntimeException(ERROR_MSG_ZERO_ROWS);
-					
-				retBB = ApplyTfBBMR.runJob(oprnds.inputPath, insts[1], otherInst, 
-							specWithIDs, oprnds.txMtdPath, tmpPath, outputs[bboutputs.get(0)].getFileName(), 
-							ret1.counterFile.toString(), oprnds.inputCSVProperties, numRowsTf, numColumns, 
-							numColumnsTf, replication, outHeader);
-			}
-			
-			MapReduceTool.deleteFileIfExistOnHDFS(new Path(partOffsetsFile), job);
-				
-		}
-		else {
-			colNamesToIds = null; // enable GC on colNamesToIds
-			
-			// copy given transform metadata (applyTxPath) to specified location (txMtdPath)
-			MapReduceTool.deleteFileIfExistOnHDFS(new Path(oprnds.txMtdPath), job);
-			MapReduceTool.copyFileOnHDFS(oprnds.applyTxPath, oprnds.txMtdPath);
-			
-			// path to specification file
-			String specWithIDs = (oprnds.spec != null) ? oprnds.spec : 
-				MapReduceTool.readStringFromHDFSFile(oprnds.txMtdPath + "/" + "spec.json");
-			numColumnsTf = getNumColumnsTf(fs, outHeader, 
-												oprnds.inputCSVProperties.getDelim(), 
-												oprnds.txMtdPath);
-			
-			if (isCSV) 
-			{
-				DMLConfig conf = ConfigurationManager.getDMLConfig();
-				int blockSize = conf.getIntValue(DMLConfig.DEFAULT_BLOCK_SIZE);
-				CSVReblockInstruction rblk = prepDummyReblockInstruction(oprnds.inputCSVProperties, blockSize);
-				
-				AssignRowIDMRReturn ret1 = CSVReblockMR.runAssignRowIDMRJob(new String[]{oprnds.inputPath}, 
-							new InputInfo[]{InputInfo.CSVInputInfo}, new int[]{blockSize}, new int[]{blockSize}, 
-							rblk.toString(), replication, new String[]{smallestFile}, true, 
-							oprnds.inputCSVProperties.getNAStrings(), specWithIDs);
-				numRowsTf = ret1.rlens[0];
-				
-				if ( ret1.rlens[0] == 0 )
-					throw new DMLRuntimeException(ERROR_MSG_ZERO_ROWS);
-					
-				// Apply transformation metadata, and perform actual transformation 
-				retCSV = ApplyTfCSVMR.runJob(oprnds.inputPath, specWithIDs, oprnds.applyTxPath, tmpPath, 
-							outputs[csvoutputs.get(0)].getFileName(), ret1.counterFile.toString(), 
-							oprnds.inputCSVProperties, numColumns, replication, outHeader);
-			}
-			
-			if(isBB) 
-			{
-				// compute part offsets file
-				CSVReblockInstruction rblk = (CSVReblockInstruction) InstructionParser.parseSingleInstruction(insts[1]);
-				CSVReblockInstruction newrblk = (CSVReblockInstruction) rblk.clone((byte)0);
-				AssignRowIDMRReturn ret1 = CSVReblockMR.runAssignRowIDMRJob(new String[]{oprnds.inputPath}, 
-							new InputInfo[]{InputInfo.CSVInputInfo}, new int[]{newrblk.brlen}, new int[]{newrblk.bclen}, 
-							newrblk.toString(), replication, new String[]{smallestFile}, true, 
-							oprnds.inputCSVProperties.getNAStrings(), specWithIDs);
-				numRowsTf = ret1.rlens[0];
-				
-				if ( ret1.rlens[0] == 0 )
-					throw new DMLRuntimeException(ERROR_MSG_ZERO_ROWS);
-				
-				// apply transformation metadata, as well as reblock the resulting data
-				retBB = ApplyTfBBMR.runJob(oprnds.inputPath, insts[1], otherInst, specWithIDs, 
-							oprnds.txMtdPath, tmpPath, outputs[bboutputs.get(0)].getFileName(), 
-							ret1.counterFile.toString(), oprnds.inputCSVProperties, ret1.rlens[0], 
-							ret1.clens[0], numColumnsTf, replication, outHeader);
-			}
-		}
-		
-		// copy auxiliary data (old and new header lines) from temporary location to txMtdPath
-		moveFilesFromTmp(fs, tmpPath, oprnds.txMtdPath);
-
-		// generate matrix metadata file for outputs
-		if ( retCSV != null ) 
-		{
-			retCSV.getMatrixCharacteristics(0).setDimension(numRowsTf, numColumnsTf);
-			
-			CSVFileFormatProperties prop = new CSVFileFormatProperties(
-												false, 
-												oprnds.inputCSVProperties.getDelim(), // use the same header as the input
-												false, Double.NaN, null);
-			
-			MapReduceTool.writeMetaDataFile (outputs[csvoutputs.get(0)].getFileName()+".mtd", 
-												ValueType.DOUBLE, retCSV.getMatrixCharacteristics(0), 
-												OutputInfo.CSVOutputInfo, prop);
-			return retCSV;
-		}
-
-		if ( retBB != null )
-		{
-			retBB.getMatrixCharacteristics(0).setDimension(numRowsTf, numColumnsTf);
-			
-			MapReduceTool.writeMetaDataFile (outputs[bboutputs.get(0)].getFileName()+".mtd", 
-					ValueType.DOUBLE, retBB.getMatrixCharacteristics(0), OutputInfo.BinaryBlockOutputInfo);
-			return retBB;
-		}
-		
-		return null;
-			
-	}
-	
-	private static CSVReblockInstruction prepDummyReblockInstruction(CSVFileFormatProperties prop, int blockSize) {
-		StringBuilder sb = new StringBuilder();
-		sb.append( ExecType.MR );
-		
-		sb.append( Lop.OPERAND_DELIMITOR );
-		sb.append( CSVReBlock.OPCODE );
-		
-		sb.append( Lop.OPERAND_DELIMITOR );
-		sb.append( "0" );
-		sb.append(Lop.DATATYPE_PREFIX);
-		sb.append(DataType.MATRIX);
-		sb.append(Lop.VALUETYPE_PREFIX);
-		sb.append(ValueType.DOUBLE);
-		
-		sb.append( Lop.OPERAND_DELIMITOR );
-		sb.append( "1" );
-		sb.append(Lop.DATATYPE_PREFIX);
-		sb.append(DataType.MATRIX);
-		sb.append(Lop.VALUETYPE_PREFIX);
-		sb.append(ValueType.DOUBLE);
-		
-		sb.append( Lop.OPERAND_DELIMITOR );
-		sb.append( blockSize );
-		
-		sb.append( Lop.OPERAND_DELIMITOR );
-		sb.append( blockSize );
-		
-		sb.append( Lop.OPERAND_DELIMITOR );
-		sb.append( prop.hasHeader() );
-		sb.append( Lop.OPERAND_DELIMITOR );
-		sb.append( prop.getDelim() );
-		sb.append( Lop.OPERAND_DELIMITOR );
-		sb.append( prop.isFill() );
-		sb.append( Lop.OPERAND_DELIMITOR );
-		sb.append( prop.getFillValue() );
-
-		return (CSVReblockInstruction) CSVReblockInstruction.parseInstruction(sb.toString());
-	}
-
-	private static String getOutputHeader(FileSystem fs, String headerLine, TransformOperands oprnds) throws IOException
-	{
-		String ret = null;
-		
-		if(oprnds.isApply)
-		{
-			try( BufferedReader br = new BufferedReader(new InputStreamReader( fs.open(new Path(oprnds.applyTxPath + "/" + TfUtils.TXMTD_COLNAMES)) )) ) {
-				ret = br.readLine();
-			}
-		}
-		else {
-			if ( oprnds.outNamesFile == null )
-				ret = headerLine;
-			else {
-				try( BufferedReader br = new BufferedReader(new InputStreamReader( fs.open(new Path(oprnds.outNamesFile)) )) ) {
-					ret = br.readLine();
-				}
-			}
-		}
-		
-		return ret;
-	}
-	
-	/**
-	 * Main method to create and/or apply transformation metdata in-memory, on a
-	 * single node.
-	 * 
-	 * @param inst parameterized built-in CP instruction
-	 * @param inputs array of input data
-	 * @param outputs array of output matrices
-	 * @return MR job result
-	 * @throws IOException if IOException occurs
-	 * @throws DMLRuntimeException if DMLRuntimeException occurs
-	 * @throws IllegalArgumentException if IllegalArgumentException occurs
-	 * @throws JSONException if JSONException occurs
-	 */
-	public static JobReturn cpDataTransform(ParameterizedBuiltinCPInstruction inst, CacheableData<?>[] inputs, MatrixObject[] outputs) throws IOException, DMLRuntimeException, IllegalArgumentException, JSONException {
-		TransformOperands oprnds = new TransformOperands(inst.getParameterMap(), inputs[0]);
-		return cpDataTransform(oprnds, inputs, outputs);
-	}
-
-	public static JobReturn cpDataTransform(String inst, CacheableData<?>[] inputs, MatrixObject[] outputs) 
-		throws IOException, DMLRuntimeException, IllegalArgumentException, JSONException 
-	{
-		String[] insts = inst.split(Instruction.INSTRUCTION_DELIM);
-		// Parse transform instruction (the first instruction) to obtain relevant fields
-		TransformOperands oprnds = new TransformOperands(insts[0], inputs[0]);
-		
-		return cpDataTransform(oprnds, inputs, outputs);
-	}
-		
-	public static JobReturn cpDataTransform(TransformOperands oprnds, CacheableData<?>[] inputs, MatrixObject[] outputs) 
-		throws IOException, DMLRuntimeException, IllegalArgumentException, JSONException 
-	{
-		JobConf job = new JobConf(ConfigurationManager.getCachedJobConf());
-		
-		// find the first file in alphabetical ordering of partfiles in directory inputPath 
-		String smallestFile = CSVReblockMR.findSmallestFile(job, oprnds.inputPath);
-		FileSystem fs = IOUtilFunctions.getFileSystem(smallestFile);
-		
-		// find column names
-		String headerLine = readHeaderLine(fs, oprnds.inputCSVProperties, smallestFile);
-		HashMap<String, Integer> colNamesToIds = processColumnNames(fs, oprnds.inputCSVProperties, headerLine, smallestFile);
-		String outHeader = getOutputHeader(fs, headerLine, oprnds);
-		
-		ArrayList<Integer> csvoutputs= new ArrayList<Integer>();
-		ArrayList<Integer> bboutputs = new ArrayList<Integer>();
-		
-		// divide output objects based on output format (CSV or BinaryBlock)
-		for(int i=0; i < outputs.length; i++) 
-		{
-			if(outputs[i].getFileFormatProperties() != null && outputs[i].getFileFormatProperties().getFileFormat() == FileFormatProperties.FileFormat.CSV)
-				csvoutputs.add(i);
-			else
-				bboutputs.add(i);
-		}
-		boolean isCSV = (csvoutputs.size() > 0);
-		boolean isBB  = (bboutputs.size()  > 0);
-		
-		checkIfOutputOverlapsWithTxMtd(outputs, oprnds, isCSV, isBB, csvoutputs, bboutputs, fs);
-		
-		JobReturn ret = null;
-		
-		if (!oprnds.isApply) {
-			// build specification file with column IDs insteadof column names
-			String specWithIDs = processSpecFile(fs, oprnds.inputPath, smallestFile, colNamesToIds, oprnds.inputCSVProperties, 
-					oprnds.spec);
-			MapReduceTool.writeStringToHDFS(specWithIDs, oprnds.txMtdPath + "/" + "spec.json");
-	
-			ret = performTransform(job, fs, oprnds.inputPath, colNamesToIds.size(), oprnds.inputCSVProperties, specWithIDs, 
-					oprnds.txMtdPath, oprnds.isApply, outputs[0], outHeader, isBB, isCSV );
-		}
-		else {
-			// copy given transform metadata (applyTxPath) to specified location (txMtdPath)
-			MapReduceTool.deleteFileIfExistOnHDFS(new Path(oprnds.txMtdPath), job);
-			MapReduceTool.copyFileOnHDFS(oprnds.applyTxPath, oprnds.txMtdPath);
-			
-			// path to specification file (optionally specified)
-			String specWithIDs = (oprnds.spec != null) ? 
-				oprnds.spec : MapReduceTool.readStringFromHDFSFile(oprnds.txMtdPath + "/" + "spec.json");
-			
-			ret = performTransform(job, fs, oprnds.inputPath, colNamesToIds.size(), oprnds.inputCSVProperties, specWithIDs,  
-					oprnds.txMtdPath, oprnds.isApply, outputs[0], outHeader, isBB, isCSV );
-		}
-		
-		return ret;
-	}
-	
-	/**
-	 * Helper function to fetch and sort the list of part files under the given
-	 * input directory.
-	 * 
-	 * @param input input directory
-	 * @param fs file system
-	 * @return list of paths to input file parts
-	 * @throws FileNotFoundException if FileNotFoundException occurs
-	 * @throws IOException if IOException occurs
-	 */
-	@SuppressWarnings("unchecked")
-	private static ArrayList<Path> collectInputFiles(String input, FileSystem fs) throws FileNotFoundException, IOException 
-	{
-		Path path = new Path(input);
-		ArrayList<Path> files=new ArrayList<Path>();
-		if(fs.isDirectory(path))
-		{
-			for(FileStatus stat: fs.listStatus(path, CSVReblockMR.hiddenFileFilter))
-				files.add(stat.getPath());
-			Collections.sort(files);
-		}
-		else
-			files.add(path);
-
-		return files;
-	}
-	
-	private static int[] countNumRows(ArrayList<Path> files, CSVFileFormatProperties prop, FileSystem fs, TfUtils agents) throws IOException 
-	{
-		int[] rows = new int[2];
-		int numRows=0, numRowsTf=0;
-		
-		OmitAgent oa = agents.getOmitAgent();
-		
-		if(!oa.isApplicable())
-		{
-			for(int fileNo=0; fileNo<files.size(); fileNo++)
-			{
-				try(BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(files.get(fileNo))))) {
-					if(fileNo==0 && prop.hasHeader() ) 
-						br.readLine(); //ignore header
-					
-					while ( br.readLine() != null)
-						numRows++;
-				}
-			}
-			numRowsTf = numRows;
-		}
-		else
-		{
-			String line = null;
-			String[] words;
-			
-			Pattern delim = Pattern.compile(Pattern.quote(prop.getDelim()));
-			
-			for(int fileNo=0; fileNo<files.size(); fileNo++)
-			{
-				try( BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(files.get(fileNo)))) ) {
-					if(fileNo==0 && prop.hasHeader() ) 
-						br.readLine(); //ignore header
-					while ( (line=br.readLine()) != null)
-					{
-						numRows++;
-						
-						words = delim.split(line, -1);
-						if(!oa.omit(words, agents))
-							numRowsTf++;
-					}
-				}
-			}
-		}
-		
-		rows[0] = numRows;
-		rows[1] = numRowsTf;
-		
-		return rows;
-	}
-	
-	/**
-	 * Main method to create and/or apply transformation metdata in-memory, on a single node.
-	 * 
-	 * @param job job configuration
-	 * @param fs file system
-	 * @param inputPath path to input files
-	 * @param ncols number of columns
-	 * @param prop csv file format properties
-	 * @param specWithIDs JSON transform specification with IDs
-	 * @param tfMtdPath transform metadata path
-	 * @param isApply ?
-	 * @param result output matrix
-	 * @param headerLine header line
-	 * @param isBB true if binary block
-	 * @param isCSV true if CSV
-	 * @return MR job result
-	 * @throws IOException if IOException occurs
-	 * @throws DMLRuntimeException if DMLRuntimeException occurs
-	 * @throws IllegalArgumentException if IllegalArgumentException occurs
-	 * @throws JSONException if JSONException occurs
-	 */
-	private static JobReturn performTransform(JobConf job, FileSystem fs, String inputPath, int ncols, CSVFileFormatProperties prop, String specWithIDs, String tfMtdPath, boolean isApply, MatrixObject result, String headerLine, boolean isBB, boolean isCSV ) throws IOException, DMLRuntimeException, IllegalArgumentException, JSONException {
-		
-		String[] na = TfUtils.parseNAStrings(prop.getNAStrings());
-		
-		JSONObject spec = new JSONObject(specWithIDs);
-		TfUtils agents = new TfUtils(headerLine, prop.hasHeader(), prop.getDelim(), na, spec, ncols, tfMtdPath, null, null );
-		
-		MVImputeAgent _mia = agents.getMVImputeAgent();
-		RecodeAgent _ra = agents.getRecodeAgent();
-		BinAgent _ba = agents.getBinAgent();
-		DummycodeAgent _da = agents.getDummycodeAgent();
-
-		// List of files to read
-		ArrayList<Path> files = collectInputFiles(inputPath, fs);
-				
-		// ---------------------------------
-		// Construct transformation metadata
-		// ---------------------------------
-		
-		String line = null;
-		String[] words  = null;
-		
-		int numColumnsTf=0;
-		
-		if (!isApply) {
-			for(int fileNo=0; fileNo<files.size(); fileNo++)
-			{
-				try(BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(files.get(fileNo)))) ) {
-					if(fileNo==0 && prop.hasHeader() ) 
-						br.readLine(); //ignore header
-					
-					line = null;
-					while ( (line = br.readLine()) != null) {
-						agents.prepareTfMtd(line);
-					}
-				}
-			}
-			
-			if(agents.getValid() == 0) 
-				throw new DMLRuntimeException(ERROR_MSG_ZERO_ROWS);
-			
-			_mia.outputTransformationMetadata(tfMtdPath, fs, agents);
-			_ba.outputTransformationMetadata(tfMtdPath, fs, agents);
-			_ra.outputTransformationMetadata(tfMtdPath, fs, agents);
-		
-			// prepare agents for the subsequent phase of applying transformation metadata
-			
-			// NO need to loadTxMtd for _ra, since the maps are already present in the memory
-			Path tmp = new Path(tfMtdPath);
-			_mia.loadTxMtd(job, fs, tmp, agents);
-			_ba.loadTxMtd(job, fs, tmp, agents);
-			
-			_da.setRecodeMapsCP( _ra.getCPRecodeMaps() );
-			_da.setNumBins(_ba.getColList(), _ba.getNumBins());
-			_da.loadTxMtd(job, fs, tmp, agents);
-		}
-		else {
-			// Count the number of rows
-			int rows[] = countNumRows(files, prop, fs, agents);
-			agents.setTotal(rows[0]);
-			agents.setValid(rows[1]);
-			
-			if(agents.getValid() == 0) 
-				throw new DMLRuntimeException("Number of rows in the transformed output (potentially, after ommitting the ones with missing values) is zero. Cannot proceed.");
-			
-			// Load transformation metadata
-			// prepare agents for the subsequent phase of applying transformation metadata
-			Path tmp = new Path(tfMtdPath);
-			_mia.loadTxMtd(job, fs, tmp, agents);
-			_ra.loadTxMtd(job, fs, tmp, agents);
-			_ba.loadTxMtd(job, fs, tmp, agents);
-			
-			_da.setRecodeMaps( _ra.getRecodeMaps() );
-			_da.setNumBins(_ba.getColList(), _ba.getNumBins());
-			_da.loadTxMtd(job, fs, tmp, agents);
-		}
-		
-		// -----------------------------
-		// Apply transformation metadata
-		// -----------------------------
-        
-		numColumnsTf = getNumColumnsTf(fs, headerLine, prop.getDelim(), tfMtdPath);
-
-		MapReduceTool.deleteFileIfExistOnHDFS(result.getFileName());
-		BufferedWriter out=new BufferedWriter(new OutputStreamWriter(fs.create(new Path(result.getFileName()),true)));		
-		StringBuilder sb = new StringBuilder();
-		
-		try {
-			MatrixBlock mb = null; 
-			if ( isBB ) 
-			{
-				int estNNZ = (int)agents.getValid() * ncols;
-				mb = new MatrixBlock((int)agents.getValid(), numColumnsTf, estNNZ );
-				
-				if ( mb.isInSparseFormat() )
-					mb.allocateSparseRowsBlock();
-				else
-					mb.allocateDenseBlock();
-			}
-	
-			int rowID = 0; // rowid to be used in filling the matrix block
-			
-			for(int fileNo=0; fileNo<files.size(); fileNo++)
-			{
-				try( BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(files.get(fileNo)))) ) {
-					if ( fileNo == 0 ) 
-					{
-						if ( prop.hasHeader() )
-							br.readLine(); // ignore the header line from data file
-						
-						//TODO: fix hard-wired header propagation to meta data column names
-						
-						String dcdHeader = _da.constructDummycodedHeader(headerLine, agents.getDelim());
-						numColumnsTf = _da.genDcdMapsAndColTypes(fs, tfMtdPath, ncols, agents);
-						generateHeaderFiles(fs, tfMtdPath, headerLine, dcdHeader);
-					}
-					
-					line = null;
-					while ( (line = br.readLine()) != null) {
-						words = agents.getWords(line);
-		
-						if(!agents.omit(words))
-						{
-							words = agents.apply(words);
-			
-							if (isCSV)
-							{
-								out.write( agents.checkAndPrepOutputString(words, sb) );
-								out.write("\n");
-							}
-							
-							if( isBB ) 
-							{
-								agents.check(words);
-								for(int c=0; c<words.length; c++)
-								{
-									if(words[c] == null || words[c].isEmpty())
-										;
-									else 
-										mb.appendValue(rowID, c, UtilFunctions.parseToDouble(words[c]));
-								}
-							}
-							rowID++;
-						}
-					}
-				}
-			}
-			
-			if(mb != null)
-			{
-				mb.recomputeNonZeros();
-				mb.examSparsity();
-				
-				result.acquireModify(mb);
-				result.release();
-				result.exportData();
-			}
-		}
-		finally {
-			IOUtilFunctions.closeSilently(out);
-		}
-		
-		MatrixCharacteristics mc = new MatrixCharacteristics(agents.getValid(), numColumnsTf, (int) result.getNumRowsPerBlock(), (int) result.getNumColumnsPerBlock());
-		JobReturn ret = new JobReturn(new MatrixCharacteristics[]{mc}, true);
-
-		return ret;
-	}
-	
-	public static void generateHeaderFiles(FileSystem fs, String txMtdDir, String origHeader, String newHeader) throws IOException {
-		// write out given header line
-		try( BufferedWriter br=new BufferedWriter(new OutputStreamWriter(
-				fs.create(new Path(txMtdDir+"/" + TfUtils.TXMTD_COLNAMES),true)))) {
-			br.write(origHeader+"\n");
-		}
-
-		// write out the new header line (after all transformations)
-		try( BufferedWriter br=new BufferedWriter(new OutputStreamWriter(
-				fs.create(new Path(txMtdDir + "/" + TfUtils.TXMTD_DC_COLNAMES),true))) ){
-			br.write(newHeader+"\n");
-		}
-	}
-	
-	private static void checkIfOutputOverlapsWithTxMtd(MatrixObject[] outputs, TransformOperands oprnds,
-			boolean isCSV, boolean isBB, ArrayList<Integer> csvoutputs, ArrayList<Integer> bboutputs, FileSystem fs) throws DMLRuntimeException {
-		if(isCSV) {
-			checkIfOutputOverlapsWithTxMtd(oprnds.txMtdPath, outputs[csvoutputs.get(0)].getFileName(), fs);
-		}
-		else if(isBB) {
-			checkIfOutputOverlapsWithTxMtd(oprnds.txMtdPath, outputs[bboutputs.get(0)].getFileName(), fs);
-		}
-	}
-	
-	@SuppressWarnings("deprecation")
-	private static void checkIfOutputOverlapsWithTxMtd(String txMtdPath, String outputPath, FileSystem fs) 
-		throws DMLRuntimeException 
-	{
-		Path path1 = new Path(txMtdPath).makeQualified(fs);
-		Path path2 = new Path(outputPath).makeQualified(fs);
-		
-		String fullTxMtdPath = path1.toString();
-		String fullOutputPath = path2.toString();
-		
-		if(path1.getParent().toString().equals(path2.getParent().toString())) {
-			// Both txMtdPath and outputPath are in same folder, but outputPath can have suffix 
-			if(fullTxMtdPath.equals(fullOutputPath)) {
-				throw new DMLRuntimeException("The transform path \'" + txMtdPath 
-						+ "\' cannot overlap with the output path \'" + outputPath + "\'");
-			}
-		}
-		else if(fullTxMtdPath.startsWith(fullOutputPath) || fullOutputPath.startsWith(fullTxMtdPath)) {
-			throw new DMLRuntimeException("The transform path \'" + txMtdPath 
-					+ "\' cannot overlap with the output path \'" + outputPath + "\'");
-		}
-	}
-	
-	public static void spDataTransform(ParameterizedBuiltinSPInstruction inst, FrameObject[] inputs, MatrixObject[] outputs, ExecutionContext ec) throws Exception {
-		
-		SparkExecutionContext sec = (SparkExecutionContext)ec;
-		
-		// Parse transform instruction (the first instruction) to obtain relevant fields
-		TransformOperands oprnds = new TransformOperands(inst.getParams(), inputs[0]);
-		
-		JobConf job = new JobConf();
-		FileSystem fs = IOUtilFunctions.getFileSystem(inputs[0].getFileName());
-		
-		checkIfOutputOverlapsWithTxMtd(oprnds.txMtdPath, outputs[0].getFileName(), fs);
-		
-		// find the first file in alphabetical ordering of partfiles in directory inputPath 
-		String smallestFile = CSVReblockMR.findSmallestFile(job, oprnds.inputPath);
-		
-		// find column names and construct output header
-		String headerLine = readHeaderLine(fs, oprnds.inputCSVProperties, smallestFile);
-		HashMap<String, Integer> colNamesToIds = processColumnNames(fs, oprnds.inputCSVProperties, headerLine, smallestFile);
-		int numColumns = colNamesToIds.size();
-		String outHeader = getOutputHeader(fs, headerLine, oprnds);
-		
-		String tmpPath = MRJobConfiguration.constructTempOutputFilename();
-		
-		// Construct RDD for input data
-		@SuppressWarnings("unchecked")
-		JavaPairRDD<LongWritable, Text> inputData = (JavaPairRDD<LongWritable, Text>) sec.getRDDHandleForFrameObject(inputs[0], InputInfo.CSVInputInfo);
-		JavaRDD<Tuple2<LongWritable,Text>> csvLines = JavaPairRDD.toRDD(inputData).toJavaRDD();
-		
-		long numRowsTf=0, numColumnsTf=0;
-		JavaPairRDD<Long, String> tfPairRDD = null;
-		
-		if (!oprnds.isApply) {
-			// build specification file with column IDs insteadof column names
-			String specWithIDs = 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.
-			// Also, generate part offsets file (counters file), which is to be used in csv-reblock (if needed)
-			String partOffsetsFile =  MRJobConfiguration.constructTempOutputFilename();
-			numRowsTf = GenTfMtdSPARK.runSparkJob(sec, csvLines, oprnds.txMtdPath,  
-													specWithIDs,partOffsetsFile, 
-													oprnds.inputCSVProperties, numColumns, 
-													outHeader);
-			
-			// store the specFileWithIDs as transformation metadata
-			MapReduceTool.writeStringToHDFS(specWithIDs, oprnds.txMtdPath + "/" + "spec.json");
-			
-			numColumnsTf = getNumColumnsTf(fs, outHeader, oprnds.inputCSVProperties.getDelim(), oprnds.txMtdPath);
-			
-			tfPairRDD = ApplyTfCSVSPARK.runSparkJob(sec, csvLines, oprnds.txMtdPath, 
-					specWithIDs, tmpPath, oprnds.inputCSVProperties, numColumns, outHeader);
-
-			
-			MapReduceTool.deleteFileIfExistOnHDFS(new Path(partOffsetsFile), job);
-		}
-		else {
-			colNamesToIds = null; // enable GC on colNamesToIds
-			
-			// copy given transform metadata (applyTxPath) to specified location (txMtdPath)
-			MapReduceTool.deleteFileIfExistOnHDFS(new Path(oprnds.txMtdPath), job);
-			MapReduceTool.copyFileOnHDFS(oprnds.applyTxPath, oprnds.txMtdPath);
-			
-			// path to specification file
-			String specWithIDs = (oprnds.spec != null) ? oprnds.spec :
-					MapReduceTool.readStringFromHDFSFile(oprnds.txMtdPath + "/" + "spec.json");
-			numColumnsTf = getNumColumnsTf(fs, outHeader, 
-												oprnds.inputCSVProperties.getDelim(), 
-												oprnds.txMtdPath);
-			
-			// Apply transformation metadata, and perform actual transformation 
-			tfPairRDD = ApplyTfCSVSPARK.runSparkJob(sec, csvLines, oprnds.txMtdPath, 
-					specWithIDs, tmpPath, oprnds.inputCSVProperties, numColumns, outHeader);
-			
-		}
-		
-		// copy auxiliary data (old and new header lines) from temporary location to txMtdPath
-		moveFilesFromTmp(fs, tmpPath, oprnds.txMtdPath);
-
-		// convert to csv output format (serialized longwritable/text)
-		JavaPairRDD<LongWritable, Text> outtfPairRDD = 
-				RDDConverterUtils.stringToSerializableText(tfPairRDD);
-		
-		if ( outtfPairRDD != null ) 
-		{
-			MatrixObject outMO = outputs[0];
-			String outVar = outMO.getVarName();
-			outMO.setRDDHandle(new RDDObject(outtfPairRDD, outVar));
-			sec.addLineageRDD(outVar, inst.getParams().get("target"));
-			
-			//update output statistics (required for correctness)
-			MatrixCharacteristics mcOut = sec.getMatrixCharacteristics(outVar);
-			mcOut.setDimension(numRowsTf, numColumnsTf);
-			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, CacheableData<?> input) {
-			inputPath = input.getFileName();
-			inputCSVProperties = (CSVFileFormatProperties)input.getFileFormatProperties();
-			String[] instParts = inst.split(Instruction.OPERAND_DELIM);
-			txMtdPath = instParts[3];
-			applyTxPath = instParts[4].startsWith("applymtd=") ? instParts[4].substring(9) : null;
-			isApply = (applyTxPath != null);
-			int pos = (applyTxPath != null) ? 5 : 4;
-			if( pos<instParts.length )
-				spec = instParts[pos].startsWith("spec=") ? instParts[pos++].substring(5) : null;
-			if( pos<instParts.length )
-				outNamesFile = instParts[pos].startsWith("outnames=") ? instParts[pos].substring(9) : null;
-		}
-		
-		private TransformOperands(HashMap<String, String> params, CacheableData<?> input) {
-			inputPath = input.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)input.getFileFormatProperties();
-		}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/DistinctValue.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/DistinctValue.java b/src/main/java/org/apache/sysml/runtime/transform/DistinctValue.java
deleted file mode 100644
index c35e160..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/DistinctValue.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * 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.runtime.transform;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.nio.charset.Charset;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-import org.apache.sysml.runtime.matrix.CSVReblockMR.OffsetCount;
-import org.apache.sysml.runtime.util.UtilFunctions;
-
-public class DistinctValue implements Writable, Serializable {
-	
-	private static final long serialVersionUID = -8236705946336974836L;
-
-	private static final byte [] EMPTY_BYTES = new byte[0];
-	  
-	// word (distinct value)
-	private byte[] _bytes;
-	private int _length;
-	// count
-	private long _count;
-	
-	public DistinctValue() {
-		_bytes = EMPTY_BYTES;
-		_length = 0;
-		_count = -1;
-	}
-	
-	public DistinctValue(String w, long count) throws CharacterCodingException {
-	    ByteBuffer bb = Text.encode(w, true);
-	    _bytes = bb.array();
-	    _length = bb.limit();
-		_count = count;
-	}
-	
-	public DistinctValue(OffsetCount oc) throws CharacterCodingException {
-		this(oc.filename + "," + oc.fileOffset, oc.count);
-	}
-	
-	public void reset() {
-		_bytes = EMPTY_BYTES;
-		_length = 0;
-		_count = -1;
-	}
-	
-	public String getWord() {  
-		return new String( _bytes, 0, _length, Charset.forName("UTF-8") ); 
-	}
-	
-	public long getCount() { 
-		return _count; 
-	}
-	
-	@Override
-	public void write(DataOutput out) throws IOException {
-	    // write word
-		WritableUtils.writeVInt(out, _length);
-	    out.write(_bytes, 0, _length);
-		// write count
-	    out.writeLong(_count);
-	}
-	
-	@Override
-	public void readFields(DataInput in) throws IOException {
-	    // read word 
-		_length = WritableUtils.readVInt(in);
-	    _bytes = new byte[_length];
-	    in.readFully(_bytes, 0, _length);
-	    // read count
-	    _count = in.readLong();
-	}
-	
-	public OffsetCount getOffsetCount() {
-		String[] parts = getWord().split(",");
-		return new OffsetCount( parts[0],
-				UtilFunctions.parseToLong(parts[1]),
-				getCount() );
-	}
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/DummycodeAgent.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/DummycodeAgent.java b/src/main/java/org/apache/sysml/runtime/transform/DummycodeAgent.java
deleted file mode 100644
index 676b31e..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/DummycodeAgent.java
+++ /dev/null
@@ -1,461 +0,0 @@
-/*
- * 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.runtime.transform;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.sysml.runtime.matrix.data.FrameBlock;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.transform.encode.Encoder;
-import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
-import org.apache.sysml.runtime.util.UtilFunctions;
-import org.apache.wink.json4j.JSONException;
-import org.apache.wink.json4j.JSONObject;
-
-public class DummycodeAgent extends Encoder 
-{		
-	private static final long serialVersionUID = 5832130477659116489L;
-
-	private HashMap<Integer, HashMap<String,String>> _finalMaps = null;
-	private HashMap<Integer, HashMap<String,Long>> _finalMapsCP = null;
-	private int[] _binList = null;
-	private int[] _numBins = null;
-	
-	private int[] _domainSizes = null;			// length = #of dummycoded columns
-	private int[] _dcdColumnMap = null;			// to help in translating between original and dummycoded column IDs
-	private long _dummycodedLength = 0;			// #of columns after dummycoded
-
-	public DummycodeAgent(JSONObject parsedSpec, String[] colnames, int clen) throws JSONException {
-		super(null, clen);
-		
-		if ( parsedSpec.containsKey(TfUtils.TXMETHOD_DUMMYCODE) ) {
-			int[] collist = TfMetaUtils.parseJsonIDList(parsedSpec, colnames, TfUtils.TXMETHOD_DUMMYCODE);
-			initColList(collist);
-		}
-	}
-	
-	@Override
-	public int getNumCols() {
-		return (int)_dummycodedLength;
-	}
-	
-	/**
-	 * Method to output transformation metadata from the mappers. 
-	 * This information is collected and merged by the reducers.
-	 */
-	@Override
-	public void mapOutputTransformationMetadata(OutputCollector<IntWritable, DistinctValue> out, int taskID, TfUtils agents) throws IOException {
-		// There is no metadata required for dummycode.
-		// Required information is output from RecodeAgent.
-		return;
-	}
-	
-	@Override
-	public void mergeAndOutputTransformationMetadata(Iterator<DistinctValue> values,
-			String outputDir, int colID, FileSystem fs, TfUtils agents) throws IOException {
-		// Nothing to do here
-	}
-
-	public void setRecodeMaps(HashMap<Integer, HashMap<String,String>> maps) {
-		_finalMaps = maps;
-	}
-	
-	public void setRecodeMapsCP(HashMap<Integer, HashMap<String,Long>> maps) {
-		_finalMapsCP = maps;
-	}
-	
-	public void setNumBins(int[] binList, int[] numbins) {
-		_binList = binList;
-		_numBins = numbins;
-	}
-	
-	/**
-	 * Method to generate dummyCodedMaps.csv, with the range of column IDs for each variable in the original data.
-	 * 
-	 * Each line in dummyCodedMaps.csv file is of the form: [ColID, 1/0, st, end]
-	 * 		1/0 indicates if ColID is dummycoded or not
-	 * 		[st,end] is the range of dummycoded column numbers for the given ColID
-	 * 
-	 * It also generates coltypes.csv, with the type (scale, nominal, etc.) of columns in the output.
-	 * Recoded columns are of type nominal, binner columns are of type ordinal, dummycoded columns are of type 
-	 * dummycoded, and the remaining are of type scale.
-	 * 
-	 * @param fs file system
-	 * @param txMtdDir path to transform metadata directory
-	 * @param numCols number of columns
-	 * @param agents ?
-	 * @return ?
-	 * @throws IOException if IOException occurs
-	 */
-	public int genDcdMapsAndColTypes(FileSystem fs, String txMtdDir, int numCols, TfUtils agents) throws IOException {
-		
-		// initialize all column types in the transformed data to SCALE
-		TfUtils.ColumnTypes[] ctypes = new TfUtils.ColumnTypes[(int) _dummycodedLength];
-		for(int i=0; i < _dummycodedLength; i++)
-			ctypes[i] = TfUtils.ColumnTypes.SCALE;
-		
-		_dcdColumnMap = new int[numCols];
-
-		int sum=1;
-		try( BufferedWriter br=new BufferedWriter(new OutputStreamWriter(
-				fs.create(new Path(txMtdDir+"/Dummycode/" + TfUtils.DCD_FILE_NAME),true))) ) {
-			int idx = 0;
-			for(int colID=1; colID <= numCols; colID++) 
-			{
-				if ( _colList != null && idx < _colList.length && _colList[idx] == colID )
-				{
-					br.write(colID + TfUtils.TXMTD_SEP + "1" + TfUtils.TXMTD_SEP + sum + TfUtils.TXMTD_SEP + (sum+_domainSizes[idx]-1) + "\n");
-					_dcdColumnMap[colID-1] = (sum+_domainSizes[idx]-1)-1;
-	
-					for(int i=sum; i <=(sum+_domainSizes[idx]-1); i++)
-						ctypes[i-1] = TfUtils.ColumnTypes.DUMMYCODED;
-					
-					sum += _domainSizes[idx];
-					idx++;
-				}
-				else 
-				{
-					br.write(colID + TfUtils.TXMTD_SEP + "0" + TfUtils.TXMTD_SEP + sum + TfUtils.TXMTD_SEP + sum + "\n");
-					_dcdColumnMap[colID-1] = sum-1;
-					
-					if ( agents.getBinAgent().isApplicable(colID) != -1 )
-						ctypes[sum-1] = TfUtils.ColumnTypes.ORDINAL;	// binned variable results in an ordinal column
-					
-					if ( agents.getRecodeAgent().isApplicable(colID) != -1 )
-						ctypes[sum-1] = TfUtils.ColumnTypes.NOMINAL;
-					
-					sum += 1;
-				}
-			}
-		}
-
-		// Write coltypes.csv
-		try(BufferedWriter br=new BufferedWriter(new OutputStreamWriter(
-				fs.create(new Path(txMtdDir + File.separator + TfUtils.TXMTD_COLTYPES),true))) ) {
-			br.write(ctypes[0].toID() + "");
-			for(int i = 1; i < _dummycodedLength; i++) 
-				br.write( TfUtils.TXMTD_SEP + ctypes[i].toID() );
-		}
-		
-		return sum-1;
-	}
-	
-	/**
-	 * Given a dummycoded column id, find the corresponding original column ID.
-	 *  
-	 * @param colID dummycoded column ID
-	 * @return original column ID, -1 if not found
-	 */
-	public int mapDcdColumnID(int colID) 
-	{
-		for(int i=0; i < _dcdColumnMap.length; i++)
-		{
-			int st = (i==0 ? 1 : _dcdColumnMap[i-1]+1+1);
-			int end = _dcdColumnMap[i]+1;
-			//System.out.println((i+1) + ": " + "[" + st + "," + end + "]");
-			
-			if ( colID >= st && colID <= end)
-				return i+1;
-		}
-		return -1;
-	}
-	
-	public String constructDummycodedHeader(String header, Pattern delim) {
-		
-		if(_colList == null && _binList == null )
-			// none of the columns are dummycoded, simply return the given header
-			return header;
-		
-		String[] names = delim.split(header, -1);
-		List<String> newNames = null;
-		
-		StringBuilder sb = new StringBuilder();
-		
-		// Dummycoding can be performed on either on a recoded column or on a binned column
-		
-		// process recoded columns
-		if(_finalMapsCP != null && _colList != null) 
-		{
-			for(int i=0; i <_colList.length; i++) 
-			{
-				int colID = _colList[i];
-				HashMap<String,Long> map = _finalMapsCP.get(colID);
-				String colName = UtilFunctions.unquote(names[colID-1]);
-				
-				if ( map != null  ) 
-				{
-					// order map entries by their recodeID
-					List<Map.Entry<String, Long>> entryList = new ArrayList<Map.Entry<String, Long>>(map.entrySet());
-					Comparator<Map.Entry<String, Long>> comp = new Comparator<Map.Entry<String, Long>>() {
-						@Override
-						public int compare(Entry<String, Long> entry1, Entry<String, Long> entry2) {
-							Long value1 = entry1.getValue();
-							Long value2 = entry2.getValue();
-							return (int) (value1 - value2);
-						}
-					};
-					Collections.sort(entryList, comp);
-					newNames = new ArrayList<String>();
-					for (Entry<String, Long> entry : entryList) {
-						newNames.add(entry.getKey());
-					}
-					
-					// construct concatenated string of map entries
-					sb.setLength(0);
-					for(int idx=0; idx < newNames.size(); idx++) 
-					{
-						if(idx==0) 
-							sb.append( colName + TfUtils.DCD_NAME_SEP + newNames.get(idx));
-						else
-							sb.append( delim + colName + TfUtils.DCD_NAME_SEP + newNames.get(idx));
-					}
-					names[colID-1] = sb.toString();			// replace original column name with dcd name
-				}
-			}
-		}
-		else if(_finalMaps != null && _colList != null) {
-			for(int i=0; i <_colList.length; i++) {
-				int colID = _colList[i];
-				HashMap<String,String> map = _finalMaps.get(colID);
-				String colName = UtilFunctions.unquote(names[colID-1]);
-				
-				if ( map != null ) 
-				{
-					
-					// order map entries by their recodeID (represented as Strings .. "1", "2", etc.)
-					List<Map.Entry<String, String>> entryList = new ArrayList<Map.Entry<String, String>>(map.entrySet());
-					Comparator<Map.Entry<String, String>> comp = new Comparator<Map.Entry<String, String>>() {
-						@Override
-						public int compare(Entry<String, String> entry1, Entry<String, String> entry2) {
-							String value1 = entry1.getValue();
-							String value2 = entry2.getValue();
-							return (Integer.parseInt(value1) - Integer.parseInt(value2));
-						}
-					};
-					Collections.sort(entryList, comp);
-					newNames = new ArrayList<String>();
-					for (Entry<String, String> entry : entryList) {
-						newNames.add(entry.getKey());
-					}
-					
-					// construct concatenated string of map entries
-					sb.setLength(0);
-					for(int idx=0; idx < newNames.size(); idx++) 
-					{
-						if(idx==0) 
-							sb.append( colName + TfUtils.DCD_NAME_SEP + newNames.get(idx));
-						else
-							sb.append( delim + colName + TfUtils.DCD_NAME_SEP + newNames.get(idx));
-					}
-					names[colID-1] = sb.toString();			// replace original column name with dcd name
-				}
-			}
-		}
-		
-		// process binned columns
-		if (_binList != null) 
-			for(int i=0; i < _binList.length; i++) 
-			{
-				int colID = _binList[i];
-				
-				// need to consider only binned and dummycoded columns
-				if(isApplicable(colID) == -1)
-					continue;
-				
-				int numBins = _numBins[i];
-				String colName = UtilFunctions.unquote(names[colID-1]);
-				
-				sb.setLength(0);
-				for(int idx=0; idx < numBins; idx++) 
-					if(idx==0) 
-						sb.append( colName + TfUtils.DCD_NAME_SEP + "Bin" + (idx+1) );
-					else
-						sb.append( delim + colName + TfUtils.DCD_NAME_SEP + "Bin" + (idx+1) );
-				names[colID-1] = sb.toString();			// replace original column name with dcd name
-			}
-		
-		// Construct the full header
-		sb.setLength(0);
-		for(int colID=0; colID < names.length; colID++) 
-		{
-			if (colID == 0)
-				sb.append(names[colID]);
-			else
-				sb.append(delim + names[colID]);
-		}
-		//System.out.println("DummycodedHeader: " + sb.toString());
-		
-		return sb.toString();
-	}
-	
-	@Override
-	public void loadTxMtd(JobConf job, FileSystem fs, Path txMtdDir, TfUtils agents) throws IOException {
-		if ( !isApplicable() ) {
-			_dummycodedLength = _clen;
-			return;
-		}
-		
-		// sort to-be dummycoded column IDs in ascending order. This is the order in which the new dummycoded record is constructed in apply() function.
-		Arrays.sort(_colList);	
-		_domainSizes = new int[_colList.length];
-
-		_dummycodedLength = _clen;
-		
-		//HashMap<String, String> map = null;
-		for(int i=0; i<_colList.length; i++) {
-			int colID = _colList[i];
-			
-			// Find the domain size for colID using _finalMaps or _finalMapsCP
-			int domainSize = 0;
-			if(_finalMaps != null) {
-				if(_finalMaps.get(colID) != null)
-					domainSize = _finalMaps.get(colID).size();
-			}
-			else {
-				if(_finalMapsCP.get(colID) != null)
-					domainSize = _finalMapsCP.get(colID).size();
-			}
-			
-			if ( domainSize != 0 ) {
-				// dummycoded column
-				_domainSizes[i] = domainSize;
-			}
-			else {
-				// binned column
-				if ( _binList != null )
-				for(int j=0; j<_binList.length; j++) {
-					if (colID == _binList[j]) {
-						_domainSizes[i] = _numBins[j];
-						break;
-					}
-				}
-			}
-			_dummycodedLength += _domainSizes[i]-1;
-		}
-	}
-
-
-	@Override
-	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
-		return apply(in, out);
-	}
-
-	@Override
-	public void build(FrameBlock in) {
-		//do nothing
-	}
-	
-	/**
-	 * Method to apply transformations.
-	 * 
-	 * @param words array of strings
-	 * @return array of transformed strings
-	 */
-	@Override
-	public String[] apply(String[] words) 
-	{
-		if( !isApplicable() )
-			return words;
-		
-		String[] nwords = new String[(int)_dummycodedLength];
-		int rcdVal = 0;
-		
-		for(int colID=1, idx=0, ncolID=1; colID <= words.length; colID++) {
-			if(idx < _colList.length && colID==_colList[idx]) {
-				// dummycoded columns
-				try {
-					rcdVal = UtilFunctions.parseToInt(UtilFunctions.unquote(words[colID-1]));
-					nwords[ ncolID-1+rcdVal-1 ] = "1";
-					ncolID += _domainSizes[idx];
-					idx++;
-				} 
-				catch (Exception e) {
-					throw new RuntimeException("Error in dummycoding: colID="+colID + ", rcdVal=" + rcdVal+", word="+words[colID-1] 
-							+ ", domainSize=" + _domainSizes[idx] + ", dummyCodedLength=" + _dummycodedLength);
-				}
-			}
-			else {
-				nwords[ncolID-1] = words[colID-1];
-				ncolID++;
-			}
-		}
-		
-		return nwords;
-	}
-	
-	@Override
-	public MatrixBlock apply(FrameBlock in, MatrixBlock out) 
-	{
-		MatrixBlock ret = new MatrixBlock(out.getNumRows(), (int)_dummycodedLength, false);
-		
-		for( int i=0; i<out.getNumRows(); i++ ) {
-			for(int colID=1, idx=0, ncolID=1; colID <= out.getNumColumns(); colID++) {
-				double val = out.quickGetValue(i, colID-1);
-				if(idx < _colList.length && colID==_colList[idx]) {
-					ret.quickSetValue(i, ncolID-1+(int)val-1, 1);
-					ncolID += _domainSizes[idx];
-					idx++;
-				}
-				else {
-					double ptval = UtilFunctions.objectToDouble(in.getSchema()[colID-1], in.get(i, colID-1));
-					ret.quickSetValue(i, ncolID-1, ptval);
-					ncolID++;
-				}
-			}
-		}
-		
-		return ret;
-	}
-
-	@Override
-	public FrameBlock getMetaData(FrameBlock out) {
-		return out;
-	}
-	
-	@Override
-	public void initMetaData(FrameBlock meta) {
-		//initialize domain sizes and output num columns
-		_domainSizes = new int[_colList.length];
-		_dummycodedLength = _clen;
-		for( int j=0; j<_colList.length; j++ ) {
-			int colID = _colList[j]; //1-based
-			_domainSizes[j] = (int)meta.getColumnMetadata()[colID-1].getNumDistinct();
-			_dummycodedLength +=  _domainSizes[j]-1;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/GTFMTDMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/GTFMTDMapper.java b/src/main/java/org/apache/sysml/runtime/transform/GTFMTDMapper.java
deleted file mode 100644
index 1e45036..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/GTFMTDMapper.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * 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.runtime.transform;
-import java.io.IOException;
-
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.wink.json4j.JSONException;
-import org.apache.sysml.runtime.matrix.CSVReblockMR.OffsetCount;
-import org.apache.sysml.runtime.matrix.mapred.MRConfigurationNames;
-
-
-public class GTFMTDMapper implements Mapper<LongWritable, Text, IntWritable, DistinctValue>{
-	
-	private OutputCollector<IntWritable, DistinctValue> _collector = null; 
-	private int _mapTaskID = -1;
-	
-	TfUtils _agents = null;
-
-	private boolean _partFileWithHeader = false;
-	private boolean _firstRecordInSplit = true;
-	private String _partFileName = null;
-	private long _offsetInPartFile = -1;
-	
-	// ----------------------------------------------------------------------------------------------
-	
-	/**
-	 * Configure the information used in the mapper, and setup transformation agents.
-	 */
-	@Override
-	public void configure(JobConf job) {
-		String[] parts = job.get(MRConfigurationNames.MR_TASK_ATTEMPT_ID).split("_");
-		if ( parts[0].equalsIgnoreCase("task")) {
-			_mapTaskID = Integer.parseInt(parts[parts.length-1]);
-		}
-		else if ( parts[0].equalsIgnoreCase("attempt")) {
-			_mapTaskID = Integer.parseInt(parts[parts.length-2]);
-		}
-		else {
-			throw new RuntimeException("Unrecognized format for taskID: " + job.get(MRConfigurationNames.MR_TASK_ATTEMPT_ID));
-		}
-
-		try {
-			_partFileName = TfUtils.getPartFileName(job);
-			_partFileWithHeader = TfUtils.isPartFileWithHeader(job);
-			_agents = new TfUtils(job);
-		} catch(IOException e) { throw new RuntimeException(e); }
-		  catch(JSONException e)  { throw new RuntimeException(e); }
-
-	}
-	
-	
-	public void map(LongWritable rawKey, Text rawValue, OutputCollector<IntWritable, DistinctValue> out, Reporter reporter) throws IOException  {
-		
-		if(_firstRecordInSplit)
-		{
-			_firstRecordInSplit = false;
-			_collector = out;
-			_offsetInPartFile = rawKey.get();
-		}
-		
-		// ignore header
-		if (_agents.hasHeader() && rawKey.get() == 0 && _partFileWithHeader)
-			return;
-		
-		_agents.prepareTfMtd(rawValue.toString());
-	}
-
-	@Override
-	public void close() throws IOException {
-		_agents.getMVImputeAgent().mapOutputTransformationMetadata(_collector, _mapTaskID, _agents);
-		_agents.getRecodeAgent().mapOutputTransformationMetadata(_collector, _mapTaskID, _agents);
-		_agents.getBinAgent().mapOutputTransformationMetadata(_collector, _mapTaskID, _agents);
-		
-		// Output part-file offsets to create OFFSETS_FILE, which is to be used in csv reblocking.
-		// OffsetCount is denoted as a DistinctValue by concatenating parfile name and offset within partfile.
-		if(_collector != null) {
-			IntWritable key = new IntWritable((int)_agents.getNumCols()+1);
-			DistinctValue val = new DistinctValue(new OffsetCount(_partFileName, _offsetInPartFile, _agents.getValid())); 
-			_collector.collect(key, val);
-		}
-		
-		// reset global variables, required when the jvm is reused.
-		_firstRecordInSplit = true;
-		_offsetInPartFile = -1;
-		_partFileWithHeader = false;
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/GTFMTDReducer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/GTFMTDReducer.java b/src/main/java/org/apache/sysml/runtime/transform/GTFMTDReducer.java
deleted file mode 100644
index 01fc784..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/GTFMTDReducer.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * 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.runtime.transform;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.ByteWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.wink.json4j.JSONException;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.matrix.CSVReblockMR.OffsetCount;
-import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration;
-
-
-public class GTFMTDReducer implements Reducer<IntWritable, DistinctValue, Text, LongWritable> {
-	
-	private JobConf _rJob = null;
-	TfUtils _agents = null;
-	
-	@Override
-	public void configure(JobConf job) {
-		_rJob = job;
-		
-		try {
-			String outputDir = MRJobConfiguration.getOutputs(job)[0];
-			_agents = new TfUtils(job, outputDir);
-		} 
-		catch(IOException e)  { throw new RuntimeException(e); }
-		catch(JSONException e)  { throw new RuntimeException(e); }
-	}
-
-	@Override
-	public void close() throws IOException {
-	}
-	
-	@Override
-	public void reduce(IntWritable key, Iterator<DistinctValue> values,
-			OutputCollector<Text, LongWritable> output, Reporter reporter)
-			throws IOException {
-		
-		FileSystem fs = FileSystem.get(_rJob);
-		
-		int colID = key.get();
-		
-		if(colID < 0) 
-		{
-			// process mapper output for MV and Bin agents
-			colID = colID*-1;
-			_agents.getMVImputeAgent().mergeAndOutputTransformationMetadata(values, _agents.getTfMtdDir(), colID, fs, _agents);
-		}
-		else if ( colID == _agents.getNumCols() + 1)
-		{
-			// process mapper output for OFFSET_FILE
-			ArrayList<OffsetCount> list = new ArrayList<OffsetCount>();
-			while(values.hasNext())
-				list.add(new OffsetCount(values.next().getOffsetCount()));
-			
-			long numTfRows = generateOffsetsFile(list);
-			reporter.incrCounter(MRJobConfiguration.DataTransformCounters.TRANSFORMED_NUM_ROWS, numTfRows);
-
-		}
-		else 
-		{
-			// process mapper output for Recode agent
-			_agents.getRecodeAgent().mergeAndOutputTransformationMetadata(values, _agents.getTfMtdDir(), colID, fs, _agents);
-		}
-		
-	}
-	
-	@SuppressWarnings({"unchecked","deprecation"})
-	private long generateOffsetsFile(ArrayList<OffsetCount> list) throws IllegalArgumentException, IOException 
-	{
-		Collections.sort(list);
-		
-		SequenceFile.Writer writer = null;
-		long lineOffset=0;
-		try {
-			writer = new SequenceFile.Writer(
-				FileSystem.get(_rJob), _rJob, 
-				new Path(_agents.getOffsetFile()+"/part-00000"), 
-				ByteWritable.class, OffsetCount.class);
-		
-			for(OffsetCount oc: list)
-			{
-				long count=oc.count;
-				oc.count=lineOffset;
-				writer.append(new ByteWritable((byte)0), oc);
-				lineOffset+=count;
-			}
-		}
-		finally {
-			IOUtilFunctions.closeSilently(writer);
-		}
-		list.clear();
-		return lineOffset;
-	}
-	
-}
-


[3/6] systemml git commit: [SYSTEMML-1300] Remove file-based transform from compiler/runtime

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/GenTfMtdMR.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/GenTfMtdMR.java b/src/main/java/org/apache/sysml/runtime/transform/GenTfMtdMR.java
deleted file mode 100644
index 68024d9..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/GenTfMtdMR.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * 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.runtime.transform;
-import java.io.IOException;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.Counters;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.lib.NullOutputFormat;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
-import org.apache.sysml.runtime.matrix.mapred.MRConfigurationNames;
-import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration;
-
-/**
- * MR Job to Generate Transform Metadata based on a given transformation specification file (JSON format).
- *
- */
-
-public class GenTfMtdMR {
-
-	public static long runJob(String inputPath, String txMtdPath, String specWithIDs, String smallestFile, String partOffsetsFile, CSVFileFormatProperties inputDataProperties, long numCols, int replication, String headerLine) throws IOException, ClassNotFoundException, InterruptedException {
-		JobConf job = new JobConf(GenTfMtdMR.class);
-		job.setJobName("GenTfMTD");
-		
-		/* Setup MapReduce Job */
-		job.setJarByClass(GenTfMtdMR.class);
-		
-		// set relevant classes
-		job.setMapperClass(GTFMTDMapper.class);
-		job.setReducerClass(GTFMTDReducer.class);
-	
-		// set input and output properties
-		job.setInputFormat(TextInputFormat.class);
-		job.setOutputFormat(NullOutputFormat.class);
-		
-		job.setMapOutputKeyClass(IntWritable.class);
-		job.setMapOutputValueClass(DistinctValue.class);
-		
-		job.setOutputKeyClass(Text.class);
-		job.setOutputValueClass(LongWritable.class);
-		
-		job.setInt(MRConfigurationNames.DFS_REPLICATION, replication);
-
-		FileInputFormat.addInputPath(job, new Path(inputPath));
-		// delete outputPath, if exists already.
-		Path outPath = new Path(txMtdPath);
-		FileSystem fs = IOUtilFunctions.getFileSystem(outPath, job);
-		fs.delete(outPath, true);
-		FileOutputFormat.setOutputPath(job, outPath);
-
-		job.set(MRJobConfiguration.TF_HAS_HEADER, Boolean.toString(inputDataProperties.hasHeader()));
-		job.set(MRJobConfiguration.TF_DELIM, inputDataProperties.getDelim());
-		if ( inputDataProperties.getNAStrings() != null)
-			// Adding "dummy" string to handle the case of na_strings = ""
-			job.set(MRJobConfiguration.TF_NA_STRINGS, TfUtils.prepNAStrings(inputDataProperties.getNAStrings()) );
-		job.set(MRJobConfiguration.TF_SPEC, specWithIDs);
-		job.set(MRJobConfiguration.TF_SMALLEST_FILE, smallestFile);
-		job.setLong(MRJobConfiguration.TF_NUM_COLS, numCols);
-		job.set(MRJobConfiguration.TF_HEADER, headerLine);
-		
-		job.set(MRJobConfiguration.OUTPUT_MATRICES_DIRS_CONFIG, txMtdPath);
-		
-		// offsets file to store part-file names and offsets for each input split
-		job.set(MRJobConfiguration.TF_OFFSETS_FILE, partOffsetsFile);
-		
-		//turn off adaptivemr
-		job.setBoolean("adaptivemr.map.enable", false);
-		
-		// Run the job
-		RunningJob runjob = JobClient.runJob(job);
-		
-		Counters c = runjob.getCounters();
-		long tx_numRows = c.findCounter(MRJobConfiguration.DataTransformCounters.TRANSFORMED_NUM_ROWS).getCounter();
-
-		return tx_numRows;
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/GenTfMtdSPARK.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/GenTfMtdSPARK.java b/src/main/java/org/apache/sysml/runtime/transform/GenTfMtdSPARK.java
deleted file mode 100644
index 6e688ef..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/GenTfMtdSPARK.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/*
- * 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.runtime.transform;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.ByteWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.function.FlatMapFunction;
-import org.apache.spark.api.java.function.Function2;
-import org.apache.wink.json4j.JSONException;
-import org.apache.wink.json4j.JSONObject;
-
-import scala.Tuple2;
-
-import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.matrix.CSVReblockMR.OffsetCount;
-import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
-import org.apache.sysml.runtime.matrix.data.Pair;
-
-public class GenTfMtdSPARK 
-{
-	/**
-	 * Spark code to Generate Transform Metadata based on the given transformation
-	 * specification file (JSON format).
-	 * 
-	 * @param sec spark execution context
-	 * @param inputRDD input rdd
-	 * @param tfMtdPath transform metadata path
-	 * @param spec JSON transform specification
-	 * @param partOffsetsFile ?
-	 * @param prop csv file format properties
-	 * @param numCols number of columns
-	 * @param headerLine header line
-	 * @return number of rows
-	 * @throws IOException if IOException occurs
-	 * @throws ClassNotFoundException if ClassNotFoundException occurs
-	 * @throws InterruptedException if InterruptedException occurs
-	 * @throws IllegalArgumentException if IllegalArgumentException occurs
-	 * @throws JSONException if JSONException occurs
-	 */
-	public static long runSparkJob(SparkExecutionContext sec, JavaRDD<Tuple2<LongWritable, Text>> inputRDD, 
-									String tfMtdPath, String spec, String partOffsetsFile, 
-									CSVFileFormatProperties prop, long numCols, String headerLine) 
-		throws IOException, ClassNotFoundException, InterruptedException, IllegalArgumentException, JSONException 
-	{	
-		// Construct transformation metadata (map-side)
-		// Note: logic is similar to GTFMTDMapper
-		JavaRDD<Tuple2<Integer,DistinctValue>> tfMapOutput 
-			= inputRDD.mapPartitionsWithIndex(
-					new GenTfMtdMap(prop.hasHeader(), prop.getDelim(), prop.getNAStrings(), 
-									spec, numCols, headerLine), 
-					true );
-		
-		// Shuffle to group by DistinctValue
-		JavaPairRDD<Integer,Iterable<DistinctValue>> rdd = JavaPairRDD.fromJavaRDD(tfMapOutput).groupByKey();
-		
-		// Construct transformation metadata (Reduce-side)
-		// Note: logic is similar to GTFMTDReducer
-		JavaRDD<Long> out 
-			= rdd.flatMap(new GenTfMtdReduce(prop.hasHeader(), prop.getDelim(), prop.getNAStrings(), 
-									headerLine, tfMtdPath, partOffsetsFile, spec, numCols)  );
-		
-		// Compute the total number of transformed rows
-		long numRows = out.reduce(new Function2<Long,Long,Long>() {
-			private static final long serialVersionUID = 1263336168859959795L;
-
-			@Override
-			public Long call(Long v1, Long v2) throws Exception {
-				return v1+v2;
-			}			
-		});
-		
-		return numRows;
-	}
-	
-	// ----------------------------------------------------------------------------------------------------------------------
-	
-	private  static class GenTfMtdMap implements Function2<Integer, Iterator<Tuple2<LongWritable, Text>>, Iterator<Tuple2<Integer,DistinctValue>>> 
-	{
-		private static final long serialVersionUID = -5622745445470598215L;
-		
-		private TfUtils _agents = null;
-		
-		public GenTfMtdMap(boolean hasHeader, String delim, String naStrings, String spec, long numCols, String headerLine) throws IllegalArgumentException, IOException, JSONException {
-			
-			// Setup Transformation Agents
-			String[] nas = TfUtils.parseNAStrings(naStrings);
-			JSONObject jspec = new JSONObject(spec);
-			_agents = new TfUtils(headerLine, hasHeader, delim, nas, jspec, numCols, null, null, null);
-
-		}
-		
-		@Override
-		public Iterator<Tuple2<Integer,DistinctValue>> call(Integer partitionID,
-				Iterator<Tuple2<LongWritable, Text>> csvLines) throws Exception {
-			
-			// Construct transformation metadata by looping through csvLines
-			// Note: logic is similar to GTFMTDMapper
-			
-			boolean first = true;
-			Tuple2<LongWritable, Text> rec = null;
-			long _offsetInPartFile = -1;
-			
-			while(csvLines.hasNext()) {
-				rec = csvLines.next();
-				
-				if (first) {
-					first = false;
-					_offsetInPartFile = rec._1().get();
-					
-					if (partitionID == 0 && _agents.hasHeader() && _offsetInPartFile == 0 )
-						continue; // skip the header line
-				}
-				
-				_agents.prepareTfMtd(rec._2().toString());
-			}
-			
-			// Prepare the output in the form of DistinctValues, which subsequently need to be grouped and aggregated. 
-			
-			ArrayList<Pair<Integer,DistinctValue>> outList = new ArrayList<Pair<Integer,DistinctValue>>();
-			
-			_agents.getMVImputeAgent().mapOutputTransformationMetadata(partitionID, outList, _agents);
-			_agents.getRecodeAgent().mapOutputTransformationMetadata(partitionID, outList, _agents);
-			_agents.getBinAgent().mapOutputTransformationMetadata(partitionID, outList, _agents);
-			
-			DistinctValue dv = new DistinctValue(new OffsetCount("Partition"+partitionID, _offsetInPartFile, _agents.getTotal()));
-			Pair<Integer, DistinctValue> tuple = new Pair<Integer, DistinctValue>((int) (_agents.getNumCols()+1), dv); 
-			outList.add(tuple);
-
-			return toTuple2List(outList).iterator();
-		}
-		
-	}
-	
-	// ------------------------------------------------------------------------------------------------
-	
-	private static class GenTfMtdReduce implements FlatMapFunction<Tuple2<Integer, Iterable<DistinctValue>>, Long> 
-	{	
-		private static final long serialVersionUID = -2733233671193035242L;
-		private TfUtils _agents = null;
-		
-		public GenTfMtdReduce(boolean hasHeader, String delim, String naStrings, String headerLine, String tfMtdDir, String offsetFile, String spec, long numCols) throws IOException, JSONException {
-			String[] nas = TfUtils.parseNAStrings(naStrings); 
-			JSONObject jspec = new JSONObject(spec);
-			_agents = new TfUtils(headerLine, hasHeader, delim, nas, jspec, numCols, tfMtdDir, offsetFile, null);
-		}
-
-		@SuppressWarnings({"unchecked","deprecation"})
-		@Override
-		public Iterator<Long> call(Tuple2<Integer, Iterable<DistinctValue>> t)
-				throws Exception {
-			
-			int colID = t._1();
-			Iterator<DistinctValue> iterDV = t._2().iterator();
-
-			JobConf job = new JobConf();
-			FileSystem fs = FileSystem.get(job);
-			
-			ArrayList<Long> numRows = new ArrayList<Long>();
-			
-			if(colID < 0) 
-			{
-				// process mapper output for MV and Bin agents
-				colID = colID*-1;
-				_agents.getMVImputeAgent().mergeAndOutputTransformationMetadata(iterDV, _agents.getTfMtdDir(), colID, fs, _agents);
-				numRows.add(0L);
-			}
-			else if ( colID == _agents.getNumCols() + 1)
-			{
-				// process mapper output for OFFSET_FILE
-				ArrayList<OffsetCount> list = new ArrayList<OffsetCount>();
-				while(iterDV.hasNext())
-					list.add(new OffsetCount(iterDV.next().getOffsetCount()));
-				Collections.sort(list);
-				
-				SequenceFile.Writer writer = null;
-				long lineOffset=0;
-				try {
-					writer = new SequenceFile.Writer(fs, job, new Path(_agents.getOffsetFile()+"/part-00000"), ByteWritable.class, OffsetCount.class);
-					for(OffsetCount oc: list) {
-						long count=oc.count;
-						oc.count=lineOffset;
-						writer.append(new ByteWritable((byte)0), oc);
-						lineOffset+=count;
-					}
-				}
-				finally {
-					IOUtilFunctions.closeSilently(writer);
-				}
-				list.clear();
-				
-				numRows.add(lineOffset);
-			}
-			else 
-			{
-				// process mapper output for Recode agent
-				_agents.getRecodeAgent().mergeAndOutputTransformationMetadata(iterDV, _agents.getTfMtdDir(), colID, fs, _agents);
-				numRows.add(0L);
-			}
-			
-			return numRows.iterator();
-		}
-	}
-
-	public static List<Tuple2<Integer,DistinctValue>> toTuple2List(List<Pair<Integer,DistinctValue>> in) {
-		ArrayList<Tuple2<Integer,DistinctValue>> ret = new ArrayList<Tuple2<Integer,DistinctValue>>();
-		for( Pair<Integer,DistinctValue> e : in )
-			ret.add(new Tuple2<Integer,DistinctValue>(e.getKey(), e.getValue()));
-		return ret;
-	}
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/MVImputeAgent.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/MVImputeAgent.java b/src/main/java/org/apache/sysml/runtime/transform/MVImputeAgent.java
deleted file mode 100644
index 97881e7..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/MVImputeAgent.java
+++ /dev/null
@@ -1,1046 +0,0 @@
-/*
- * 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.runtime.transform;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.nio.charset.CharacterCodingException;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.wink.json4j.JSONArray;
-import org.apache.wink.json4j.JSONException;
-import org.apache.wink.json4j.JSONObject;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.functionobjects.CM;
-import org.apache.sysml.runtime.functionobjects.KahanPlus;
-import org.apache.sysml.runtime.functionobjects.Mean;
-import org.apache.sysml.runtime.instructions.cp.CM_COV_Object;
-import org.apache.sysml.runtime.instructions.cp.KahanObject;
-import org.apache.sysml.runtime.matrix.data.FrameBlock;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.matrix.data.Pair;
-import org.apache.sysml.runtime.matrix.operators.CMOperator;
-import org.apache.sysml.runtime.matrix.operators.CMOperator.AggregateOperationTypes;
-import org.apache.sysml.runtime.transform.encode.Encoder;
-import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
-import org.apache.sysml.runtime.util.UtilFunctions;
-
-public class MVImputeAgent extends Encoder 
-{	
-	private static final long serialVersionUID = 9057868620144662194L;
-
-	public static final String MEAN_PREFIX = "mean";
-	public static final String VARIANCE_PREFIX = "var";
-	public static final String CORRECTION_PREFIX = "correction";
-	public static final String COUNT_PREFIX = "validcount";		// #of valid or non-missing values in a column
-	public static final String TOTAL_COUNT_PREFIX = "totalcount";	// #of total records processed by a mapper
-	public static final String CONSTANT_PREFIX = "constant";
-	
-	public enum MVMethod { INVALID, GLOBAL_MEAN, GLOBAL_MODE, CONSTANT };
-	
-	private MVMethod[] _mvMethodList = null;
-	private MVMethod[] _mvscMethodList = null;	// scaling methods for attributes that are imputed and also scaled
-	
-	private BitSet _isMVScaled = null;
-	private CM _varFn = CM.getCMFnObject(AggregateOperationTypes.VARIANCE);		// function object that understands variance computation
-	
-	// objects required to compute mean and variance of all non-missing entries 
-	private Mean _meanFn = Mean.getMeanFnObject();	// function object that understands mean computation
-	private KahanObject[] _meanList = null; 		// column-level means, computed so far
-	private long[] _countList = null;				// #of non-missing values
-	
-	private CM_COV_Object[] _varList = null;		// column-level variances, computed so far (for scaling)
-
-	private int[] 			_scnomvList = null;			// List of attributes that are scaled but not imputed
-	private MVMethod[]		_scnomvMethodList = null;	// scaling methods: 0 for invalid; 1 for mean-subtraction; 2 for z-scoring
-	private KahanObject[] 	_scnomvMeanList = null;		// column-level means, for attributes scaled but not imputed
-	private long[] 			_scnomvCountList = null;	// #of non-missing values, for attributes scaled but not imputed
-	private CM_COV_Object[] _scnomvVarList = null;		// column-level variances, computed so far
-	
-	private String[] _replacementList = null;		// replacements: for global_mean, mean; and for global_mode, recode id of mode category
-	private String[] _NAstrings = null;
-	private List<Integer> _rcList = null; 
-	private HashMap<Integer,HashMap<String,Long>> _hist = null;
-	
-	public String[] getReplacements() { return _replacementList; }
-	public KahanObject[] getMeans()   { return _meanList; }
-	public CM_COV_Object[] getVars()  { return _varList; }
-	public KahanObject[] getMeans_scnomv()   { return _scnomvMeanList; }
-	public CM_COV_Object[] getVars_scnomv()  { return _scnomvVarList; }
-	
-	public MVImputeAgent(JSONObject parsedSpec, String[] colnames, int clen) 
-		throws JSONException
-	{
-		super(null, clen);
-		
-		//handle column list
-		int[] collist = TfMetaUtils.parseJsonObjectIDList(parsedSpec, colnames, TfUtils.TXMETHOD_IMPUTE);
-		initColList(collist);
-	
-		//handle method list
-		parseMethodsAndReplacments(parsedSpec);
-		
-		//create reuse histograms
-		_hist = new HashMap<Integer, HashMap<String,Long>>();
-	}
-			
-	public MVImputeAgent(JSONObject parsedSpec, String[] colnames, String[] NAstrings, int clen)
-		throws JSONException 
-	{
-		super(null, clen);	
-		boolean isMV = parsedSpec.containsKey(TfUtils.TXMETHOD_IMPUTE);
-		boolean isSC = parsedSpec.containsKey(TfUtils.TXMETHOD_SCALE);		
-		_NAstrings = NAstrings;
-		
-		if(!isMV) {
-			// MV Impute is not applicable
-			_colList = null;
-			_mvMethodList = null;
-			_meanList = null;
-			_countList = null;
-			_replacementList = null;
-		}
-		else {
-			JSONObject mvobj = (JSONObject) parsedSpec.get(TfUtils.TXMETHOD_IMPUTE);
-			JSONArray mvattrs = (JSONArray) mvobj.get(TfUtils.JSON_ATTRS);
-			JSONArray mvmthds = (JSONArray) mvobj.get(TfUtils.JSON_MTHD);
-			int mvLength = mvattrs.size();
-			
-			_colList = new int[mvLength];
-			_mvMethodList = new MVMethod[mvLength];
-			
-			_meanList = new KahanObject[mvLength];
-			_countList = new long[mvLength];
-			_varList = new CM_COV_Object[mvLength];
-			
-			_isMVScaled = new BitSet(_colList.length);
-			_isMVScaled.clear();
-			
-			for(int i=0; i < _colList.length; i++) {
-				_colList[i] = UtilFunctions.toInt(mvattrs.get(i));
-				_mvMethodList[i] = MVMethod.values()[UtilFunctions.toInt(mvmthds.get(i))]; 
-				_meanList[i] = new KahanObject(0, 0);
-			}
-			
-			_replacementList = new String[mvLength]; 	// contains replacements for all columns (scale and categorical)
-			
-			JSONArray constants = (JSONArray)mvobj.get(TfUtils.JSON_CONSTS);
-			for(int i=0; i < constants.size(); i++) {
-				if ( constants.get(i) == null )
-					_replacementList[i] = "NaN";
-				else
-					_replacementList[i] = constants.get(i).toString();
-			}
-		}
-		
-		// Handle scaled attributes
-		if ( !isSC )
-		{
-			// scaling is not applicable
-			_scnomvCountList = null;
-			_scnomvMeanList = null;
-			_scnomvVarList = null;
-		}
-		else
-		{
-			if ( _colList != null ) 
-				_mvscMethodList = new MVMethod[_colList.length];
-			
-			JSONObject scobj = (JSONObject) parsedSpec.get(TfUtils.TXMETHOD_SCALE);
-			JSONArray scattrs = (JSONArray) scobj.get(TfUtils.JSON_ATTRS);
-			JSONArray scmthds = (JSONArray) scobj.get(TfUtils.JSON_MTHD);
-			int scLength = scattrs.size();
-			
-			int[] _allscaled = new int[scLength];
-			int scnomv = 0, colID;
-			byte mthd;
-			for(int i=0; i < scLength; i++)
-			{
-				colID = UtilFunctions.toInt(scattrs.get(i));
-				mthd = (byte) UtilFunctions.toInt(scmthds.get(i)); 
-						
-				_allscaled[i] = colID;
-				
-				// check if the attribute is also MV imputed
-				int mvidx = isApplicable(colID);
-				if(mvidx != -1)
-				{
-					_isMVScaled.set(mvidx);
-					_mvscMethodList[mvidx] = MVMethod.values()[mthd];
-					_varList[mvidx] = new CM_COV_Object();
-				}
-				else
-					scnomv++;	// count of scaled but not imputed 
-			}
-			
-			if(scnomv > 0)
-			{
-				_scnomvList = new int[scnomv];			
-				_scnomvMethodList = new MVMethod[scnomv];	
-	
-				_scnomvMeanList = new KahanObject[scnomv];
-				_scnomvCountList = new long[scnomv];
-				_scnomvVarList = new CM_COV_Object[scnomv];
-				
-				for(int i=0, idx=0; i < scLength; i++)
-				{
-					colID = UtilFunctions.toInt(scattrs.get(i));
-					mthd = (byte)UtilFunctions.toInt(scmthds.get(i)); 
-							
-					if(isApplicable(colID) == -1)
-					{	// scaled but not imputed
-						_scnomvList[idx] = colID;
-						_scnomvMethodList[idx] = MVMethod.values()[mthd];
-						_scnomvMeanList[idx] = new KahanObject(0, 0);
-						_scnomvVarList[idx] = new CM_COV_Object();
-						idx++;
-					}
-				}
-			}
-		}
-	}
-
-	private void parseMethodsAndReplacments(JSONObject parsedSpec) throws JSONException {
-		JSONArray mvspec = (JSONArray) parsedSpec.get(TfUtils.TXMETHOD_IMPUTE);
-		_mvMethodList = new MVMethod[mvspec.size()];
-		_replacementList = new String[mvspec.size()];
-		_meanList = new KahanObject[mvspec.size()];
-		_countList = new long[mvspec.size()];
-		for(int i=0; i < mvspec.size(); i++) {
-			JSONObject mvobj = (JSONObject)mvspec.get(i);
-			_mvMethodList[i] = MVMethod.valueOf(mvobj.get("method").toString().toUpperCase()); 
-			if( _mvMethodList[i] == MVMethod.CONSTANT ) {
-				_replacementList[i] = mvobj.getString("value").toString();
-			}
-			_meanList[i] = new KahanObject(0, 0);
-		}
-	}
-	
-	
-	public void prepare(String[] words) throws IOException {
-		
-		try {
-			String w = null;
-			if(_colList != null)
-			for(int i=0; i <_colList.length; i++) {
-				int colID = _colList[i];
-				w = UtilFunctions.unquote(words[colID-1].trim());
-				
-				try {
-				if(!TfUtils.isNA(_NAstrings, w)) {
-					_countList[i]++;
-					
-					boolean computeMean = (_mvMethodList[i] == MVMethod.GLOBAL_MEAN || _isMVScaled.get(i) );
-					if(computeMean) {
-						// global_mean
-						double d = UtilFunctions.parseToDouble(w);
-						_meanFn.execute2(_meanList[i], d, _countList[i]);
-						
-						if (_isMVScaled.get(i) && _mvscMethodList[i] == MVMethod.GLOBAL_MODE)
-							_varFn.execute(_varList[i], d);
-					}
-					else {
-						// global_mode or constant
-						// Nothing to do here. Mode is computed using recode maps.
-					}
-				}
-				} catch (NumberFormatException e) 
-				{
-					throw new RuntimeException("Encountered \"" + w + "\" in column ID \"" + colID + "\", when expecting a numeric value. Consider adding \"" + w + "\" to na.strings, along with an appropriate imputation method.");
-				}
-			}
-			
-			// Compute mean and variance for attributes that are scaled but not imputed
-			if(_scnomvList != null)
-			for(int i=0; i < _scnomvList.length; i++) 
-			{
-				int colID = _scnomvList[i];
-				w = UtilFunctions.unquote(words[colID-1].trim());
-				double d = UtilFunctions.parseToDouble(w);
-				_scnomvCountList[i]++; 		// not required, this is always equal to total #records processed
-				_meanFn.execute2(_scnomvMeanList[i], d, _scnomvCountList[i]);
-				if(_scnomvMethodList[i] == MVMethod.GLOBAL_MODE)
-					_varFn.execute(_scnomvVarList[i], d);
-			}
-		} catch(Exception e) {
-			throw new IOException(e);
-		}
-	}
-	
-	// ----------------------------------------------------------------------------------------------------------
-	
-	private String encodeCMObj(CM_COV_Object obj)
-	{
-		StringBuilder sb = new StringBuilder();
-		sb.append(obj.w);
-		sb.append(",");
-		sb.append(obj.mean._sum);
-		sb.append(",");
-		sb.append(obj.mean._correction);
-		sb.append(",");
-		sb.append(obj.m2._sum);
-		sb.append(",");
-		sb.append(obj.m2._correction);
-		return sb.toString();
-	}
-	
-	private CM_COV_Object decodeCMObj(String s) 
-	{
-		CM_COV_Object obj = new CM_COV_Object();
-		String[] parts = s.split(",");
-		obj.w = UtilFunctions.parseToDouble(parts[0]);
-		obj.mean._sum = UtilFunctions.parseToDouble(parts[1]);
-		obj.mean._correction = UtilFunctions.parseToDouble(parts[2]);
-		obj.m2._sum = UtilFunctions.parseToDouble(parts[3]);
-		obj.m2._correction = UtilFunctions.parseToDouble(parts[4]);
-		
-		return obj;
-	}
-	
-	private DistinctValue prepMeanOutput(int taskID, int idx, StringBuilder sb, boolean scnomv) throws CharacterCodingException {
-		
-		MVMethod mthd = (scnomv ? _scnomvMethodList[idx] : _mvMethodList[idx]);
-		
-		if ( scnomv || mthd == MVMethod.GLOBAL_MEAN || _isMVScaled.get(idx) ) {
-			String suffix = null;
-			if(scnomv)
-				suffix = "scnomv";
-			else if ( mthd == MVMethod.GLOBAL_MEAN && _isMVScaled.get(idx) )
-				suffix = "scmv"; 	// both scaled and mv imputed
-			else if ( mthd == MVMethod.GLOBAL_MEAN )
-				suffix = "noscmv";
-			else
-				suffix = "scnomv";
-			
-			sb.setLength(0);
-			sb.append(MEAN_PREFIX);
-			sb.append("_");
-			sb.append(taskID);
-			sb.append("_");
-			double mean = (scnomv ? _scnomvMeanList[idx]._sum : _meanList[idx]._sum);
-			sb.append(Double.toString(mean));
-			sb.append(",");
-			sb.append(suffix);
-			//String s = MEAN_PREFIX + "_" + taskID + "_" + Double.toString(_meanList[idx]._sum) + "," + suffix;
-			return new DistinctValue(sb.toString(), -1L);
-		}
-		
-		return null;
-	}
-	
-	private DistinctValue prepMeanCorrectionOutput(int taskID, int idx, StringBuilder sb, boolean scnomv) throws CharacterCodingException {
-		MVMethod mthd = (scnomv ? _scnomvMethodList[idx] : _mvMethodList[idx]);
-		if ( scnomv || mthd == MVMethod.GLOBAL_MEAN || _isMVScaled.get(idx) ) {
-			sb.setLength(0);
-			//CORRECTION_PREFIX + "_" + taskID + "_" + Double.toString(mean._correction);
-			sb.append(CORRECTION_PREFIX);
-			sb.append("_");
-			sb.append(taskID);
-			sb.append("_");
-			double corr = (scnomv ? _scnomvMeanList[idx]._correction : _meanList[idx]._correction);
-			sb.append(Double.toString(corr));
-			return new DistinctValue(sb.toString(), -1L);
-		}
-		return null;
-	}
-	
-	private DistinctValue prepMeanCountOutput(int taskID, int idx, StringBuilder sb, boolean scnomv) throws CharacterCodingException {
-		MVMethod mthd = (scnomv ? _scnomvMethodList[idx] : _mvMethodList[idx]);
-		if ( scnomv || mthd == MVMethod.GLOBAL_MEAN || _isMVScaled.get(idx) ) {
-			sb.setLength(0);
-			//s = COUNT_PREFIX + "_" + taskID + "_" + Long.toString(count);
-			sb.append(COUNT_PREFIX);
-			sb.append("_");
-			sb.append(taskID);
-			sb.append("_");
-			long count = (scnomv ? _scnomvCountList[idx] : _countList[idx]);
-			sb.append( Long.toString(count));
-			return new DistinctValue(sb.toString(), -1L);
-		}
-		return null;
-	}
-	
-	private DistinctValue prepTotalCountOutput(int taskID, int idx, StringBuilder sb, boolean scnomv, TfUtils agents) throws CharacterCodingException {
-		MVMethod mthd = (scnomv ? _scnomvMethodList[idx] : _mvMethodList[idx]);
-		if ( scnomv || mthd == MVMethod.GLOBAL_MEAN || _isMVScaled.get(idx) ) {
-			sb.setLength(0);
-			//TOTAL_COUNT_PREFIX + "_" + taskID + "_" + Long.toString(TransformationAgent._numValidRecords);
-			sb.append(TOTAL_COUNT_PREFIX);
-			sb.append("_");
-			sb.append(taskID);
-			sb.append("_");
-			sb.append( Long.toString(agents.getValid()) );
-			return new DistinctValue(sb.toString(), -1L);
-		}
-		return null;
-	}
-	
-	private DistinctValue prepConstantOutput(int idx, StringBuilder sb) throws CharacterCodingException {
-		if ( _mvMethodList == null )
-			return null;
-		MVMethod mthd = _mvMethodList[idx];
-		if ( mthd == MVMethod.CONSTANT ) {
-			sb.setLength(0);
-			sb.append(CONSTANT_PREFIX);
-			sb.append("_");
-			sb.append(_replacementList[idx]);
-			return new DistinctValue(sb.toString(), -1);
-		}
-		return null;
-	}
-	
-	private DistinctValue prepVarOutput(int taskID, int idx, StringBuilder sb, boolean scnomv) throws CharacterCodingException {
-		if ( scnomv || _isMVScaled.get(idx) && _mvscMethodList[idx] == MVMethod.GLOBAL_MODE ) {
-			sb.setLength(0);
-			sb.append(VARIANCE_PREFIX);
-			sb.append("_");
-			sb.append(taskID);
-			sb.append("_");
-			CM_COV_Object cm = (scnomv ? _scnomvVarList[idx] : _varList[idx]);
-			sb.append(encodeCMObj(cm));
-		
-			return new DistinctValue(sb.toString(), -1L);
-		}
-		return null;
-	}
-	
-	private void outDV(IntWritable iw, DistinctValue dv, OutputCollector<IntWritable, DistinctValue> out) throws IOException {
-		if ( dv != null )	
-			out.collect(iw, dv);
-	}
-	
-	/**
-	 * Method to output transformation metadata from the mappers. 
-	 * This information is collected and merged by the reducers.
-	 */
-	@Override
-	public void mapOutputTransformationMetadata(OutputCollector<IntWritable, DistinctValue> out, int taskID, TfUtils agents) throws IOException {
-		try { 
-			StringBuilder sb = new StringBuilder();
-			DistinctValue dv = null;
-			
-			if(_colList != null)
-				for(int i=0; i < _colList.length; i++) {
-					int colID = _colList[i];
-					IntWritable iw = new IntWritable(-colID);
-					
-					dv = prepMeanOutput(taskID, i, sb, false);				outDV(iw, dv, out);
-					dv = prepMeanCorrectionOutput(taskID, i, sb, false);	outDV(iw, dv, out);
-					dv = prepMeanCountOutput(taskID, i, sb, false);			outDV(iw, dv, out);
-					dv = prepTotalCountOutput(taskID, i, sb, false, agents); outDV(iw, dv, out);
-					
-					dv = prepConstantOutput(i, sb);							outDV(iw, dv, out);
-					
-					// output variance information relevant to scaling
-					dv = prepVarOutput(taskID, i, sb, false);				outDV(iw, dv, out);
-				}
-			
-			// handle attributes that are scaled but not imputed
-			if(_scnomvList != null)
-				for(int i=0; i < _scnomvList.length; i++)
-				{
-					int colID = _scnomvList[i];
-					IntWritable iw = new IntWritable(-colID);
-					
-					dv = prepMeanOutput(taskID, i, sb, true);				outDV(iw, dv, out);
-					dv = prepMeanCorrectionOutput(taskID, i, sb, true);		outDV(iw, dv, out);
-					dv = prepMeanCountOutput(taskID, i, sb, true);			outDV(iw, dv, out);
-					dv = prepTotalCountOutput(taskID, i, sb, true, agents);	outDV(iw, dv, out);
-					
-					dv = prepVarOutput(taskID, i, sb, true);				outDV(iw, dv, out); 
-				}
-		} catch(Exception e) {
-			throw new IOException(e);
-		}
-	}
-	
-	/**
-	 * Applicable when running on SPARK.
-	 * Helper function to output transformation metadata into shuffle.
-	 * 
-	 * @param iw integer value
-	 * @param dv distinct value
-	 * @param list list of integer-distinct value pairs
-	 * @throws IOException if IOException occurs
-	 */
-	private void addDV(Integer iw, DistinctValue dv, ArrayList<Pair<Integer, DistinctValue>> list) throws IOException {
-		if ( dv != null )	
-			list.add( new Pair<Integer, DistinctValue>(iw, dv) );	
-	}
-
-	public ArrayList<Pair<Integer, DistinctValue>> mapOutputTransformationMetadata(int taskID, ArrayList<Pair<Integer, DistinctValue>> list, TfUtils agents) throws IOException {
-		try { 
-			StringBuilder sb = new StringBuilder();
-			DistinctValue dv = null;
-			
-			if(_colList != null)
-				for(int i=0; i < _colList.length; i++) {
-					int colID = _colList[i];
-					Integer iw = -colID;
-					
-					dv = prepMeanOutput(taskID, i, sb, false);				addDV(iw, dv, list);
-					dv = prepMeanCorrectionOutput(taskID, i, sb, false);	addDV(iw, dv, list);
-					dv = prepMeanCountOutput(taskID, i, sb, false);			addDV(iw, dv, list);
-					dv = prepTotalCountOutput(taskID, i, sb, false, agents); addDV(iw, dv, list);
-					
-					dv = prepConstantOutput(i, sb);							addDV(iw, dv, list);
-					
-					// output variance information relevant to scaling
-					dv = prepVarOutput(taskID, i, sb, false);				addDV(iw, dv, list);
-				}
-			
-			// handle attributes that are scaled but not imputed
-			if(_scnomvList != null)
-				for(int i=0; i < _scnomvList.length; i++)
-				{
-					int colID = _scnomvList[i];
-					Integer iw = -colID;
-					
-					dv = prepMeanOutput(taskID, i, sb, true);				addDV(iw, dv, list);
-					dv = prepMeanCorrectionOutput(taskID, i, sb, true);		addDV(iw, dv, list);
-					dv = prepMeanCountOutput(taskID, i, sb, true);			addDV(iw, dv, list);
-					dv = prepTotalCountOutput(taskID, i, sb, true, agents);	addDV(iw, dv, list);
-					
-					dv = prepVarOutput(taskID, i, sb, true);				addDV(iw, dv, list); 
-				}
-		} catch(Exception e) {
-			throw new IOException(e);
-		}
-		return list;
-	}
-	
-	// ----------------------------------------------------------------------------------------------------------
-	
-	private void writeTfMtd(int colID, String mean, String tfMtdDir, FileSystem fs, TfUtils agents) throws IOException 
-	{
-		Path pt=new Path(tfMtdDir+"/Impute/"+ agents.getName(colID) + TfUtils.TXMTD_MV_FILE_SUFFIX);
-		try( BufferedWriter br=new BufferedWriter(new OutputStreamWriter(fs.create(pt,true))) ) {
-			br.write(colID + TfUtils.TXMTD_SEP + mean + "\n");
-		}
-	}
-	
-	private void writeTfMtd(int colID, String mean, String sdev, String tfMtdDir, FileSystem fs, TfUtils agents) throws IOException 
-	{
-		Path pt=new Path(tfMtdDir+"/Scale/"+ agents.getName(colID) + TfUtils.SCALE_FILE_SUFFIX);
-		try( BufferedWriter br=new BufferedWriter(new OutputStreamWriter(fs.create(pt,true))) ) {
-			br.write(colID + TfUtils.TXMTD_SEP + mean + TfUtils.TXMTD_SEP + sdev + "\n");
-		}
-	}
-	
-	private void writeTfMtd(int colID, String min, String max, String binwidth, String nbins, String tfMtdDir, FileSystem fs, TfUtils agents) throws IOException 
-	{
-		Path pt = new Path(tfMtdDir+"/Bin/"+ agents.getName(colID) + TfUtils.TXMTD_BIN_FILE_SUFFIX);
-		try( BufferedWriter br=new BufferedWriter(new OutputStreamWriter(fs.create(pt,true))) ) {
-			br.write(colID + TfUtils.TXMTD_SEP + min + TfUtils.TXMTD_SEP + max + TfUtils.TXMTD_SEP + binwidth + TfUtils.TXMTD_SEP + nbins + "\n");
-		}
-	}
-	
-	public void outputTransformationMetadata(String outputDir, FileSystem fs, TfUtils agents) throws IOException {
-		
-		try{
-			if (_colList != null)
-				for(int i=0; i < _colList.length; i++) {
-					int colID = _colList[i];
-					
-					double imputedValue = Double.NaN;
-					KahanObject gmean = null;
-					if ( _mvMethodList[i] == MVMethod.GLOBAL_MEAN ) 
-					{
-						gmean = _meanList[i];
-						imputedValue = _meanList[i]._sum;
-						
-						double mean = ( _countList[i] == 0 ? 0.0 : _meanList[i]._sum); 
-						writeTfMtd(colID, Double.toString(mean), outputDir, fs, agents);
-					}
-					else if ( _mvMethodList[i] == MVMethod.CONSTANT ) 
-					{
-						writeTfMtd(colID, _replacementList[i], outputDir, fs, agents);
-						
-						if (_isMVScaled.get(i) )
-						{
-							imputedValue = UtilFunctions.parseToDouble(_replacementList[i]);
-							// adjust the global mean, by combining gmean with "replacement" (weight = #missing values)
-							gmean = new KahanObject(_meanList[i]._sum, _meanList[i]._correction);
-							_meanFn.execute(gmean, imputedValue, agents.getValid());
-						}
-					}
-						
-					if ( _isMVScaled.get(i) ) 
-					{
-						double sdev = -1.0;
-						if ( _mvscMethodList[i] == MVMethod.GLOBAL_MODE ) {
-							// Adjust variance with missing values
-							long totalMissingCount = (agents.getValid() - _countList[i]);
-							_varFn.execute(_varList[i], imputedValue, totalMissingCount);
-							double var = _varList[i].getRequiredResult(new CMOperator(_varFn, AggregateOperationTypes.VARIANCE));
-							sdev = Math.sqrt(var);
-						}
-						writeTfMtd(colID, Double.toString(gmean._sum), Double.toString(sdev), outputDir, fs, agents);
-					}
-				}
-		
-			if(_scnomvList != null)
-				for(int i=0; i < _scnomvList.length; i++ )
-				{
-					int colID = _scnomvList[i];
-					double mean = (_scnomvCountList[i] == 0 ? 0.0 : _scnomvMeanList[i]._sum);
-					double sdev = -1.0;
-					if ( _scnomvMethodList[i] == MVMethod.GLOBAL_MODE ) 
-					{
-						double var = _scnomvVarList[i].getRequiredResult(new CMOperator(_varFn, AggregateOperationTypes.VARIANCE));
-						sdev = Math.sqrt(var);
-					}
-					writeTfMtd(colID, Double.toString(mean), Double.toString(sdev), outputDir, fs, agents);
-				}
-			
-		} catch(DMLRuntimeException e) {
-			throw new IOException(e); 
-		}
-	}
-	
-	/** 
-	 * Method to merge map output transformation metadata. 
-	 */
-	@Override
-	public void mergeAndOutputTransformationMetadata(Iterator<DistinctValue> values, String outputDir, int colID, FileSystem fs, TfUtils agents) throws IOException {
-		double min = Double.MAX_VALUE;
-		double max = -Double.MAX_VALUE;
-		int nbins = 0;
-		double d;
-		long totalRecordCount = 0, totalValidCount=0;
-		String mvConstReplacement = null;
-		
-		DistinctValue val = new DistinctValue();
-		String w = null;
-		
-		class MeanObject {
-			double mean, correction;
-			long count;
-			
-			MeanObject() { }
-			public String toString() {
-				return mean + "," + correction + "," + count;
-			}
-		};
-		HashMap<Integer, MeanObject> mapMeans = new HashMap<Integer, MeanObject>();
-		HashMap<Integer, CM_COV_Object> mapVars = new HashMap<Integer, CM_COV_Object>();
-		boolean isImputed = false;
-		boolean isScaled = false;
-		boolean isBinned = false;
-		
-		while(values.hasNext()) {
-			val.reset();
-			val = values.next();
-			w = val.getWord();
-			
-			if(w.startsWith(MEAN_PREFIX)) {
-				String[] parts = w.split("_");
-				int taskID = UtilFunctions.parseToInt(parts[1]);
-				MeanObject mo = mapMeans.get(taskID);
-				if ( mo==null ) 
-					mo = new MeanObject();
-				
-				mo.mean = UtilFunctions.parseToDouble(parts[2].split(",")[0]);
-				
-				// check if this attribute is scaled
-				String s = parts[2].split(",")[1]; 
-				if(s.equalsIgnoreCase("scmv"))
-					isScaled = isImputed = true;
-				else if ( s.equalsIgnoreCase("scnomv") )
-					isScaled = true;
-				else
-					isImputed = true;
-				
-				mapMeans.put(taskID, mo);
-			}
-			else if (w.startsWith(CORRECTION_PREFIX)) {
-				String[] parts = w.split("_");
-				int taskID = UtilFunctions.parseToInt(parts[1]);
-				MeanObject mo = mapMeans.get(taskID);
-				if ( mo==null ) 
-					mo = new MeanObject();
-				mo.correction = UtilFunctions.parseToDouble(parts[2]);
-				mapMeans.put(taskID, mo);
-			}
-			else if ( w.startsWith(CONSTANT_PREFIX) )
-			{
-				isImputed = true;
-				String[] parts = w.split("_");
-				mvConstReplacement = parts[1];
-			}
-			else if (w.startsWith(COUNT_PREFIX)) {
-				String[] parts = w.split("_");
-				int taskID = UtilFunctions.parseToInt(parts[1]);
-				MeanObject mo = mapMeans.get(taskID);
-				if ( mo==null ) 
-					mo = new MeanObject();
-				mo.count = UtilFunctions.parseToLong(parts[2]);
-				totalValidCount += mo.count;
-				mapMeans.put(taskID, mo);
-			}
-			else if (w.startsWith(TOTAL_COUNT_PREFIX)) {
-				String[] parts = w.split("_");
-				//int taskID = UtilFunctions.parseToInt(parts[1]);
-				totalRecordCount += UtilFunctions.parseToLong(parts[2]);
-			}
-			else if (w.startsWith(VARIANCE_PREFIX)) {
-				isScaled = true;
-				String[] parts = w.split("_");
-				int taskID = UtilFunctions.parseToInt(parts[1]);
-				CM_COV_Object cm = decodeCMObj(parts[2]);
-				mapVars.put(taskID, cm);
-			}
-			else if(w.startsWith(BinAgent.MIN_PREFIX)) {
-				isBinned = true;
-				d = UtilFunctions.parseToDouble( w.substring( BinAgent.MIN_PREFIX.length() ) );
-				if ( d < min )
-					min = d;
-			}
-			else if(w.startsWith(BinAgent.MAX_PREFIX)) {
-				isBinned = true;
-				d = UtilFunctions.parseToDouble( w.substring( BinAgent.MAX_PREFIX.length() ) );
-				if ( d > max )
-					max = d;
-			}
-			else if (w.startsWith(BinAgent.NBINS_PREFIX)) {
-				isBinned = true;
-				nbins = (int) UtilFunctions.parseToLong( w.substring(BinAgent.NBINS_PREFIX.length() ) );
-			}
-			else
-				throw new RuntimeException("MVImputeAgent: Invalid prefix while merging map output: " + w);
-		}
-		
-		// compute global mean across all map outputs
-		KahanObject gmean = new KahanObject(0, 0);
-		KahanPlus kp = KahanPlus.getKahanPlusFnObject();
-		long gcount = 0;
-		for(MeanObject mo : mapMeans.values()) {
-			gcount = gcount + mo.count;
-			if ( gcount > 0) {
-				double delta = mo.mean - gmean._sum;
-				kp.execute2(gmean, delta*mo.count/gcount);
-				//_meanFn.execute2(gmean, mo.mean*mo.count, gcount);
-			}
-		}
-		
-		// compute global variance across all map outputs
-		CM_COV_Object gcm = new CM_COV_Object();
-		try {
-			for(CM_COV_Object cm : mapVars.values())
-				gcm = (CM_COV_Object) _varFn.execute(gcm, cm);
-		} catch (DMLRuntimeException e) {
-			throw new IOException(e);
-		}
-		
-		// If the column is imputed with a constant, then adjust min and max based the value of the constant.
-		if(isImputed && isBinned && mvConstReplacement != null)
-		{
-			double cst = UtilFunctions.parseToDouble(mvConstReplacement);
-			if ( cst < min)
-				min = cst;
-			if ( cst > max)
-				max = cst;
-		}
-
-		// write merged metadata
-		if( isImputed ) 
-		{
-			String imputedValue = null;
-			if ( mvConstReplacement != null )
-				imputedValue = mvConstReplacement;
-			else 
-				imputedValue = Double.toString(gcount == 0 ? 0.0 : gmean._sum);
-			
-			writeTfMtd(colID, imputedValue, outputDir, fs, agents);
-		}
-		
-		if ( isBinned ) {
-			double binwidth = (max-min)/nbins;
-			writeTfMtd(colID, Double.toString(min), Double.toString(max), Double.toString(binwidth), Integer.toString(nbins), outputDir, fs, agents);
-		}
-		
-		if ( isScaled ) 
-		{
-			try {
-				if( totalValidCount != totalRecordCount) {
-					// In the presence of missing values, the variance needs to be adjusted.
-					// The mean does not need to be adjusted, when mv impute method is global_mean, 
-					// since missing values themselves are replaced with gmean.
-					long totalMissingCount = (totalRecordCount-totalValidCount);
-					int idx = isApplicable(colID);
-					if(idx != -1 && _mvMethodList[idx] == MVMethod.CONSTANT) 
-						_meanFn.execute(gmean, UtilFunctions.parseToDouble(_replacementList[idx]), totalRecordCount);
-					_varFn.execute(gcm, gmean._sum, totalMissingCount);
-				}
-				
-				double mean = (gcount == 0 ? 0.0 : gmean._sum);
-				double var = gcm.getRequiredResult(new CMOperator(_varFn, AggregateOperationTypes.VARIANCE));
-				double sdev = (mapVars.size() > 0 ? Math.sqrt(var) : -1.0 );
-				
-				writeTfMtd(colID, Double.toString(mean), Double.toString(sdev), outputDir, fs, agents);
-				
-				
-			} catch (DMLRuntimeException e) {
-				throw new IOException(e);
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------------------------------
-
-	private String readReplacement(int colID, FileSystem fs, Path  txMtdDir, TfUtils agents) throws IOException
-	{
-		Path path = new Path( txMtdDir + "/Impute/" + agents.getName(colID) + TfUtils.TXMTD_MV_FILE_SUFFIX);
-		TfUtils.checkValidInputFile(fs, path, true); 
-		String replacement = null;
-		try( BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(path))) ) {
-			String line = br.readLine();
-			replacement = UtilFunctions.unquote(line.split(TfUtils.TXMTD_SEP)[1]);
-		}
-		
-		return replacement;
-	}
-	
-	public String readScaleLine(int colID, FileSystem fs, Path txMtdDir, TfUtils agents) throws IOException
-	{
-		Path path = new Path( txMtdDir + "/Scale/" + agents.getName(colID) + TfUtils.SCALE_FILE_SUFFIX);
-		TfUtils.checkValidInputFile(fs, path, true); 
-		String line = null;
-		try( BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(path))) ) {
-			line = br.readLine();
-		}
-		
-		return line;
-	}
-	
-	private void processScalingFile(int i, int[] list, KahanObject[] meanList, CM_COV_Object[] varList, FileSystem fs, Path tfMtdDir, TfUtils agents ) throws IOException
-	{
-		int colID = list[i];
-		
-		String line = readScaleLine(colID, fs, tfMtdDir, agents);
-		String[] parts = line.split(",");
-		double mean = UtilFunctions.parseToDouble(parts[1]);
-		double sd = UtilFunctions.parseToDouble(parts[2]);
-		
-		meanList[i]._sum = mean;
-		varList[i].mean._sum = sd;
-	}
-	
-	// ------------------------------------------------------------------------------------------------
-
-	/**
-	 * Method to load transform metadata for all attributes
-	 */
-	@Override
-	public void loadTxMtd(JobConf job, FileSystem fs, Path tfMtdDir, TfUtils agents) throws IOException {
-		
-		if(fs.isDirectory(tfMtdDir)) {
-			
-			// Load information about missing value imputation
-			if (_colList != null)
-				for(int i=0; i<_colList.length;i++) {
-					int colID = _colList[i];
-					
-					if ( _mvMethodList[i] == MVMethod.GLOBAL_MEAN || _mvMethodList[i] == MVMethod.GLOBAL_MODE )
-						// global_mean or global_mode
-						_replacementList[i] = readReplacement(colID, fs, tfMtdDir, agents);
-					else if ( _mvMethodList[i] == MVMethod.CONSTANT ) {
-						// constant: replace a missing value by a given constant
-						// nothing to do. The constant values are loaded already during configure 
-					}
-					else
-						throw new RuntimeException("Invalid Missing Value Imputation methods: " + _mvMethodList[i]);
-				}
-			
-			// Load scaling information
-			if(_colList != null)
-				for(int i=0; i < _colList.length; i++)
-					if ( _isMVScaled.get(i) ) 
-						processScalingFile(i, _colList, _meanList, _varList, fs, tfMtdDir, agents);
-			
-			if(_scnomvList != null)
-				for(int i=0; i < _scnomvList.length; i++)
-					processScalingFile(i, _scnomvList, _scnomvMeanList, _scnomvVarList, fs, tfMtdDir, agents);
-		}
-		else {
-			throw new RuntimeException("Path to recode maps must be a directory: " + tfMtdDir);
-		}
-	}
-	
-	public MVMethod getMethod(int colID) {
-		int idx = isApplicable(colID);		
-		if(idx == -1)
-			return MVMethod.INVALID;
-		else
-			return _mvMethodList[idx];
-	}
-	
-	public long getNonMVCount(int colID) {
-		int idx = isApplicable(colID);
-		return (idx == -1) ? 0 : _countList[idx];
-	}
-	
-	public String getReplacement(int colID)  {
-		int idx = isApplicable(colID);		
-		return (idx == -1) ? null : _replacementList[idx];
-	}
-	
-	@Override
-	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
-		build(in);
-		return apply(in, out);
-	}
-	
-	@Override
-	public void build(FrameBlock in) {
-		try {
-			for( int j=0; j<_colList.length; j++ ) {
-				int colID = _colList[j];
-				if( _mvMethodList[j] == MVMethod.GLOBAL_MEAN ) {
-					//compute global column mean (scale)
-					long off = _countList[j];
-					for( int i=0; i<in.getNumRows(); i++ )
-						_meanFn.execute2(_meanList[j], UtilFunctions.objectToDouble(
-							in.getSchema()[colID-1], in.get(i, colID-1)), off+i+1);
-					_replacementList[j] = String.valueOf(_meanList[j]._sum);
-					_countList[j] += in.getNumRows();
-				}
-				else if( _mvMethodList[j] == MVMethod.GLOBAL_MODE ) {
-					//compute global column mode (categorical), i.e., most frequent category
-					HashMap<String,Long> hist = _hist.containsKey(colID) ? 
-							_hist.get(colID) : new HashMap<String,Long>();
-					for( int i=0; i<in.getNumRows(); i++ ) {
-						String key = String.valueOf(in.get(i, colID-1));
-						if( key != null && !key.isEmpty() ) {
-							Long val = hist.get(key);
-							hist.put(key, (val!=null) ? val+1 : 1);
-						}	
-					}
-					_hist.put(colID, hist);
-					long max = Long.MIN_VALUE; 
-					for( Entry<String, Long> e : hist.entrySet() ) 
-						if( e.getValue() > max  ) {
-							_replacementList[j] = e.getKey();
-							max = e.getValue();
-						}
-				}
-			}
-		}
-		catch(Exception ex) {
-			throw new RuntimeException(ex);
-		}
-	}
-
-	@Override
-	public String[] apply(String[] words) 
-	{	
-		if( isApplicable() )
-			for(int i=0; i < _colList.length; i++) {
-				int colID = _colList[i];
-				String w = UtilFunctions.unquote(words[colID-1]);
-				if(TfUtils.isNA(_NAstrings, w))
-					w = words[colID-1] = _replacementList[i];
-				
-				if ( _isMVScaled.get(i) )
-					if ( _mvscMethodList[i] == MVMethod.GLOBAL_MEAN )
-						words[colID-1] = Double.toString( UtilFunctions.parseToDouble(w) - _meanList[i]._sum );
-					else
-						words[colID-1] = Double.toString( (UtilFunctions.parseToDouble(w) - _meanList[i]._sum) / _varList[i].mean._sum );
-			}
-		
-		if(_scnomvList != null)
-		for(int i=0; i < _scnomvList.length; i++)
-		{
-			int colID = _scnomvList[i];
-			if ( _scnomvMethodList[i] == MVMethod.GLOBAL_MEAN )
-				words[colID-1] = Double.toString( UtilFunctions.parseToDouble(words[colID-1]) - _scnomvMeanList[i]._sum );
-			else
-				words[colID-1] = Double.toString( (UtilFunctions.parseToDouble(words[colID-1]) - _scnomvMeanList[i]._sum) / _scnomvVarList[i].mean._sum );
-		}
-			
-		return words;
-	}
-	
-	@Override
-	public MatrixBlock apply(FrameBlock in, MatrixBlock out) {
-		for(int i=0; i<in.getNumRows(); i++) {
-			for(int j=0; j<_colList.length; j++) {
-				int colID = _colList[j];
-				if( Double.isNaN(out.quickGetValue(i, colID-1)) )
-					out.quickSetValue(i, colID-1, Double.parseDouble(_replacementList[j]));
-			}
-		}
-		return out;
-	}
-	
-	@Override
-	public FrameBlock getMetaData(FrameBlock out) {
-		for( int j=0; j<_colList.length; j++ ) {
-			out.getColumnMetadata(_colList[j]-1)
-			   .setMvValue(_replacementList[j]);
-		}
-		return out;
-	}
-
-	public void initMetaData(FrameBlock meta) {
-		//init replacement lists, replace recoded values to
-		//apply mv imputation potentially after recoding
-		for( int j=0; j<_colList.length; j++ ) {
-			int colID = _colList[j];	
-			String mvVal = UtilFunctions.unquote(meta.getColumnMetadata(colID-1).getMvValue()); 
-			if( _rcList.contains(colID) ) {
-				Long mvVal2 = meta.getRecodeMap(colID-1).get(mvVal);
-				if( mvVal2 == null)
-					throw new RuntimeException("Missing recode value for impute value '"+mvVal+"' (colID="+colID+").");
-				_replacementList[j] = mvVal2.toString();
-			}
-			else {
-				_replacementList[j] = mvVal;
-			}
-		}
-	}
-
-	public void initRecodeIDList(List<Integer> rcList) {
-		_rcList = rcList;
-	}
-	
-	/**
-	 * Exposes the internal histogram after build.
-	 * 
-	 * @param colID column ID
-	 * @return histogram (map of string keys and long values)
-	 */
-	public HashMap<String,Long> getHistogram( int colID ) {
-		return _hist.get(colID);
-	}
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/OmitAgent.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/OmitAgent.java b/src/main/java/org/apache/sysml/runtime/transform/OmitAgent.java
deleted file mode 100644
index 0208c0e..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/OmitAgent.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * 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.runtime.transform;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.wink.json4j.JSONException;
-import org.apache.wink.json4j.JSONObject;
-import org.apache.sysml.runtime.matrix.data.FrameBlock;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.transform.encode.Encoder;
-import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
-import org.apache.sysml.runtime.util.UtilFunctions;
-
-public class OmitAgent extends Encoder 
-{	
-	private static final long serialVersionUID = 1978852120416654195L;
-
-	private int _rmRows = 0;
-
-	public OmitAgent(JSONObject parsedSpec, String[] colnames, int clen) 
-		throws JSONException 
-	{
-		super(null, clen);
-		if (!parsedSpec.containsKey(TfUtils.TXMETHOD_OMIT))
-			return;
-		int[] collist = TfMetaUtils.parseJsonIDList(parsedSpec, colnames, TfUtils.TXMETHOD_OMIT);
-		initColList(collist);
-	}
-	
-	public int getNumRemovedRows() {
-		return _rmRows;
-	}
-	
-	public boolean omit(String[] words, TfUtils agents) 
-	{
-		if( !isApplicable() )
-			return false;
-		
-		for(int i=0; i<_colList.length; i++) {
-			int colID = _colList[i];
-			if(TfUtils.isNA(agents.getNAStrings(),UtilFunctions.unquote(words[colID-1].trim())))
-				return true;
-		}
-		return false;
-	}
-
-	@Override
-	public void mapOutputTransformationMetadata(
-			OutputCollector<IntWritable, DistinctValue> out, int taskID,
-			TfUtils agents) throws IOException {
-	}
-
-	@Override
-	public void mergeAndOutputTransformationMetadata(
-			Iterator<DistinctValue> values, String outputDir, int colID,
-			FileSystem fs, TfUtils agents) throws IOException {
-	}
-
-	@Override
-	public void loadTxMtd(JobConf job, FileSystem fs, Path txMtdDir, TfUtils agents)
-			throws IOException {
-	}
-
-	@Override
-	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
-		return apply(in, out);
-	}
-	
-	@Override
-	public void build(FrameBlock in) {	
-		//do nothing
-	}
-	
-	@Override
-	public String[] apply(String[] words) {
-		return null;
-	}
-	
-	@Override
-	public MatrixBlock apply(FrameBlock in, MatrixBlock out) 
-	{
-		//determine output size
-		int numRows = 0;
-		for(int i=0; i<out.getNumRows(); i++) {
-			boolean valid = true;
-			for(int j=0; j<_colList.length; j++)
-				valid &= !Double.isNaN(out.quickGetValue(i, _colList[j]-1));
-			numRows += valid ? 1 : 0;
-		}
-		
-		//copy over valid rows into the output
-		MatrixBlock ret = new MatrixBlock(numRows, out.getNumColumns(), false);
-		int pos = 0;
-		for(int i=0; i<in.getNumRows(); i++) {
-			//determine if valid row or omit
-			boolean valid = true;
-			for(int j=0; j<_colList.length; j++)
-				valid &= !Double.isNaN(out.quickGetValue(i, _colList[j]-1));
-			//copy row if necessary
-			if( valid ) {
-				for(int j=0; j<out.getNumColumns(); j++)
-					ret.quickSetValue(pos, j, out.quickGetValue(i, j));
-				pos++;
-			}
-		}
-	
-		//keep info an remove rows
-		_rmRows = out.getNumRows() - pos;
-		
-		return ret; 
-	}
-
-	@Override
-	public FrameBlock getMetaData(FrameBlock out) {
-		//do nothing
-		return out;
-	}
-	
-	@Override
-	public void initMetaData(FrameBlock meta) {
-		//do nothing
-	}
-}
- 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/0cd3905f/src/main/java/org/apache/sysml/runtime/transform/RecodeAgent.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/transform/RecodeAgent.java b/src/main/java/org/apache/sysml/runtime/transform/RecodeAgent.java
deleted file mode 100644
index 57ff608..0000000
--- a/src/main/java/org/apache/sysml/runtime/transform/RecodeAgent.java
+++ /dev/null
@@ -1,534 +0,0 @@
-/*
- * 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.runtime.transform;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.sysml.lops.Lop;
-import org.apache.sysml.runtime.io.IOUtilFunctions;
-import org.apache.sysml.runtime.matrix.data.FrameBlock;
-import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.matrix.data.Pair;
-import org.apache.sysml.runtime.transform.MVImputeAgent.MVMethod;
-import org.apache.sysml.runtime.transform.decode.DecoderRecode;
-import org.apache.sysml.runtime.transform.encode.Encoder;
-import org.apache.sysml.runtime.transform.meta.TfMetaUtils;
-import org.apache.sysml.runtime.util.UtilFunctions;
-import org.apache.wink.json4j.JSONException;
-import org.apache.wink.json4j.JSONObject;
-
-public class RecodeAgent extends Encoder 
-{	
-	private static final long serialVersionUID = 8213163881283341874L;
-
-	private int[] _mvrcdList = null;
-	private int[] _fullrcdList = null;
-	
-	//recode maps and custom map for partial recode maps 
-	private HashMap<Integer, HashMap<String, Long>> _rcdMaps  = new HashMap<Integer, HashMap<String, Long>>();
-	private HashMap<Integer, HashMap<String,String>> _finalMaps = null;
-	private HashMap<Integer, HashSet<Object>> _rcdMapsPart = null;
-	
-	public RecodeAgent(JSONObject parsedSpec, String[] colnames, int clen)
-		throws JSONException 
-	{
-		super(null, clen);
-		int rcdCount = 0;
-		
-		if( parsedSpec.containsKey(TfUtils.TXMETHOD_RECODE) ) {
-			int[] collist = TfMetaUtils.parseJsonIDList(parsedSpec, colnames, TfUtils.TXMETHOD_RECODE);
-			rcdCount = initColList(collist);
-		}
-		
-		if ( parsedSpec.containsKey(TfUtils.TXMETHOD_MVRCD)) {
-			_mvrcdList = TfMetaUtils.parseJsonIDList(parsedSpec, colnames, TfUtils.TXMETHOD_MVRCD);
-			rcdCount += _mvrcdList.length;
-		}
-		
-		if ( rcdCount > 0 ) {
-			_fullrcdList = new int[rcdCount];
-			int idx = -1;
-			if(_colList != null)
-				for(int i=0; i < _colList.length; i++)
-					_fullrcdList[++idx] = _colList[i]; 
-			
-			if(_mvrcdList != null)
-				for(int i=0; i < _mvrcdList.length; i++)
-					_fullrcdList[++idx] = _mvrcdList[i]; 
-		}
-	}
-	
-	public HashMap<Integer, HashMap<String,Long>> getCPRecodeMaps() { 
-		return _rcdMaps; 
-	}
-	
-	public HashMap<Integer, HashSet<Object>> getCPRecodeMapsPartial() { 
-		return _rcdMapsPart; 
-	}
-	
-	public HashMap<Integer, HashMap<String,String>> getRecodeMaps() {
-		return _finalMaps;
-	}
-	
-	void prepare(String[] words, TfUtils agents) {
-		if ( _colList == null && _mvrcdList == null )
-			return;
-		
-		String w = null;
-		for (int colID : _fullrcdList) {
-			w = UtilFunctions.unquote(words[colID-1].trim());
-			if(_rcdMaps.get(colID) == null ) 
-				_rcdMaps.put(colID, new HashMap<String, Long>());
-			
-			HashMap<String, Long> map = _rcdMaps.get(colID);
-			Long count = map.get(w);
-			if(count == null)
-				map.put(w, new Long(1));
-			else
-				map.put(w, count+1);
-		}
-	}
-	
-	private HashMap<String, Long> handleMVConstant(int colID, TfUtils agents, HashMap<String, Long> map)
-	{
-		MVImputeAgent mvagent = agents.getMVImputeAgent();
-		if ( mvagent.getMethod(colID) == MVMethod.CONSTANT ) 
-		{
-			// check if the "replacement" is part of the map. If not, add it.
-			String repValue = mvagent.getReplacement(colID);
-			if(repValue == null)
-				throw new RuntimeException("Expecting a constant replacement value for column ID " + colID);
-			
-			repValue = UtilFunctions.unquote(repValue);
-			Long count = map.get(repValue);
-			long mvCount = agents.getValid() - mvagent.getNonMVCount(colID);
-			if(count == null)
-				map.put(repValue, mvCount);
-			else
-				map.put(repValue, count + mvCount);
-		}
-		return map;
-	}
-	
-	/**
-	 * Method to output transformation metadata from the mappers. 
-	 * This information is collected and merged by the reducers.
-	 */
-	@Override
-	public void mapOutputTransformationMetadata(OutputCollector<IntWritable, DistinctValue> out, int taskID, TfUtils agents) throws IOException {
-		mapOutputHelper(taskID, out, null, agents);
-	}
-	
-	public ArrayList<Pair<Integer, DistinctValue>> mapOutputTransformationMetadata(int taskID, ArrayList<Pair<Integer, DistinctValue>> list, TfUtils agents) throws IOException {
-		mapOutputHelper(taskID, null, list, agents);
-		return list;
-	}
-	
-	public void mapOutputHelper(int taskID, OutputCollector<IntWritable, DistinctValue> out, ArrayList<Pair<Integer, DistinctValue>> list, TfUtils agents) throws IOException {
-		if ( _colList == null  && _mvrcdList == null )
-			return;
-		
-		try 
-		{ 
-			for(int i=0; i < _fullrcdList.length; i++) 
-			{
-				int colID = _fullrcdList[i];
-				HashMap<String, Long> map = _rcdMaps.get(colID);
-				
-				if(map != null) 
-				{
-					map = handleMVConstant(colID, agents,  map);
-					
-					if ( out != null ) {
-						IntWritable iw = new IntWritable(colID);
-						for(String s : map.keySet()) 
-							out.collect(iw, new DistinctValue(s, map.get(s)));
-					}
-					else if ( list != null ) {
-						for(String s : map.keySet()) 
-							list.add(new Pair<Integer,DistinctValue>(colID, new DistinctValue(s, map.get(s))) );
-					}
-				}
-			}
-		} catch(Exception e) {
-			throw new IOException(e);
-		}
-	}
-	
-	/**
-	 * Function to output transformation metadata, including: 
-	 * - recode maps, 
-	 * - number of distinct values, 
-	 * - mode, and 
-	 * - imputation value (in the case of global_mode)
-	 * 
-	 * The column for which this function is invoked can be one of the following:
-	 * - just recoded						(write .map, .ndistinct, .mode)
-	 * - just mv imputed (w/ global_mode)	(write .impute)
-	 * - both recoded and mv imputed		(write .map, .ndistinct, .mode, .impute)
-	 * 
-	 * @param map recode maps
-	 * @param outputDir output directory
-	 * @param colID column id
-	 * @param fs file system
-	 * @param agents ?
-	 * @param fromCP ?
-	 * @throws IOException if IOException occurs
-	 */
-	private void writeMetadata(HashMap<String,Long> map, String outputDir, int colID, FileSystem fs, TfUtils agents, boolean fromCP) throws IOException {
-		// output recode maps and mode
-		
-		MVImputeAgent mvagent = agents.getMVImputeAgent();
-		String mode = null;
-		Long count = null;
-		int rcdIndex = 0, modeIndex = 0;
-		long maxCount = Long.MIN_VALUE;
-		
-		boolean isRecoded = (isApplicable(colID) != -1);
-		boolean isModeImputed = (mvagent.getMethod(colID) == MVMethod.GLOBAL_MODE);
-		
-		Path pt=new Path(outputDir+"/Recode/"+ agents.getName(colID) + TfUtils.TXMTD_RCD_MAP_SUFFIX);
-		BufferedWriter br=null;
-		try { 
-			if(isRecoded)
-				br = new BufferedWriter(new OutputStreamWriter(fs.create(pt,true)));		
-	
-			// remove NA strings
-			if ( agents.getNAStrings() != null)
-				for(String naword : agents.getNAStrings()) 
-					map.remove(naword);
-			
-			if(fromCP)
-				map = handleMVConstant(colID, agents,  map);
-			
-			if ( map.size() == 0 ) 
-				throw new RuntimeException("Can not proceed since \"" + agents.getName(colID) + "\" (id=" + colID + ") contains only the missing values, and not a single valid value -- set imputation method to \"constant\".");
-			
-			// Order entries by category (string) value
-			List<String> newNames = new ArrayList<String>(map.keySet());
-			Collections.sort(newNames);
-	
-			for(String w : newNames) { //map.keySet()) {
-					count = map.get(w);
-					++rcdIndex;
-					
-					// output (w, count, rcdIndex)
-					if(br != null)		
-						br.write(UtilFunctions.quote(w) + TfUtils.TXMTD_SEP + rcdIndex + TfUtils.TXMTD_SEP + count  + "\n");
-					
-					if(maxCount < count) {
-						maxCount = count;
-						mode = w;
-						modeIndex = rcdIndex;
-					}
-					
-					// Replace count with recode index (useful when invoked from CP)
-					map.put(w, (long)rcdIndex);
-			}
-		}
-		finally {
-			IOUtilFunctions.closeSilently(br);
-		}
-		
-		if ( mode == null ) {
-			mode = "";
-			maxCount = 0;
-		}
-		
-		if ( isRecoded ) 
-		{
-			// output mode
-			pt=new Path(outputDir+"/Recode/"+ agents.getName(colID) + TfUtils.MODE_FILE_SUFFIX);
-			try(BufferedWriter br2=new BufferedWriter(new OutputStreamWriter(fs.create(pt,true))) ) {
-				br2.write(UtilFunctions.quote(mode) + "," + modeIndex + "," + maxCount );
-			}
-		
-			// output number of distinct values
-			pt=new Path(outputDir+"/Recode/"+ agents.getName(colID) + TfUtils.TXMTD_RCD_DISTINCT_SUFFIX);
-			try(BufferedWriter br2=new BufferedWriter(new OutputStreamWriter(fs.create(pt,true))) ) {
-				br2.write(""+map.size());
-			}
-		}
-		
-		if (isModeImputed) 
-		{
-			pt=new Path(outputDir+"/Impute/"+ agents.getName(colID) + TfUtils.TXMTD_MV_FILE_SUFFIX);
-			try( BufferedWriter br2=new BufferedWriter(new OutputStreamWriter(fs.create(pt,true)))) {
-				br2.write(colID + "," + UtilFunctions.quote(mode));
-			}
-		}
-		
-	}
-	
-	public void outputTransformationMetadata(String outputDir, FileSystem fs, TfUtils agents) throws IOException {
-		if(_colList == null && _mvrcdList == null )
-			return;
-		
-		for(int i=0; i<_fullrcdList.length; i++) {
-			int colID = _fullrcdList[i];
-			writeMetadata(_rcdMaps.get(colID), outputDir, colID, fs, agents, true);
-		}
-	}
-	
-	/** 
-	 * Method to merge map output transformation metadata.
-	 */
-	@Override
-	public void mergeAndOutputTransformationMetadata(Iterator<DistinctValue> values, String outputDir, int colID, FileSystem fs, TfUtils agents) throws IOException {
-		HashMap<String, Long> map = new HashMap<String,Long>();
-		
-		DistinctValue d = new DistinctValue();
-		String word = null;
-		Long count = null, val = null;
-		while(values.hasNext()) {
-			d.reset();
-			d = values.next();
-			
-			word = d.getWord();
-			count = d.getCount();
-			
-			val = map.get(word);
-			if(val == null) 
-				map.put(word, count);
-			else 
-				map.put(word, val+count);
-		}
-		
-		writeMetadata(map, outputDir, colID, fs, agents, false);
-	}
-	
-	/**
-	 * Method to load recode maps of all attributes, at once.
-	 */
-	@Override
-	public void loadTxMtd(JobConf job, FileSystem fs, Path txMtdDir, TfUtils agents) throws IOException {
-		if( !isApplicable() )
-			return;
-		
-		_finalMaps = new HashMap<Integer, HashMap<String, String>>();
-	
-		if(fs.isDirectory(txMtdDir)) {
-			for(int i=0; i<_colList.length;i++) {
-				int colID = _colList[i];
-				
-				Path path = new Path( txMtdDir + "/Recode/" + agents.getName(colID) + TfUtils.TXMTD_RCD_MAP_SUFFIX);
-				TfUtils.checkValidInputFile(fs, path, true); 
-				
-				HashMap<String,String> map = new HashMap<String,String>();
-				Pair<String,String> pair = new Pair<String,String>();
-				
-				String line = null;
-				try( BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(path))) ) {
-					// Example line to parse: "WN (1)67492",1,61975
-					while((line=br.readLine())!=null) {
-						DecoderRecode.parseRecodeMapEntry(line, pair);
-						map.put(pair.getKey(), pair.getValue());
-					}
-				}
-				_finalMaps.put(colID, map);
-			}
-		}
-		else {
-			throw new RuntimeException("Path to recode maps must be a directory: " + txMtdDir);
-		}
-	}	
-
-	private String lookupRCDMap(int colID, String key) {
-		if( _finalMaps!=null )
-			return _finalMaps.get(colID).get(key);
-		else { //used for cp
-			Long tmp = _rcdMaps.get(colID).get(key);
-			return (tmp!=null) ? Long.toString(tmp) : null;
-		}
-	}
-	
-
-	@Override
-	public MatrixBlock encode(FrameBlock in, MatrixBlock out) {
-		if( !isApplicable() )
-			return out;
-		
-		//build and apply recode maps 
-		build(in);
-		apply(in, out);
-		
-		return out;
-	}
-
-	@Override
-	public void build(FrameBlock in) {
-		if( !isApplicable() )
-			return;		
-
-		Iterator<String[]> iter = in.getStringRowIterator();
-		while( iter.hasNext() ) {
-			String[] row = iter.next(); 
-			for( int j=0; j<_colList.length; j++ ) {
-				int colID = _colList[j]; //1-based
-				//allocate column map if necessary
-				if( !_rcdMaps.containsKey(colID) ) 
-					_rcdMaps.put(colID, new HashMap<String,Long>());
-				//probe and build column map
-				HashMap<String,Long> map = _rcdMaps.get(colID);
-				String key = row[colID-1];
-				if( key!=null && !key.isEmpty() && !map.containsKey(key) )
-					map.put(key, Long.valueOf(map.size()+1));
-			}
-		}
-	}
-
-	public void buildPartial(FrameBlock in) {
-		if( !isApplicable() )
-			return;		
-
-		//ensure allocated partial recode map
-		if( _rcdMapsPart == null )
-			_rcdMapsPart = new HashMap<Integer, HashSet<Object>>();
-		
-		//construct partial recode map (tokens w/o codes)
-		//iterate over columns for sequential access
-		for( int j=0; j<_colList.length; j++ ) {
-			int colID = _colList[j]; //1-based
-			//allocate column map if necessary
-			if( !_rcdMapsPart.containsKey(colID) ) 
-				_rcdMapsPart.put(colID, new HashSet<Object>());
-			HashSet<Object> map = _rcdMapsPart.get(colID);
-			//probe and build column map
-			for( int i=0; i<in.getNumRows(); i++ )
-				map.add(in.get(i, colID-1));
-			//cleanup unnecessary entries once
-			map.remove(null);
-			map.remove("");
-		}
-	}
-	
-	/**
-	 * Method to apply transformations.
-	 */
-	@Override
-	public String[] apply(String[] words) 
-	{
-		if( !isApplicable() )
-			return words;
-		
-		//apply recode maps on relevant columns of given row
-		for(int i=0; i < _colList.length; i++) {
-			//prepare input and get code
-			int colID = _colList[i];
-			String key = UtilFunctions.unquote(words[colID-1].trim());
-			String val = lookupRCDMap(colID, key);			
-			// replace unseen keys with NaN 
-			words[colID-1] = (val!=null) ? val : "NaN";
-		}
-			
-		return words;
-	}
-	
-	@Override
-	public MatrixBlock apply(FrameBlock in, MatrixBlock out) {
-		//apply recode maps column wise
-		for( int j=0; j<_colList.length; j++ ) {
-			int colID = _colList[j];
-			for( int i=0; i<in.getNumRows(); i++ ) {
-				Object okey = in.get(i, colID-1);
-				String key = (okey!=null) ? okey.toString() : null;
-				String val = lookupRCDMap(colID, key);			
-				out.quickSetValue(i, colID-1, (val!=null) ? 
-						Double.parseDouble(val) : Double.NaN);
-			}
-		}
-		
-		return out;
-	}
-
-	@Override
-	public FrameBlock getMetaData(FrameBlock meta) {
-		if( !isApplicable() )
-			return meta;
-		
-		//inverse operation to initRecodeMaps
-		
-		//allocate output rows
-		int maxDistinct = 0;
-		for( int j=0; j<_colList.length; j++ )
-			if( _rcdMaps.containsKey(_colList[j]) )
-				maxDistinct = Math.max(maxDistinct, _rcdMaps.get(_colList[j]).size());
-		meta.ensureAllocatedColumns(maxDistinct);
-		
-		//create compact meta data representation
-		for( int j=0; j<_colList.length; j++ ) {
-			int colID = _colList[j]; //1-based
-			int rowID = 0;
-			if( _rcdMaps.containsKey(_colList[j]) )
-				for( Entry<String, Long> e : _rcdMaps.get(colID).entrySet() ) {
-					String tmp = constructRecodeMapEntry(e.getKey(), e.getValue());
-					meta.set(rowID++, colID-1, tmp); 
-				}
-			meta.getColumnMetadata(colID-1).setNumDistinct(
-					_rcdMaps.get(colID).size());
-		}
-		
-		return meta;
-	}
-	
-
-	/**
-	 * Construct the recodemaps from the given input frame for all 
-	 * columns registered for recode.
-	 * 
-	 * @param meta frame block
-	 */
-	public void initMetaData( FrameBlock meta ) {
-		if( meta == null || meta.getNumRows()<=0 )
-			return;
-		
-		for( int j=0; j<_colList.length; j++ ) {
-			int colID = _colList[j]; //1-based
-			_rcdMaps.put(colID, meta.getRecodeMap(colID-1));
-		}
-	}
-	
-	/**
-	 * Returns the Recode map entry which consists of concatenation of code, delimiter and token. 
-	 * @param token	is part of Recode map
-	 * @param code  is code for token 
-	 * @return the concatenation of code and token with delimiter in between
-	 */
-	public static String constructRecodeMapEntry(String token, Long code) {
-		return token + Lop.DATATYPE_PREFIX + code.toString();
-	}
-}
- 
\ No newline at end of file