You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by mb...@apache.org on 2016/03/22 18:42:07 UTC

[3/3] incubator-systemml git commit: [SYSTEMML-583] Read utils for transform metadata into frames, incl tests

[SYSTEMML-583] Read utils for transform metadata into frames, incl tests

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

Branch: refs/heads/master
Commit: 2d3672e6c34c344507433ba5a38d960981df234b
Parents: ca3bc2e
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Mon Mar 21 21:36:40 2016 -0700
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Tue Mar 22 10:37:14 2016 -0700

----------------------------------------------------------------------
 .../org/apache/sysml/api/jmlc/Connection.java   | 103 +++++++++++
 .../sysml/runtime/matrix/data/FrameBlock.java   |  10 +-
 .../sysml/runtime/util/DataConverter.java       |  16 +-
 .../functions/jmlc/FrameReadMetaTest.java       | 181 +++++++++++++++++++
 .../jmlc/tfmtd_example/Bin/saleprice.bin        |   1 +
 .../functions/jmlc/tfmtd_example/Bin/sqft.bin   |   1 +
 .../tfmtd_example/Dummycode/dummyCodeMaps.csv   |   9 +
 .../tfmtd_example/Impute/askingprice.impute     |   1 +
 .../jmlc/tfmtd_example/Impute/district.impute   |   1 +
 .../jmlc/tfmtd_example/Impute/floors.impute     |   1 +
 .../tfmtd_example/Impute/numbathrooms.impute    |   1 +
 .../tfmtd_example/Impute/numbedrooms.impute     |   1 +
 .../jmlc/tfmtd_example/Impute/view.impute       |   1 +
 .../jmlc/tfmtd_example/Impute/zipcode.impute    |   1 +
 .../jmlc/tfmtd_example/Recode/district.map      |   4 +
 .../jmlc/tfmtd_example/Recode/district.mode     |   1 +
 .../tfmtd_example/Recode/district.ndistinct     |   1 +
 .../jmlc/tfmtd_example/Recode/floors.map        |   3 +
 .../jmlc/tfmtd_example/Recode/floors.mode       |   1 +
 .../jmlc/tfmtd_example/Recode/floors.ndistinct  |   1 +
 .../jmlc/tfmtd_example/Recode/numbathrooms.map  |   5 +
 .../jmlc/tfmtd_example/Recode/numbathrooms.mode |   1 +
 .../tfmtd_example/Recode/numbathrooms.ndistinct |   1 +
 .../jmlc/tfmtd_example/Recode/numbedrooms.map   |   7 +
 .../jmlc/tfmtd_example/Recode/numbedrooms.mode  |   1 +
 .../tfmtd_example/Recode/numbedrooms.ndistinct  |   1 +
 .../jmlc/tfmtd_example/Recode/view.map          |   2 +
 .../jmlc/tfmtd_example/Recode/view.mode         |   1 +
 .../jmlc/tfmtd_example/Recode/view.ndistinct    |   1 +
 .../jmlc/tfmtd_example/Recode/zipcode.map       |   5 +
 .../jmlc/tfmtd_example/Recode/zipcode.mode      |   1 +
 .../jmlc/tfmtd_example/Recode/zipcode.ndistinct |   1 +
 .../functions/jmlc/tfmtd_example/coltypes.csv   |   1 +
 .../functions/jmlc/tfmtd_example/column.names   |   1 +
 .../jmlc/tfmtd_example/dummycoded.column.names  |   1 +
 .../functions/jmlc/tfmtd_example/spec.json      |   1 +
 src/test/scripts/functions/jmlc/transform3.dml  |  29 +++
 37 files changed, 384 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/main/java/org/apache/sysml/api/jmlc/Connection.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/jmlc/Connection.java b/src/main/java/org/apache/sysml/api/jmlc/Connection.java
index bd764ac..dc3fb29 100644
--- a/src/main/java/org/apache/sysml/api/jmlc/Connection.java
+++ b/src/main/java/org/apache/sysml/api/jmlc/Connection.java
@@ -21,11 +21,15 @@ package org.apache.sysml.api.jmlc;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
+import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -38,17 +42,29 @@ import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.rewrite.ProgramRewriter;
 import org.apache.sysml.hops.rewrite.RewriteRemovePersistentReadWrite;
+import org.apache.sysml.lops.Lop;
 import org.apache.sysml.parser.AParserWrapper;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DMLTranslator;
+import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.Program;
 import org.apache.sysml.runtime.controlprogram.caching.CacheableData;
+import org.apache.sysml.runtime.io.IOUtilFunctions;
 import org.apache.sysml.runtime.io.MatrixReaderFactory;
 import org.apache.sysml.runtime.io.ReaderTextCell;
+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.transform.TransformationAgent;
+import org.apache.sysml.runtime.transform.TransformationAgent.TX_METHOD;
 import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.runtime.util.MapReduceTool;
+import org.apache.sysml.runtime.util.UtilFunctions;
+import org.apache.wink.json4j.JSONArray;
+import org.apache.wink.json4j.JSONObject;
+
+import scala.actors.threadpool.Arrays;
 
 /**
  * JMLC (Java Machine Learning Connector) API:
@@ -255,4 +271,91 @@ public class Connection
 		return ret;
 	}
 	
+	/**
+	 * 
+	 * @param spec
+	 * @param metapath
+	 * @return
+	 * @throws IOException 
+	 */
+	@SuppressWarnings("unchecked")
+	public FrameBlock readTransformMetaData(String spec, String metapath) 
+		throws IOException 
+	{
+		//read column names
+		String colStr = MapReduceTool.readStringFromHDFSFile(metapath+File.separator+"column.names");
+		List<String> colnames = Arrays.asList(IOUtilFunctions.split(colStr.trim(), ","));
+		
+		//read meta data (currently only recode supported, without parsing spec)
+		HashMap<String,String> meta = new HashMap<String,String>();
+		int rows = 0;
+		for( String colName : colnames ) {
+			String name = metapath+File.separator+"Recode"+File.separator+colName;
+			if( MapReduceTool.existsFileOnHDFS(name+".map") ) {
+				meta.put(colName, MapReduceTool.readStringFromHDFSFile(name+".map"));
+				String ndistinct = MapReduceTool.readStringFromHDFSFile(name+".ndistinct");
+				rows = Math.max(rows, Integer.parseInt(ndistinct));
+			}
+		}
+		
+		//create frame block from in-memory strings
+		return readTransformMetaData(spec, rows, colnames, meta);
+	}
+	
+	/**
+	 * 
+	 * @param spec
+	 * @param meta
+	 * @return
+	 * @throws IOException 
+	 */
+	public FrameBlock readTransformMetaData(String spec, int rows, List<String> colnames, HashMap<String,String> meta) 
+		throws IOException 
+	{
+		//create frame block w/ pure string schema
+		List<ValueType> schema = Collections.nCopies(colnames.size(), ValueType.STRING);
+		FrameBlock ret = new FrameBlock(schema, colnames);
+		ret.ensureAllocatedColumns(rows);
+		
+		try
+		{
+			ArrayList<Integer> specRecodeIDs = new ArrayList<Integer>();
+			
+			//parse json transform specification
+			JSONObject jSpec = new JSONObject(spec);
+			if ( jSpec.containsKey(TX_METHOD.RECODE.toString()))  {
+				JSONArray attrs = null; //TODO simplify once json spec consolidated
+				if( jSpec.get(TX_METHOD.RECODE.toString()) instanceof JSONObject ) {
+					JSONObject obj = (JSONObject) jSpec.get(TX_METHOD.RECODE.toString());
+					attrs = (JSONArray) obj.get(TransformationAgent.JSON_ATTRS);
+				}
+				else
+					attrs = (JSONArray)jSpec.get(TX_METHOD.RECODE.toString());				
+				for(int j=0; j<attrs.length(); j++) 
+					specRecodeIDs.add(UtilFunctions.toInt(attrs.get(j)));
+			}	
+			
+			//encode recode maps into frame
+			for( Integer colID : specRecodeIDs ) {
+				String name = colnames.get(colID-1);
+				String map = meta.get(name);
+				if( map == null )
+					throw new IOException("Recode map for column '"+name+"' (id="+colID+") not existing.");
+				
+				InputStream is = new ByteArrayInputStream(map.getBytes("UTF-8"));
+				BufferedReader br = new BufferedReader(new InputStreamReader(is));
+				String line = null; int rpos = 0;
+				while( (line = br.readLine()) != null ) {
+					String parts[] = IOUtilFunctions.split(line.trim(), ",");
+					String pair = parts[0] + Lop.DATATYPE_PREFIX + parts[1]; //sval.code
+					ret.set(rpos++, colID-1, pair);
+				}
+			}
+		}
+		catch(Exception ex) {
+			throw new IOException(ex);
+		}
+		
+		return ret;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/main/java/org/apache/sysml/runtime/matrix/data/FrameBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/FrameBlock.java b/src/main/java/org/apache/sysml/runtime/matrix/data/FrameBlock.java
index 2fc7bbb..34e380c 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/FrameBlock.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/FrameBlock.java
@@ -27,6 +27,7 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -66,8 +67,7 @@ public class FrameBlock implements Writable, CacheBlock, Externalizable
 	
 	public FrameBlock(int ncols, ValueType vt) {
 		this();
-		for( int j=0; j<ncols; j++ )
-			_schema.add(vt);
+		_schema.addAll(Collections.nCopies(ncols, vt));
 	}
 	
 	public FrameBlock(List<ValueType> schema) {
@@ -573,8 +573,10 @@ public class FrameBlock implements Writable, CacheBlock, Externalizable
 		
 		@Override
 		public String[] next( ) {
-			for( int j=0; j<getNumColumns(); j++ )
-				_curRow[j] = get(_curPos, j).toString();
+			for( int j=0; j<getNumColumns(); j++ ) {
+				Object tmp = get(_curPos, j);
+				_curRow[j] = (tmp!=null) ? tmp.toString() : null;
+			}
 			_curPos++;			
 			return _curRow;
 		}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/main/java/org/apache/sysml/runtime/util/DataConverter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/util/DataConverter.java b/src/main/java/org/apache/sysml/runtime/util/DataConverter.java
index 2b642db..df3dcbe 100644
--- a/src/main/java/org/apache/sysml/runtime/util/DataConverter.java
+++ b/src/main/java/org/apache/sysml/runtime/util/DataConverter.java
@@ -22,6 +22,7 @@ package org.apache.sysml.runtime.util;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -675,12 +676,8 @@ public class DataConverter
 		if( data == null || data.length==0 )
 			return new FrameBlock();
 		
-		//construct temporary schema 
-		List<ValueType> schema = new ArrayList<ValueType>();
-		for( int j=0; j<data[0].length; j++ )
-			schema.add(ValueType.STRING);
-		
-		//create frame block
+		//create schema and frame block
+		List<ValueType> schema = Collections.nCopies(data[0].length, ValueType.STRING);
 		return convertToFrameBlock(data, schema);
 	}
 	
@@ -733,11 +730,8 @@ public class DataConverter
 	 * @return
 	 */
 	public static FrameBlock convertToFrameBlock(MatrixBlock mb, ValueType vt) {
-		//construct temporary schema 
-		List<ValueType> schema = new ArrayList<ValueType>();
-		for( int j=0; j<mb.getNumColumns(); j++ )
-			schema.add(vt);
-		
+		//create schema and frame block
+		List<ValueType> schema = Collections.nCopies(mb.getNumColumns(), vt);
 		return convertToFrameBlock(mb, schema);
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/java/org/apache/sysml/test/integration/functions/jmlc/FrameReadMetaTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/jmlc/FrameReadMetaTest.java b/src/test/java/org/apache/sysml/test/integration/functions/jmlc/FrameReadMetaTest.java
new file mode 100644
index 0000000..35a2e6e
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/jmlc/FrameReadMetaTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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.jmlc;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.sysml.api.jmlc.Connection;
+import org.apache.sysml.api.jmlc.PreparedScript;
+import org.apache.sysml.api.jmlc.ResultVariables;
+import org.apache.sysml.lops.Lop;
+import org.apache.sysml.runtime.matrix.data.FrameBlock;
+import org.apache.sysml.runtime.util.MapReduceTool;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.integration.TestConfiguration;
+
+/**
+ * 
+ * 
+ */
+public class FrameReadMetaTest extends AutomatedTestBase 
+{
+	private final static String TEST_NAME1 = "transform3";
+	private final static String TEST_DIR = "functions/jmlc/";
+	
+	private final static int rows = 300;
+	private final static int cols = 9;	
+	
+	
+	@Override
+	public void setUp() {
+		addTestConfiguration(TEST_NAME1, new TestConfiguration(TEST_DIR, TEST_NAME1, new String[] { "F2" }) ); 
+	}
+	
+	@Test
+	public void testJMLCTransformDense() throws IOException {
+		runJMLCReadMetaTest(TEST_NAME1, false);
+	}
+	
+	@Test
+	public void testJMLCTransformDenseReuse() throws IOException {
+		runJMLCReadMetaTest(TEST_NAME1, true);
+	}
+
+	/**
+	 * 
+	 * @param sparseM1
+	 * @param sparseM2
+	 * @param instType
+	 * @throws IOException 
+	 */
+	private void runJMLCReadMetaTest( String testname, boolean modelReuse ) 
+		throws IOException
+	{	
+		String TEST_NAME = testname;
+		
+		TestConfiguration config = getTestConfiguration(TEST_NAME);
+		loadTestConfiguration(config);
+	
+		//establish connection to SystemML
+		Connection conn = new Connection();
+		
+		//read meta data frame
+		String spec = MapReduceTool.readStringFromHDFSFile(SCRIPT_DIR + TEST_DIR+"tfmtd_example/spec.json");
+		FrameBlock M = conn.readTransformMetaData(spec, SCRIPT_DIR + TEST_DIR+"tfmtd_example/");
+		
+		//generate data based on recode maps
+		HashMap<String,Long>[] RC = getRecodeMaps(M);
+		double[][] X = generateData(rows, cols, RC);
+		String[][] F = null;
+		
+		try
+		{
+			//prepare input arguments
+			HashMap<String,String> args = new HashMap<String,String>();
+			args.put("$TRANSFORM_SPEC", spec);
+			
+			//read and precompile script
+			String script = conn.readScript(SCRIPT_DIR + TEST_DIR + testname + ".dml");	
+			PreparedScript pstmt = conn.prepareScript(script, args, new String[]{"X","M"}, new String[]{"F"}, false);
+			
+			if( modelReuse )
+				pstmt.setFrame("M", M, true);
+			
+			//execute script multiple times (2 runs)
+			for( int i=0; i<2; i++ )
+			{
+				//bind input parameters
+				if( !modelReuse )
+					pstmt.setFrame("M", M, false);
+				pstmt.setMatrix("X", X);
+				
+				//execute script
+				ResultVariables rs = pstmt.executeScript();
+				
+				//get output parameter
+				F = rs.getFrame("F");
+			}
+		}
+		catch(Exception ex) {
+			ex.printStackTrace();
+			throw new IOException(ex);
+		}
+		finally
+		{
+			if( conn != null )
+				conn.close();
+		}
+		
+		//check correct result 
+		//for all generated data, probe recode maps and compare versus output
+		for( int i=0; i<rows; i++ ) 
+			for( int j=0; j<cols; j++ ) 
+				if( RC[j] != null ) {
+					Assert.assertEquals("Wrong result: "+F[i][j]+".", 
+							Double.valueOf(X[i][j]), 
+							Double.valueOf(RC[j].get(F[i][j]).toString()));
+				}	
+	}
+
+	/**
+	 * 
+	 * @param M
+	 * @return
+	 */
+	@SuppressWarnings("unchecked")
+	private HashMap<String,Long>[] getRecodeMaps(FrameBlock M) {
+		HashMap<String,Long>[] ret = new HashMap[M.getNumColumns()];
+		Iterator<Object[]> iter = M.getObjectRowIterator();
+		while( iter.hasNext() ) {
+			Object[] tmp = iter.next();
+			for( int j=0; j<tmp.length; j++ ) 
+				if( tmp[j] != null ) {
+					if( ret[j] == null )
+						ret[j] = new HashMap<String,Long>();
+					String[] parts = tmp[j].toString().split(Lop.DATATYPE_PREFIX);
+					ret[j].put(parts[0], Long.parseLong(parts[1]));
+				}
+		}
+		
+		return ret;
+	}
+	
+	/**
+	 * 
+	 * @param rows
+	 * @param cols
+	 * @param RC
+	 * @return
+	 */
+	private double[][] generateData(int rows, int cols, HashMap<String,Long>[] RC) {
+		double[][] ret = new double[rows][cols];
+		for( int i=0; i<rows; i++ ) 
+			for( int j=0; j<cols; j++ ) 
+				if( RC[j] != null ) {
+					ret[i][j] = RC[j].values().toArray(new Long[0])[i%RC[j].size()];
+				}
+		
+		return ret;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Bin/saleprice.bin
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Bin/saleprice.bin b/src/test/scripts/functions/jmlc/tfmtd_example/Bin/saleprice.bin
new file mode 100644
index 0000000..eab0b65
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Bin/saleprice.bin
@@ -0,0 +1 @@
+8,478.0,1133.0,218.33333333333334,3

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Bin/sqft.bin
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Bin/sqft.bin b/src/test/scripts/functions/jmlc/tfmtd_example/Bin/sqft.bin
new file mode 100644
index 0000000..b96dfc2
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Bin/sqft.bin
@@ -0,0 +1 @@
+3,1031.0,3967.0,734.0,4

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Dummycode/dummyCodeMaps.csv
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Dummycode/dummyCodeMaps.csv b/src/test/scripts/functions/jmlc/tfmtd_example/Dummycode/dummyCodeMaps.csv
new file mode 100644
index 0000000..4b44783
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Dummycode/dummyCodeMaps.csv
@@ -0,0 +1,9 @@
+1,0,1,1
+2,1,2,5
+3,1,6,9
+4,0,10,10
+5,1,11,15
+6,1,16,18
+7,1,19,20
+8,1,21,23
+9,0,24,24

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Impute/askingprice.impute
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Impute/askingprice.impute b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/askingprice.impute
new file mode 100644
index 0000000..73d0007
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/askingprice.impute
@@ -0,0 +1 @@
+9,819.1306532663317

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Impute/district.impute
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Impute/district.impute b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/district.impute
new file mode 100644
index 0000000..ba68e08
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/district.impute
@@ -0,0 +1 @@
+2,south

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Impute/floors.impute
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Impute/floors.impute b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/floors.impute
new file mode 100644
index 0000000..732fa36
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/floors.impute
@@ -0,0 +1 @@
+6,1

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Impute/numbathrooms.impute
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Impute/numbathrooms.impute b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/numbathrooms.impute
new file mode 100644
index 0000000..5066c76
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/numbathrooms.impute
@@ -0,0 +1 @@
+5,1

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Impute/numbedrooms.impute
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Impute/numbedrooms.impute b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/numbedrooms.impute
new file mode 100644
index 0000000..39c0c3a
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/numbedrooms.impute
@@ -0,0 +1 @@
+4,2

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Impute/view.impute
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Impute/view.impute b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/view.impute
new file mode 100644
index 0000000..27570a1
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/view.impute
@@ -0,0 +1 @@
+7,"FALSE"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Impute/zipcode.impute
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Impute/zipcode.impute b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/zipcode.impute
new file mode 100644
index 0000000..3002026
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Impute/zipcode.impute
@@ -0,0 +1 @@
+1,"95141"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.map
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.map b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.map
new file mode 100644
index 0000000..6f092f5
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.map
@@ -0,0 +1,4 @@
+"east",1,39
+"north",2,46
+"south",3,63
+"west",4,52

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.mode
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.mode b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.mode
new file mode 100644
index 0000000..a8f28f5
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.mode
@@ -0,0 +1 @@
+"south",3,63
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.ndistinct
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.ndistinct b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.ndistinct
new file mode 100644
index 0000000..bf0d87a
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/district.ndistinct
@@ -0,0 +1 @@
+4
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.map
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.map b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.map
new file mode 100644
index 0000000..4a2b4de
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.map
@@ -0,0 +1,3 @@
+"1",1,78
+"2",2,66
+"3",3,56

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.mode
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.mode b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.mode
new file mode 100644
index 0000000..c4c00f9
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.mode
@@ -0,0 +1 @@
+"1",1,78
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.ndistinct
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.ndistinct b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.ndistinct
new file mode 100644
index 0000000..e440e5c
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/floors.ndistinct
@@ -0,0 +1 @@
+3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.map
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.map b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.map
new file mode 100644
index 0000000..7e62337
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.map
@@ -0,0 +1,5 @@
+"1",1,54
+"1.5",2,42
+"2",3,39
+"2.5",4,33
+"3",5,32

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.mode
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.mode b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.mode
new file mode 100644
index 0000000..389c785
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.mode
@@ -0,0 +1 @@
+"1",1,54
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.ndistinct
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.ndistinct b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.ndistinct
new file mode 100644
index 0000000..7813681
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbathrooms.ndistinct
@@ -0,0 +1 @@
+5
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.map
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.map b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.map
new file mode 100644
index 0000000..7046c6e
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.map
@@ -0,0 +1,7 @@
+"1",1,19
+"2",2,38
+"3",3,32
+"4",4,32
+"5",5,23
+"6",6,32
+"7",7,24

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.mode
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.mode b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.mode
new file mode 100644
index 0000000..97c003b
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.mode
@@ -0,0 +1 @@
+"2",2,38
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.ndistinct
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.ndistinct b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.ndistinct
new file mode 100644
index 0000000..c793025
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/numbedrooms.ndistinct
@@ -0,0 +1 @@
+7
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.map
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.map b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.map
new file mode 100644
index 0000000..4d97f0e
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.map
@@ -0,0 +1,2 @@
+"FALSE",1,94
+"TRUE",2,89

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.mode
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.mode b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.mode
new file mode 100644
index 0000000..653d895
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.mode
@@ -0,0 +1 @@
+"FALSE",1,94
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.ndistinct
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.ndistinct b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.ndistinct
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/view.ndistinct
@@ -0,0 +1 @@
+2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.map
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.map b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.map
new file mode 100644
index 0000000..ac6e504
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.map
@@ -0,0 +1,5 @@
+"91312",1,38
+"94555",2,39
+"95141",3,42
+"96334",4,37
+"98755",5,42

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.mode
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.mode b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.mode
new file mode 100644
index 0000000..096266b
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.mode
@@ -0,0 +1 @@
+"95141",3,42
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.ndistinct
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.ndistinct b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.ndistinct
new file mode 100644
index 0000000..7813681
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/Recode/zipcode.ndistinct
@@ -0,0 +1 @@
+5
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/coltypes.csv
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/coltypes.csv b/src/test/scripts/functions/jmlc/tfmtd_example/coltypes.csv
new file mode 100644
index 0000000..0b80246
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/coltypes.csv
@@ -0,0 +1 @@
+2,1,1,1,1,1,1,1,1,2,1,1,1,1,1,1,1,1,1,1,1,1,1,1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/column.names
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/column.names b/src/test/scripts/functions/jmlc/tfmtd_example/column.names
new file mode 100644
index 0000000..0600650
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/column.names
@@ -0,0 +1 @@
+zipcode,district,sqft,numbedrooms,numbathrooms,floors,view,saleprice,askingprice

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/dummycoded.column.names
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/dummycoded.column.names b/src/test/scripts/functions/jmlc/tfmtd_example/dummycoded.column.names
new file mode 100644
index 0000000..484898b
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/dummycoded.column.names
@@ -0,0 +1 @@
+zipcode\Q,\Edistrict_east\Q,\Edistrict_north\Q,\Edistrict_south\Q,\Edistrict_west\Q,\Esqft_Bin1\Q,\Esqft_Bin2\Q,\Esqft_Bin3\Q,\Esqft_Bin4\Q,\Enumbedrooms\Q,\Enumbathrooms_1\Q,\Enumbathrooms_1.5\Q,\Enumbathrooms_2\Q,\Enumbathrooms_2.5\Q,\Enumbathrooms_3\Q,\Efloors_1\Q,\Efloors_2\Q,\Efloors_3\Q,\Eview_FALSE\Q,\Eview_TRUE\Q,\Esaleprice_Bin1\Q,\Esaleprice_Bin2\Q,\Esaleprice_Bin3\Q,\Easkingprice

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/tfmtd_example/spec.json
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/tfmtd_example/spec.json b/src/test/scripts/functions/jmlc/tfmtd_example/spec.json
new file mode 100644
index 0000000..9b97811
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/tfmtd_example/spec.json
@@ -0,0 +1 @@
+{"dummycode":{"attributes":[2,3,5,6,7,8]},"recode":{"attributes":[1,2,4,5,6,7]},"bin":{"numbins":[4,3],"methods":[1,1],"attributes":[3,8]},"impute":{"methods":[2,3,3,3,3,2,1],"attributes":[1,2,4,5,6,7,9],"constants":[null,"south","2","1","1",null,null]}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2d3672e6/src/test/scripts/functions/jmlc/transform3.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/jmlc/transform3.dml b/src/test/scripts/functions/jmlc/transform3.dml
new file mode 100644
index 0000000..6bdcc63
--- /dev/null
+++ b/src/test/scripts/functions/jmlc/transform3.dml
@@ -0,0 +1,29 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+
+X = read($X); #new data
+M = read($M, data_type="frame", format="csv"); #existing recode maps
+specJson = $TRANSFORM_SPEC
+
+F = transformdecode(target=X, meta=M, spec=specJson);
+
+write(F, $F);
+